You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by mo...@apache.org on 2013/09/26 19:34:36 UTC

svn commit: r1526618 [2/2] - in /oozie/trunk: ./ core/src/main/java/org/apache/oozie/ core/src/main/java/org/apache/oozie/command/bundle/ core/src/main/java/org/apache/oozie/command/coord/ core/src/main/java/org/apache/oozie/command/wf/ core/src/main/j...

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java?rev=1526618&r1=1526617&r2=1526618&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java Thu Sep 26 17:34:35 2013
@@ -30,9 +30,10 @@ import org.apache.oozie.ErrorCode;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.command.CommandException;
-import org.apache.oozie.executor.jpa.BundleActionsGetJPAExecutor;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor;
 import org.apache.oozie.executor.jpa.BundleJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor.BundleActionQuery;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.test.XDataTestCase;
@@ -153,8 +154,8 @@ public class TestBundleJobSuspendXComman
 
         sleep(2000);
 
-        BundleActionsGetJPAExecutor bundleActionsGetCmd = new BundleActionsGetJPAExecutor(job.getId());
-        List<BundleActionBean> actions = jpaService.execute(bundleActionsGetCmd);
+        List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
 
         assertEquals(2, actions.size());
         assertNotNull(actions.get(0).getCoordId());
@@ -165,7 +166,8 @@ public class TestBundleJobSuspendXComman
         job = jpaService.execute(bundleJobGetCmd);
         assertEquals(Job.Status.SUSPENDED, job.getStatus());
 
-        actions = jpaService.execute(bundleActionsGetCmd);
+        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE,
+                job.getId());
 
         assertEquals(true, actions.get(0).isPending());
         assertEquals(true, actions.get(1).isPending());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java?rev=1526618&r1=1526617&r2=1526618&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java Thu Sep 26 17:34:35 2013
@@ -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.
@@ -30,7 +30,8 @@ import org.apache.oozie.ErrorCode;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.command.CommandException;
-import org.apache.oozie.executor.jpa.BundleActionsGetJPAExecutor;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor.BundleActionQuery;
 import org.apache.oozie.executor.jpa.BundleJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
 import org.apache.oozie.service.JPAService;
@@ -113,8 +114,8 @@ public class TestBundleKillXCommand exte
 
         sleep(2000);
 
-        BundleActionsGetJPAExecutor bundleActionsGetCmd = new BundleActionsGetJPAExecutor(job.getId());
-        List<BundleActionBean> actions = jpaService.execute(bundleActionsGetCmd);
+        List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
 
         assertEquals(2, actions.size());
         assertNotNull(actions.get(0).getCoordId());
@@ -125,7 +126,8 @@ public class TestBundleKillXCommand exte
         job = jpaService.execute(bundleJobGetCmd);
         assertEquals(Job.Status.KILLED, job.getStatus());
 
-        actions = jpaService.execute(bundleActionsGetCmd);
+        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE,
+                job.getId());
 
         assertEquals(true, actions.get(0).isPending());
         assertEquals(true, actions.get(1).isPending());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java?rev=1526618&r1=1526617&r2=1526618&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java Thu Sep 26 17:34:35 2013
@@ -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.
@@ -30,9 +30,10 @@ import org.apache.oozie.ErrorCode;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.command.CommandException;
-import org.apache.oozie.executor.jpa.BundleActionsGetJPAExecutor;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor;
 import org.apache.oozie.executor.jpa.BundleJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobInsertJPAExecutor;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor.BundleActionQuery;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.UUIDService;
@@ -79,8 +80,8 @@ public class TestBundleStartXCommand ext
 
         sleep(2000);
 
-        BundleActionsGetJPAExecutor bundleActionsGetExecutor = new BundleActionsGetJPAExecutor(job.getId());
-        List<BundleActionBean> actions = jpaService.execute(bundleActionsGetExecutor);
+        List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
 
         assertEquals(2, actions.size());
         assertEquals(true, actions.get(0).isCritical());
@@ -129,19 +130,22 @@ public class TestBundleStartXCommand ext
 
         sleep(2000);
 
-        final BundleActionsGetJPAExecutor bundleActionsGetExecutor = new BundleActionsGetJPAExecutor(job.getId());
-        List<BundleActionBean> actions = jpaService.execute(bundleActionsGetExecutor);
+        List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
         assertEquals(2, actions.size());
 
+        final String jobId = job.getId();
         waitFor(200000, new Predicate() {
             public boolean evaluate() throws Exception {
-                List<BundleActionBean> actions = jpaService.execute(bundleActionsGetExecutor);
+                List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                        BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, jobId);
                 return actions.get(0).getStatus().equals(Job.Status.RUNNING)
                         && actions.get(1).getStatus().equals(Job.Status.RUNNING);
             }
         });
 
