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/09/06 21:29:37 UTC

svn commit: r1381725 - in /incubator/oozie/trunk: ./ core/src/main/java/org/apache/oozie/service/ core/src/test/java/org/apache/oozie/command/bundle/ core/src/test/java/org/apache/oozie/command/coord/ core/src/test/java/org/apache/oozie/command/wf/ cor...

Author: virag
Date: Thu Sep  6 19:29:36 2012
New Revision: 1381725

URL: http://svn.apache.org/viewvc?rev=1381725&view=rev
Log:
OOZIE-975 Test cases should not load all the services classes (virag)

Modified:
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleRerunXCommand.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestPurgeXCommand.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPauseTransitService.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPurgeService.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java
    incubator/oozie/trunk/release-log.txt

Modified: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java (original)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java Thu Sep  6 19:29:36 2012
@@ -121,20 +121,20 @@ public class CoordMaterializeTriggerServ
                 CoordJobsToBeMaterializedJPAExecutor cmatcmd = new CoordJobsToBeMaterializedJPAExecutor(currDate,
                         materializationLimit);
                 List<CoordinatorJobBean> materializeJobs = jpaService.execute(cmatcmd);
-                LOG.debug("CoordMaterializeTriggerService - Curr Date= " + currDate + ", Num jobs to materialize = "
+                LOG.info("CoordMaterializeTriggerService - Curr Date= " + currDate + ", Num jobs to materialize = "
                         + materializeJobs.size());
                 for (CoordinatorJobBean coordJob : materializeJobs) {
                     Services.get().get(InstrumentationService.class).get()
                             .incr(INSTRUMENTATION_GROUP, INSTR_MAT_JOBS_COUNTER, 1);
                     int numWaitingActions = jpaService
                             .execute(new CoordActionsActiveCountJPAExecutor(coordJob.getId()));
-                    LOG.debug("Job :" + coordJob.getId() + "  numWaitingActions : " + numWaitingActions
+                    LOG.info("Job :" + coordJob.getId() + "  numWaitingActions : " + numWaitingActions
                             + " MatThrottle : " + coordJob.getMatThrottling());
                     // update lastModifiedTime so next time others might have higher chance to get pick up
                     coordJob.setLastModifiedTime(new Date());
                     jpaService.execute(new CoordJobUpdateJPAExecutor(coordJob));
                     if (numWaitingActions >= coordJob.getMatThrottling()) {
-                        LOG.debug("Materialization skipped for JobID [" + coordJob.getId() + " already waiting "
+                        LOG.info("info for JobID [" + coordJob.getId() + " already waiting "
                                 + numWaitingActions + " actions. MatThrottle is : " + coordJob.getMatThrottling());
                         continue;
                     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleRerunXCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleRerunXCommand.java?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleRerunXCommand.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleRerunXCommand.java Thu Sep  6 19:29:36 2012
@@ -186,6 +186,9 @@ public class TestBundleRerunXCommand ext
     public void testBundleRerunInPausedWithError() throws Exception {
         Date curr = new Date();
         Date pauseTime = new Date(curr.getTime() - 1000);
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
         BundleJobBean job = this.addRecordToBundleJobTableWithPausedTime(Job.Status.PAUSEDWITHERROR, false, pauseTime);
         this.addRecordToBundleActionTable(job.getId(), "action1", 0, Job.Status.FAILED);
         this.addRecordToBundleActionTable(job.getId(), "action2", 0, Job.Status.PAUSED);

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java Thu Sep  6 19:29:36 2012
@@ -21,6 +21,7 @@ import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.CoordinatorActionBean;
 import org.apache.oozie.CoordinatorJobBean;
 import org.apache.oozie.WorkflowJobBean;
@@ -37,15 +38,20 @@ import org.apache.oozie.service.SchemaSe
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.StatusTransitService;
 import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.workflow.WorkflowInstance;
 
 public class TestCoordKillXCommand extends XDataTestCase {
     private Services services;
+    private String[] excludedServices = { "org.apache.oozie.service.StatusTransitService",
+            "org.apache.oozie.service.PauseTransitService",
+            "org.apache.oozie.service.CoordMaterializeTriggerService", "org.apache.oozie.service.RecoveryService" };
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         services = new Services();
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         cleanUpDBTables();
     }
@@ -62,7 +68,11 @@ public class TestCoordKillXCommand exten
      * @throws Exception
      */
     public void testCoordKillSuccess1() throws Exception {
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, false, true, 0);
         CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.READY, "coord-action-get.xml", 0);
 
         JPAService jpaService = Services.get().get(JPAService.class);
@@ -89,7 +99,11 @@ public class TestCoordKillXCommand exten
      * @throws Exception
      */
     public void testCoordKillSuccess2() throws Exception {
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, false, true, 0);
         CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
 
         JPAService jpaService = Services.get().get(JPAService.class);
@@ -116,7 +130,7 @@ public class TestCoordKillXCommand exten
      * @throws Exception
      */
     public void testCoordKillFailedOnAction() throws Exception {
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, false, false);
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, false, true);
         CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
 
         JPAService jpaService = Services.get().get(JPAService.class);
@@ -146,12 +160,14 @@ public class TestCoordKillXCommand exten
     public void testCoordKillForBackwardSupport() throws Exception {
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_COORD_STATUS, "true");
-        new Services().init();
+        Services services = new Services();
+        setClassesToBeExcluded(services.getConf(), excludedServices);
+        services.init();
 
         JPAService jpaService = Services.get().get(JPAService.class);
         assertNotNull(jpaService);
 
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, false, false);
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, false, true);
         CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
 
         job.setAppNamespace(SchemaService.COORDINATOR_NAMESPACE_URI_1);
@@ -182,7 +198,7 @@ public class TestCoordKillXCommand exten
     public void testCoordKillFailed() throws Exception {
         final String testJobId = "0000001-" + new Date().getTime() + "-testCoordKill-C";
 
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, false, false);
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, false, true);
         CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.READY, "coord-action-get.xml", 0);
 
         JPAService jpaService = Services.get().get(JPAService.class);
@@ -209,7 +225,10 @@ public class TestCoordKillXCommand exten
      * @throws Exception
      */
     public void testCoordKillWaiting() throws Exception {
-        CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, false, true, 0);
         // Create a workflow job with RUNNING status
         WorkflowJobBean wfJob1 = this
                 .addRecordToWfJobTable(WorkflowJob.Status.RUNNING, WorkflowInstance.Status.RUNNING);
@@ -286,8 +305,10 @@ public class TestCoordKillXCommand exten
     }
 
     public void testCoordKillXCommandUniqueness() throws Exception {
-
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, false, false, 0);
 
         final MyCoordKillXCommand callable1 = new MyCoordKillXCommand(job.getId(), 100);
         final MyCoordKillXCommand callable2 = new MyCoordKillXCommand(job.getId(), 100);

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestPurgeXCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestPurgeXCommand.java?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestPurgeXCommand.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestPurgeXCommand.java Thu Sep  6 19:29: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.
@@ -50,11 +50,15 @@ import org.apache.oozie.workflow.lite.St
 
 public class TestPurgeXCommand extends XDataTestCase {
     private Services services;
+    private String[] excludedServices = { "org.apache.oozie.service.StatusTransitService",
+            "org.apache.oozie.service.PauseTransitService", "org.apache.oozie.service.PurgeService",
+            "org.apache.oozie.service.CoordMaterializeTriggerService", "org.apache.oozie.service.RecoveryService" };
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         services = new Services();
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         cleanUpDBTables();
     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java Thu Sep  6 19:29: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.
@@ -43,6 +43,7 @@ import org.apache.oozie.service.ActionCh
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.ActionService;
 import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.workflow.WorkflowInstance;
@@ -53,12 +54,17 @@ import org.apache.oozie.workflow.Workflo
 public class TestActionCheckerService extends XDataTestCase {
 
     private Services services;
+    private String[] excludedServices = {"org.apache.oozie.service.StatusTransitService",
+            "org.apache.oozie.service.PauseTransitService",
+            "org.apache.oozie.service.CoordMaterializeTriggerService", "org.apache.oozie.service.RecoveryService",
+            "org.apache.oozie.service.ActionCheckerService"};
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         setSystemProperty(SchemaService.WF_CONF_EXT_SCHEMAS, "wf-ext-schema.xsd");
         services = new Services();
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         cleanUpDBTables();
         services.get(ActionService.class).register(ForTestingActionExecutor.class);
@@ -70,6 +76,7 @@ public class TestActionCheckerService ex
         super.tearDown();
     }
 
+
     /**
      * Tests functionality of the Action Checker Service Runnable. </p> Starts
      * an action which behaves like an Async Action (Action and Job state set to
@@ -217,7 +224,10 @@ public class TestActionCheckerService ex
     public void testActionCheckerServiceCoord() throws Exception {
         final int actionNum = 1;
         final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
-        final CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        final CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, false, false, 0);
         final WorkflowJobBean wfJob = addRecordToWfJobTable(WorkflowJob.Status.SUCCEEDED,
                 WorkflowInstance.Status.SUCCEEDED);
         final CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), actionNum,

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPauseTransitService.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPauseTransitService.java?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPauseTransitService.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPauseTransitService.java Thu Sep  6 19:29: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.
@@ -19,6 +19,7 @@ package org.apache.oozie.service;
 
 import java.util.Date;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.BundleActionBean;
 import org.apache.oozie.BundleJobBean;
 import org.apache.oozie.CoordinatorJobBean;
@@ -35,14 +36,19 @@ import org.apache.oozie.service.JPAServi
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.PauseTransitService.PauseTransitRunnable;
 import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.util.DateUtils;
 
 public class TestPauseTransitService extends XDataTestCase {
     private Services services;
+    private String[] excludedServices = { "org.apache.oozie.service.StatusTransitService",
+            "org.apache.oozie.service.PauseTransitService",
+            "org.apache.oozie.service.CoordMaterializeTriggerService", "org.apache.oozie.service.RecoveryService" };
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         services = new Services();
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         cleanUpDBTables();
     }
@@ -53,6 +59,7 @@ public class TestPauseTransitService ext
         super.tearDown();
     }
 
+
     /**
      * Test : Pause a PREP bundle, then delay its pausetime to unpause it.
      *
@@ -157,8 +164,13 @@ public class TestPauseTransitService ext
 
         BundleActionBean bundleAction1 = this.addRecordToBundleActionTable(job.getId(), "action1", 0, Job.Status.RUNNING);
         BundleActionBean bundleAction2 = this.addRecordToBundleActionTable(job.getId(), "action2", 0, Job.Status.RUNNING);
-        CoordinatorJobBean coordJob1 = addRecordToCoordJobTable("action1", CoordinatorJob.Status.RUNNING, false);
-        CoordinatorJobBean coordJob2 = addRecordToCoordJobTable("action2", CoordinatorJob.Status.RUNNING, false);
+
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        CoordinatorJobBean coordJob1 = addRecordToCoordJobTable("action1", CoordinatorJob.Status.RUNNING, start, end, false);
+        CoordinatorJobBean coordJob2 = addRecordToCoordJobTable("action2", CoordinatorJob.Status.RUNNING, start, end, false);
 
         coordJob1.setPauseTime(pauseTime);
         coordJob1.setBundleId(job.getId());
@@ -223,8 +235,13 @@ public class TestPauseTransitService ext
 
         BundleActionBean bundleAction1 = this.addRecordToBundleActionTable(job.getId(), "action1", 0, Job.Status.PAUSED);
         BundleActionBean bundleAction2 = this.addRecordToBundleActionTable(job.getId(), "action2", 0, Job.Status.PAUSED);
-        CoordinatorJobBean coordJob1 = addRecordToCoordJobTable("action1", CoordinatorJob.Status.PAUSED, false);
-        CoordinatorJobBean coordJob2 = addRecordToCoordJobTable("action2", CoordinatorJob.Status.PAUSED, false);
+
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        CoordinatorJobBean coordJob1 = addRecordToCoordJobTable("action1", CoordinatorJob.Status.PAUSED, start, end, false);
+        CoordinatorJobBean coordJob2 = addRecordToCoordJobTable("action2", CoordinatorJob.Status.PAUSED, start, end, false);
 
         coordJob1.setPauseTime(null);
         coordJob1.setBundleId(job.getId());
@@ -291,8 +308,12 @@ public class TestPauseTransitService ext
 
         Date pauseTime = new Date(new Date().getTime() - 30 * 1000);
 
-        CoordinatorJobBean coordJob1 = addRecordToCoordJobTable("action1", CoordinatorJob.Status.RUNNING, false);
-        CoordinatorJobBean coordJob2 = addRecordToCoordJobTable("action2", CoordinatorJob.Status.RUNNING, false);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        CoordinatorJobBean coordJob1 = addRecordToCoordJobTable("action1", CoordinatorJob.Status.RUNNING, start, end, false);
+        CoordinatorJobBean coordJob2 = addRecordToCoordJobTable("action2", CoordinatorJob.Status.RUNNING, start, end, false);
 
         coordJob1.setAppNamespace(SchemaService.COORDINATOR_NAMESPACE_URI_1);
         coordJob1.setPauseTime(pauseTime);
@@ -378,9 +399,9 @@ public class TestPauseTransitService ext
         assertEquals(Job.Status.RUNNING, job.getStatus());
     }
 
-    protected CoordinatorJobBean addRecordToCoordJobTable(String coordId, CoordinatorJob.Status status, boolean pending)
-            throws Exception {
-        CoordinatorJobBean coordJob = createCoordJob(status, pending, false);
+    protected CoordinatorJobBean addRecordToCoordJobTable(String coordId, CoordinatorJob.Status status, Date start,
+            Date end, boolean pending) throws Exception {
+        CoordinatorJobBean coordJob = createCoordJob(status, start, end, pending, false, 0);
         coordJob.setId(coordId);
         coordJob.setAppName(coordId);
         try {

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPurgeService.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPurgeService.java?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPurgeService.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestPurgeService.java Thu Sep  6 19:29: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.
@@ -61,12 +61,16 @@ import org.apache.oozie.util.XConfigurat
  */
 public class TestPurgeService extends XDataTestCase {
     private Services services;
+    String[] excludedServices = { "org.apache.oozie.service.StatusTransitService",
+            "org.apache.oozie.service.PauseTransitService", "org.apache.oozie.service.PurgeService",
+            "org.apache.oozie.service.CoordMaterializeTriggerService", "org.apache.oozie.service.RecoveryService" };
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         setSystemProperty(SchemaService.WF_CONF_EXT_SCHEMAS, "wf-ext-schema.xsd");
         services = new Services();
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         services.get(ActionService.class).register(ForTestingActionExecutor.class);
     }
@@ -77,6 +81,7 @@ public class TestPurgeService extends XD
         super.tearDown();
     }
 