-        actions = jpaService.execute(bundleActionsGetExecutor);
+        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE,
+                job.getId());
         assertEquals(Job.Status.RUNNING, actions.get(0).getStatus());
         assertEquals(true, actions.get(0).isCritical());
         assertEquals(job.getId(), actions.get(0).getBundleId());
@@ -171,8 +175,8 @@ public class TestBundleStartXCommand ext
 
         sleep(2000);
 
-        BundleActionsGetJPAExecutor bundleActionsGetExecutor = new BundleActionsGetJPAExecutor(job.getId());
-        List<BundleActionBean> actions = jpaService.execute(bundleActionsGetExecutor);
+        List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
 
         assertEquals(2, actions.size());
         assertEquals(true, actions.get(0).isCritical());
@@ -261,8 +265,8 @@ public class TestBundleStartXCommand ext
         job = jpaService.execute(bundleJobGetExecutor);
         assertEquals(job.getStatus(), Job.Status.RUNNING);
         sleep(2000);
-        BundleActionsGetJPAExecutor bundleActionsGetExecutor = new BundleActionsGetJPAExecutor(job.getId());
-        List<BundleActionBean> actions = jpaService.execute(bundleActionsGetExecutor);
+        List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
         assertNull(actions.get(0).getCoordId());
         assertEquals(Job.Status.FAILED, actions.get(0).getStatus());
         Runnable runnable = new StatusTransitRunnable();

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java?rev=1526618&r1=1526617&r2=1526618&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java Thu Sep 26 17:34:35 2013
@@ -179,6 +179,11 @@ public class TestCoordJobQueryExecutor e
         retBean = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_USER_APPNAME, bean.getId());
         assertEquals(bean.getUser(), retBean.getUser());
         assertEquals(bean.getAppName(), retBean.getAppName());
+        // GET_COORD_JOB_STATUS_PARENTID
+        retBean = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_STATUS_PARENTID, bean.getId());
+        assertEquals(bean.getBundleId(), retBean.getBundleId());
+        assertEquals(bean.getStatus(), retBean.getStatus());
+        assertEquals(bean.getId(), retBean.getId());
         // GET_COORD_JOB_INPUTCHECK
         retBean = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_INPUT_CHECK, bean.getId());
         assertEquals(bean.getUser(), retBean.getUser());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java?rev=1526618&r1=1526617&r2=1526618&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java Thu Sep 26 17:34:35 2013
@@ -195,7 +195,8 @@ public class TestWorkflowActionQueryExec
         WorkflowActionBean retBean;
 
         //GET_WORKFFLOW_ID_TYPE
-        retBean = WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION_ID_TYPE, bean.getId());
+        retBean = WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION_ID_TYPE_LASTCHECK,
+                bean.getId());
         assertEquals(bean.getId(), retBean.getId());
         assertEquals(bean.getType(), retBean.getType());
 

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobQueryExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobQueryExecutor.java?rev=1526618&r1=1526617&r2=1526618&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobQueryExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobQueryExecutor.java Thu Sep 26 17:34:35 2013
@@ -302,6 +302,11 @@ public class TestWorkflowJobQueryExecuto
         assertEquals(bean.getProtoActionConf(), retBean.getProtoActionConf());
         assertNull(retBean.getConf());
         assertNull(retBean.getSlaXml());
+
+        // GET_WORKFLOW_STATUS
+        retBean = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_STATUS, bean.getId());
+        assertEquals(bean.getId(), retBean.getId());
+        assertEquals(bean.getStatus(), retBean.getStatus());
     }
 
     public void testGetList() throws Exception {

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/sla/TestSLAEventGeneration.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/sla/TestSLAEventGeneration.java?rev=1526618&r1=1526617&r2=1526618&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/sla/TestSLAEventGeneration.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/sla/TestSLAEventGeneration.java Thu Sep 26 17:34:35 2013
@@ -474,6 +474,7 @@ public class TestSLAEventGeneration exte
         WorkflowJobBean wf = new WorkflowJobBean();
         wf.setId(action.getExternalId());
         wf.setStatus(WorkflowJob.Status.KILLED);
+        wf.setParentId(action.getId());
         jpa.execute(new WorkflowJobInsertJPAExecutor(wf));
         new CoordActionUpdateXCommand(wf).call();
         assertEquals(1, ehs.getEventQueue().size());

Modified: oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/trunk/release-log.txt?rev=1526618&r1=1526617&r2=1526618&view=diff
==============================================================================
--- oozie/trunk/release-log.txt (original)
+++ oozie/trunk/release-log.txt Thu Sep 26 17:34:35 2013
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1503 [DB optimization] revisit eagerLoadState at places (mona)
 OOZIE-1555 LauncherMapper to check for sys properties before opening files for action data (mona)
 OOZIE-1546 TestMapReduceActionExecutorUberJar.testMapReduceWithUberJarEnabled fails (rkanter)
 OOZIE-1545 RecoveryService keeps repeatedly queueing SuspendXCommand (rohini)