+
     /**
      * Tests the {@link org.apache.oozie.service.PurgeService}.
      * </p>
@@ -157,7 +162,11 @@ public class TestPurgeService extends XD
      * Calls the purge service, and ensure the job does not exist in the system.
      */
     public void testPurgeServiceForCoordinator() throws Exception {
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, false, false);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, start, end, false, false, 0);
+
         final String jobId = job.getId();
         CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.SUCCEEDED,
                 "coord-action-get.xml", 0);
@@ -294,8 +303,9 @@ public class TestPurgeService extends XD
     }
 
     @Override
-    protected CoordinatorJobBean addRecordToCoordJobTable(CoordinatorJob.Status status, boolean pending, boolean doneMatd) throws Exception {
-        CoordinatorJobBean coordJob = createCoordJob(status, pending, doneMatd);
+    protected CoordinatorJobBean addRecordToCoordJobTable(CoordinatorJob.Status status, Date start, Date end,
+            boolean pending, boolean doneMatd, int lastActionNum) throws Exception {
+        CoordinatorJobBean coordJob = createCoordJob(status, start, end, pending, doneMatd, lastActionNum);
         coordJob.setLastModifiedTime(DateUtils.parseDateOozieTZ("2009-12-18T01:00Z"));
         try {
             JPAService jpaService = Services.get().get(JPAService.class);

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=1381725&r1=1381724&r2=1381725&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 Thu Sep  6 19:29:36 2012
@@ -59,12 +59,16 @@ import org.apache.oozie.workflow.lite.St
 
 public class TestStatusTransitService extends XDataTestCase {
     private Services services;
+    private String[] excludedServices = { "org.apache.oozie.service.StatusTransitService",
+            "org.apache.oozie.service.PauseTransitService",
+            "org.apache.oozie.service.CoordMaterializeTriggerService", "org.apache.oozie.service.RecoveryService" };
+
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         cleanUpDBTables();
     }
@@ -75,24 +79,6 @@ public class TestStatusTransitService ex
         super.tearDown();
     }
 
-
-    // Exclude some of the services classes from loading so they dont interfere
-    // while the test case is running
-    private void setClassesToBeExcluded(Configuration conf) {
-        String classes = conf.get(Services.CONF_SERVICE_CLASSES);
-        StringBuilder builder = new StringBuilder(classes);
-        String[] excludedService = { "org.apache.oozie.service.StatusTransitService",
-                "org.apache.oozie.service.PauseTransitService",
-                "org.apache.oozie.service.CoordMaterializeTriggerService", "org.apache.oozie.service.RecoveryService" };
-        for (String s : excludedService) {
-            int index = builder.indexOf(s);
-            if (index != -1) {
-                builder.replace(index, index + s.length() + 1, "");
-            }
-        }
-        conf.set(Services.CONF_SERVICE_CLASSES, new String(builder));
-    }
-
     /**
      * Tests functionality of the StatusTransitService Runnable command. </p> Insert a coordinator job with RUNNING and
      * pending true and coordinator actions with pending false. Then, runs the StatusTransitService runnable and ensures
@@ -101,9 +87,9 @@ public class TestStatusTransitService ex
      * @throws Exception
      */
     public void testCoordStatusTransitServiceSucceeded() throws Exception {
-
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, 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);
@@ -127,9 +113,10 @@ public class TestStatusTransitService ex
      * @throws Exception
      */
     public void testCoordStatusTransitServiceDoneWithError() throws Exception {
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
 
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, true, 3);
         addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.KILLED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
@@ -158,11 +145,13 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_COORD_STATUS, "true");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
 
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
         CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, true, 3);
 
         final JPAService jpaService = Services.get().get(JPAService.class);
@@ -192,8 +181,10 @@ public class TestStatusTransitService ex
      */
     public void testCoordStatusTransitServiceKilledByUser1() throws Exception {
         final JPAService jpaService = Services.get().get(JPAService.class);
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
         CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, false, false,
                 1);
         WorkflowJobBean wfJob = addRecordToWfJobTable(WorkflowJob.Status.RUNNING, WorkflowInstance.Status.RUNNING);
@@ -248,8 +239,9 @@ public class TestStatusTransitService ex
      * @throws Exception
      */
     public void testCoordStatusTransitServiceKilledByUser2() throws Exception {
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.KILLED, start, end, true, false, 3);
         addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.KILLED, "coord-action-get.xml", 0);
@@ -279,8 +271,10 @@ public class TestStatusTransitService ex
      * @throws Exception
      */
     public void testCoordStatusTransitServiceSuspendedByUser() throws Exception {
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDateplusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDateplusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDateplusMonth);
+
         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);
@@ -311,8 +305,10 @@ public class TestStatusTransitService ex
      * @throws Exception
      */
     public void testCoordStatusTransitServiceSuspendedBottomUp() throws Exception {
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
         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);
@@ -349,10 +345,11 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, true, 4);
         addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.KILLED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.SUSPENDED, "coord-action-get.xml", 0);
@@ -390,8 +387,11 @@ public class TestStatusTransitService ex
     public void testCoordStatusTransitServiceSuspendAndResume() throws Exception {
         final JPAService jpaService = Services.get().get(JPAService.class);
         assertNotNull(jpaService);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
 
-        CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, true);
+        CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, false, true, 2);
         final String coordJobId = coordJob.getId();
 
         final CoordinatorActionBean coordAction1_1 = addRecordToCoordActionTable(coordJobId, 1,
@@ -439,8 +439,9 @@ public class TestStatusTransitService ex
      * @throws Exception
      */
     public void testCoordStatusTransitServiceRunning1() throws Exception {
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, false, 3);
         addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
@@ -473,8 +474,10 @@ public class TestStatusTransitService ex
      * @throws Exception
      */
     public void testCoordStatusTransitServiceRunning2() throws Exception {
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
         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);
@@ -510,10 +513,11 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "true");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         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);
@@ -549,10 +553,11 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         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);
@@ -588,10 +593,11 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         CoordinatorJobBean coordJob = createCoordJob(CoordinatorJob.Status.PAUSED, start, end, true, false, 3);
         // set some pause time explicity to make sure the job is not unpaused
         coordJob.setPauseTime(DateUtils.parseDateOozieTZ("2009-02-01T01:00Z"));
@@ -629,10 +635,11 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
 
         CoordinatorJobBean job = createCoordJob(CoordinatorJob.Status.PAUSEDWITHERROR, start, end, true, false, 3);
         // set the pause time explicity to make sure the job is not unpaused
@@ -670,8 +677,9 @@ public class TestStatusTransitService ex
      * @throws Exception
      */
     public void testCoordStatusTransitServiceForTimeout() throws Exception {
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, true, 3);
         addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.TIMEDOUT, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.TIMEDOUT, "coord-action-get.xml", 0);
@@ -749,8 +757,12 @@ public class TestStatusTransitService ex
         addRecordToBundleActionTable(bundleId, "action1", 0, Job.Status.RUNNING);
         addRecordToBundleActionTable(bundleId, "action2", 0, Job.Status.RUNNING);
 
-        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, true, true, 2);
-        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, true, true, 2);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, start, end, true, true, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, start, end, true, true, 2);
 
         addRecordToCoordActionTable("action1", 1, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable("action1", 2, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
@@ -787,8 +799,12 @@ public class TestStatusTransitService ex
         addRecordToBundleActionTable(bundleId, "action1", 1, Job.Status.RUNNING);
         addRecordToBundleActionTable(bundleId, "action2", 0, Job.Status.RUNNING);
 
-        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, true, true, 2);
-        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, true, true, 2);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, start, end, true, true, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, start, end, true, true, 2);
 
         addRecordToCoordActionTable("action1", 1, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
         addRecordToCoordActionTable("action1", 2, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
@@ -828,8 +844,12 @@ 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, true, 2);
-        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, false, true, 2);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, start, end, false, true, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, start, end, false, true, 2);
 
         final CoordinatorActionBean coordAction1_1 = addRecordToCoordActionTable("action1", 1,
                 CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
@@ -908,8 +928,12 @@ public class TestStatusTransitService ex
         addRecordToBundleActionTable(bundleId, "action1", 0, Job.Status.DONEWITHERROR);
         addRecordToBundleActionTable(bundleId, "action2", 0, Job.Status.DONEWITHERROR);
 
-        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.DONEWITHERROR, false, true, 2);
-        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.DONEWITHERROR, false, true, 2);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.DONEWITHERROR, start, end, false, true, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.DONEWITHERROR, start, end, false, true, 2);
 
         new BundleKillXCommand(bundleId).call();
         waitFor(5 * 1000, new Predicate() {
@@ -952,7 +976,12 @@ public class TestStatusTransitService ex
         // Add a bundle action with no coordinator to make it fail
         addRecordToBundleActionTable(bundleId, null, 0, Job.Status.KILLED);
         addRecordToBundleActionTable(bundleId, "action2", 0, Job.Status.RUNNING);
-        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, true, true, 2);
+
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, start, end, true, true, 2);
         addRecordToCoordActionTable("action2", 1, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
 
         Runnable runnable = new StatusTransitRunnable();
@@ -982,7 +1011,7 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         BundleJobBean bundleJob = this.addRecordToBundleJobTable(Job.Status.RUNNING, true);
         final JPAService jpaService = Services.get().get(JPAService.class);
@@ -992,8 +1021,12 @@ public class TestStatusTransitService ex
         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, true, false, 2);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, start, end, false, true, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, start, end, true, false, 2);
 
         final CoordinatorActionBean coordAction1_1 = addRecordToCoordActionTable("action1", 1,
                 CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
@@ -1072,8 +1105,12 @@ public class TestStatusTransitService ex
         addRecordToBundleActionTable(bundleId, "action1", 1, Job.Status.SUSPENDED);
         addRecordToBundleActionTable(bundleId, "action2", 1, Job.Status.SUSPENDED);
 
-        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, false, false, 2);
-        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, false, false, 2);
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+
+        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, start, end, false, false, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, start, end, false, false, 2);
 
         final CoordinatorActionBean coordAction1_1 = addRecordToCoordActionTable("action1", 1,
                 CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
@@ -1145,7 +1182,7 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         BundleJobBean bundleJob = this.addRecordToBundleJobTable(Job.Status.RUNNING, true);
         final JPAService jpaService = Services.get().get(JPAService.class);
@@ -1179,7 +1216,7 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         BundleJobBean bundleJob = createBundleJob(Job.Status.PAUSED, true);
         bundleJob.setPauseTime(DateUtils.parseDateOozieTZ("2009-02-01T01:00Z"));
@@ -1215,7 +1252,7 @@ public class TestStatusTransitService ex
         Services.get().destroy();
         setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
         Services services = new Services();
-        setClassesToBeExcluded(services.getConf());
+        setClassesToBeExcluded(services.getConf(), excludedServices);
         services.init();
         BundleJobBean bundleJob = createBundleJob(Job.Status.PAUSEDWITHERROR, true);
         bundleJob.setPauseTime(DateUtils.parseDateOozieTZ("2009-02-01T01:00Z"));
@@ -1283,8 +1320,9 @@ public class TestStatusTransitService ex
         Runnable runnable = new StatusTransitRunnable();
         runnable.run();
 
-        Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
-        Date end = DateUtils.parseDateOozieTZ("2009-02-02T23:59Z");
+        String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
+        Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
+        Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
 
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, true, 3);
         // add a record with stale reference to coord job id

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java Thu Sep  6 19:29:36 2012
@@ -198,9 +198,47 @@ public abstract class XDataTestCase exte
      * @return coordinator job bean
      * @throws Exception
      */
-    protected CoordinatorJobBean addRecordToCoordJobTableWithBundle(String bundleId, String coordId, CoordinatorJob.Status status, boolean pending, boolean doneMatd, int lastActionNumber) throws Exception {
+    protected CoordinatorJobBean addRecordToCoordJobTableWithBundle(String bundleId, String coordId,
+            CoordinatorJob.Status status, boolean pending, boolean doneMatd, int lastActionNumber) throws Exception {
         CoordinatorJobBean coordJob = createCoordJob(status, pending, doneMatd);
         coordJob.setBundleId(bundleId);
+        // coord id and coord name are the same
+        coordJob.setId(coordId);
+        coordJob.setAppName(coordId);
+        coordJob.setLastActionNumber(lastActionNumber);
+        try {
+            JPAService jpaService = Services.get().get(JPAService.class);
+            assertNotNull(jpaService);
+            CoordJobInsertJPAExecutor coordInsertCmd = new CoordJobInsertJPAExecutor(coordJob);
+            jpaService.execute(coordInsertCmd);
+        }
+        catch (JPAExecutorException je) {
+            je.printStackTrace();
+            fail("Unable to insert the test coord job record to table");
+            throw je;
+        }
+
+        return coordJob;
+    }
+
+
+    /**
+     * Add coordinator job bean with bundle id info.
+     *
+     * @param bundleId bundle id
+     * @param coordId coord id and coord name
+     * @param status job status
+     * @param start start time
+     * @param end end time
+     * @param pending true if pending is true
+     * @param doneMatd true if doneMaterialization is true
+     * @param lastActionNumber last action number
+     * @return coordinator job bean
+     * @throws Exception
+     */
+    protected CoordinatorJobBean addRecordToCoordJobTableWithBundle(String bundleId, String coordId, CoordinatorJob.Status status, Date start, Date end, boolean pending, boolean doneMatd, int lastActionNumber) throws Exception {
+        CoordinatorJobBean coordJob = createCoordJob(status, start, end, pending, doneMatd, 0);
+        coordJob.setBundleId(bundleId);
         //coord id and coord name are the same
         coordJob.setId(coordId);
         coordJob.setAppName(coordId);
@@ -220,6 +258,8 @@ public abstract class XDataTestCase exte
         return coordJob;
     }
 
+
+
     /**
      * Create coord job bean
      *
@@ -1133,6 +1173,20 @@ public abstract class XDataTestCase exte
         }
     }
 
+   // Exclude some of the services classes from loading so they dont interfere
+   // while the test case is running
+   protected void setClassesToBeExcluded(Configuration conf, String[] excludedServices) {
+       String classes = conf.get(Services.CONF_SERVICE_CLASSES);
+       StringBuilder builder = new StringBuilder(classes);
+       for (String s : excludedServices) {
+           int index = builder.indexOf(s);
+           if (index != -1) {
+               builder.replace(index, index + s.length() + 1, "");
+           }
+       }
+       conf.set(Services.CONF_SERVICE_CLASSES, new String(builder));
+   }
+
     /**
      * Adds the db records for the Bulk Monitor tests
      */
@@ -1193,4 +1247,15 @@ public abstract class XDataTestCase exte
         jpaService.execute(actionInsert);
     }
 
+    /**
+     * Add a month to the current time
+     * @param incrementMonth
+     * @return
+     */
+    protected static String getCurrentDateafterIncrementingInMonths(int incrementMonth) {
+        Calendar currentDate = Calendar.getInstance();
+        currentDate.set(Calendar.MONTH, currentDate.get(Calendar.MONTH) + incrementMonth);
+        return DateUtils.formatDateOozieTZ(currentDate);
+    }
+
 }

Modified: incubator/oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/release-log.txt?rev=1381725&r1=1381724&r2=1381725&view=diff
==============================================================================
--- incubator/oozie/trunk/release-log.txt (original)
+++ incubator/oozie/trunk/release-log.txt Thu Sep  6 19:29:36 2012
@@ -1,5 +1,6 @@
 -- Oozie 3.4.0 release (trunk - unreleased)
 
+OOZIE-975 Test cases should not load all the services classes (virag)
 OOZIE-984 Allow EL Functions in Coordinator timeout (rkanter via tucu)
 OOZIE-972 Provide  EL function to append a string in each substring of another string separated by delimiter (kamrul via virag)
 OOZIE-977 NotificationXCommand (job.notification queue entry) should set a timeout in the HTTP connections it makes (tucu)