You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by ry...@apache.org on 2013/09/21 10:07:37 UTC

svn commit: r1525202 - in /oozie/trunk: ./ core/src/main/java/org/apache/oozie/ core/src/main/java/org/apache/oozie/command/coord/ core/src/main/java/org/apache/oozie/executor/jpa/ core/src/main/java/org/apache/oozie/service/ core/src/test/java/org/apa...

Author: ryota
Date: Sat Sep 21 08:07:37 2013
New Revision: 1525202

URL: http://svn.apache.org/r1525202
Log:
OOZIE-1547 Change Coordinator SELECT query to fetch only necessary columns and consolidate JPA Executors (ryota)

Removed:
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionsGetPendingJPAExecutor.java
Modified:
    oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
    oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowActionBean.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionReadyXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionsKillXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionQueryExecutor.java
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java
    oozie/trunk/core/src/main/java/org/apache/oozie/service/RecoveryService.java
    oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java
    oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsCountNotForPurgeFromParentIdJPAExecutor.java
    oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java
    oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java
    oozie/trunk/release-log.txt

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java Sat Sep 21 08:07:37 2013
@@ -51,7 +51,7 @@ import org.json.simple.JSONObject;
 
 @Entity
 @NamedQueries( {
-        @NamedQuery(name = "UPDATE_COORD_JOB", query = "update CoordinatorJobBean w set w.appName = :appName, w.appPath = :appPath,w.concurrency = :concurrency, w.conf = :conf, w.externalId = :externalId, w.frequency = :frequency, w.lastActionNumber = :lastActionNumber, w.timeOut = :timeOut, w.timeZone = :timeZone, w.createdTimestamp = :createdTime, w.endTimestamp = :endTime, w.execution = :execution, w.jobXml = :jobXml, w.lastActionTimestamp = :lastAction, w.lastModifiedTimestamp = :lastModifiedTime, w.nextMaterializedTimestamp = :nextMaterializedTime, w.origJobXml = :origJobXml, w.slaXml=:slaXml, w.startTimestamp = :startTime, w.statusStr = :status, w.timeUnitStr = :timeUnit where w.id = :id"),
+        @NamedQuery(name = "UPDATE_COORD_JOB", query = "update CoordinatorJobBean w set w.appName = :appName, w.appPath = :appPath,w.concurrency = :concurrency, w.conf = :conf, w.externalId = :externalId, w.frequency = :frequency, w.lastActionNumber = :lastActionNumber, w.timeOut = :timeOut, w.timeZone = :timeZone, w.createdTimestamp = :createdTime, w.endTimestamp = :endTime, w.execution = :execution, w.jobXml = :jobXml, w.lastActionTimestamp = :lastAction, w.lastModifiedTimestamp = :lastModifiedTime, w.nextMaterializedTimestamp = :nextMaterializedTime, w.origJobXml = :origJobXml, w.slaXml=:slaXml, w.startTimestamp = :startTime, w.statusStr = :status, w.timeUnitStr = :timeUnit, w.appNamespace = :appNamespace, w.bundleId = :bundleId where w.id = :id"),
 
         @NamedQuery(name = "UPDATE_COORD_JOB_STATUS", query = "update CoordinatorJobBean w set w.statusStr =:status, w.lastModifiedTimestamp = :lastModifiedTime where w.id = :id"),
 
@@ -83,6 +83,18 @@ import org.json.simple.JSONObject;
 
         @NamedQuery(name = "GET_COORD_JOB", query = "select OBJECT(w) from CoordinatorJobBean w where w.id = :id"),
 
+        @NamedQuery(name = "GET_COORD_JOB_USER_APPNAME", query = "select w.user, w.appName from CoordinatorJobBean w where w.id = :id"),
+
+        @NamedQuery(name = "GET_COORD_JOB_INPUT_CHECK", query = "select w.user, w.appName, w.statusStr, w.appNamespace from CoordinatorJobBean w where w.id = :id"),
+
+        @NamedQuery(name = "GET_COORD_JOB_ACTION_READY", query = "select w.id, w.user, w.group, w.appName, w.statusStr, w.execution, w.concurrency from CoordinatorJobBean w where w.id = :id"),
+
+        @NamedQuery(name = "GET_COORD_JOB_ACTION_KILL", query = "select w.id, w.user, w.group, w.appName, w.statusStr from CoordinatorJobBean w where w.id = :id"),
+
+        @NamedQuery(name = "GET_COORD_JOB_MATERIALIZE", query = "select w.id, w.user, w.group, w.appName, w.statusStr, w.frequency, w.matThrottling, w.timeOut, w.timeZone, w.startTimestamp, w.endTimestamp, w.pauseTimestamp, w.nextMaterializedTimestamp, w.lastActionTimestamp, w.lastActionNumber, w.doneMaterialization, w.bundleId, w.conf, w.jobXml from CoordinatorJobBean w where w.id = :id"),
+
+        @NamedQuery(name = "GET_COORD_JOB_SUSPEND_KILL", query = "select w.id, w.user, w.group, w.appName, w.statusStr, w.bundleId, w.appNamespace, w.doneMaterialization from CoordinatorJobBean w where w.id = :id"),
+
         @NamedQuery(name = "GET_COORD_JOBS_PENDING", query = "select OBJECT(w) from CoordinatorJobBean w where w.pending = 1 order by w.lastModifiedTimestamp"),
 
         @NamedQuery(name = "GET_COORD_JOBS_COUNT", query = "select count(w) from CoordinatorJobBean w"),
@@ -472,6 +484,13 @@ public class CoordinatorJobBean implemen
     }
 
     /**
+     * Set doneMaterialization
+     */
+    public void setDoneMaterialization(int i) {
+        this.doneMaterialization = i;
+    }
+
+    /**
      * Set doneMaterialization to false
      */
     public void resetDoneMaterialization() {

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowActionBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowActionBean.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowActionBean.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowActionBean.java Sat Sep 21 08:07:37 2013
@@ -107,7 +107,7 @@ import org.json.simple.JSONObject;
 
     @NamedQuery(name = "GET_ACTIONS_OF_WORKFLOW_FOR_UPDATE", query = "select OBJECT(a) from WorkflowActionBean a where a.wfId = :wfId order by a.startTimestamp"),
 
-    @NamedQuery(name = "GET_PENDING_ACTIONS", query = "select OBJECT(a) from WorkflowActionBean a where a.pending = 1 AND a.pendingAgeTimestamp < :pendingAge AND a.statusStr <> 'RUNNING'"),
+    @NamedQuery(name = "GET_PENDING_ACTIONS", query = "select a.id, a.wfId, a.statusStr, a.type, a.pendingAgeTimestamp from WorkflowActionBean a where a.pending = 1 AND a.pendingAgeTimestamp < :pendingAge AND a.statusStr <> 'RUNNING'"),
 
     @NamedQuery(name = "GET_RUNNING_ACTIONS", query = "select a.id from WorkflowActionBean a where a.pending = 1 AND a.statusStr = 'RUNNING' AND a.lastCheckTimestamp < :lastCheckTime"),
 

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java Sat Sep 21 08:07:37 2013
@@ -41,6 +41,8 @@ import org.apache.oozie.executor.jpa.Coo
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.service.CallableQueueService;
 import org.apache.oozie.service.EventHandlerService;
@@ -569,7 +571,8 @@ public class CoordActionInputCheckXComma
         }
         try {
             coordAction = jpaService.execute(new CoordActionGetForInputCheckJPAExecutor(actionId));
-            coordJob = jpaService.execute(new CoordJobGetJPAExecutor(coordAction.getJobId()));
+            coordJob = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_INPUT_CHECK,
+                    coordAction.getJobId());
         }
         catch (JPAExecutorException je) {
             throw new CommandException(je);

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionReadyXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionReadyXCommand.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionReadyXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionReadyXCommand.java Sat Sep 21 08:07:37 2013
@@ -30,6 +30,8 @@ import org.apache.oozie.executor.jpa.Coo
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
 import org.apache.oozie.executor.jpa.CoordJobGetReadyActionsJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetRunningActionsCountJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
@@ -151,7 +153,7 @@ public class CoordActionReadyXCommand ex
             throw new CommandException(ErrorCode.E0610);
         }
         try {
-            coordJob = jpaService.execute(new org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor(jobId));
+            coordJob = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_ACTION_READY, jobId);
         }
         catch (JPAExecutorException e) {
             throw new CommandException(e);

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionsKillXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionsKillXCommand.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionsKillXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionsKillXCommand.java Sat Sep 21 08:07:37 2013
@@ -36,6 +36,7 @@ import org.apache.oozie.executor.jpa.Bat
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.service.EventHandlerService;
@@ -83,7 +84,7 @@ public class CoordActionsKillXCommand ex
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
+                coordJob = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_ACTION_KILL, jobId);
                 LogUtils.setLogInfo(coordJob, logInfo);
                 coordActions = CoordUtils.getCoordActions(rangeType, coordJob.getId(), scope, true);
             }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java Sat Sep 21 08:07:37 2013
@@ -34,7 +34,7 @@ import org.apache.oozie.executor.jpa.Bat
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
 import org.apache.oozie.executor.jpa.CoordJobGetActionsNotCompletedJPAExecutor;
-import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.service.EventHandlerService;
@@ -82,7 +82,7 @@ public class CoordKillXCommand extends K
             jpaService = Services.get().get(JPAService.class);
 
             if (jpaService != null) {
-                this.coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
+                this.coordJob = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_SUSPEND_KILL, jobId);
                 //Get actions which are not succeeded, failed, timed out or killed
                 this.actionList = jpaService.execute(new CoordJobGetActionsNotCompletedJPAExecutor(jobId));
                 prevStatus = coordJob.getStatus();

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java Sat Sep 21 08:07:37 2013
@@ -42,7 +42,6 @@ import org.apache.oozie.coord.TimeUnit;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordActionsActiveCountJPAExecutor;
-import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
@@ -157,7 +156,7 @@ public class CoordMaterializeTransitionX
         }
 
         try {
-            coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
+            coordJob = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_MATERIALIZE, jobId);
             prevStatus = coordJob.getStatus();
         }
         catch (JPAExecutorException jex) {

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java Sat Sep 21 08:07:37 2013
@@ -36,6 +36,7 @@ import org.apache.oozie.executor.jpa.Bat
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
 import org.apache.oozie.executor.jpa.CoordJobGetActionsRunningJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.service.JPAService;
@@ -82,7 +83,8 @@ public class CoordSuspendXCommand extend
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.coordJob = jpaService.execute(new CoordJobGetJPAExecutor(this.jobId));
+                this.coordJob = CoordJobQueryExecutor.getInstance()
+                        .get(CoordJobQuery.GET_COORD_JOB_SUSPEND_KILL, this.jobId);
                 prevStatus = coordJob.getStatus();
             }
             else {

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionQueryExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionQueryExecutor.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionQueryExecutor.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionQueryExecutor.java Sat Sep 21 08:07:37 2013
@@ -43,7 +43,8 @@ public class CoordActionQueryExecutor ex
         UPDATE_COORD_ACTION_FOR_START,
         UPDATE_COORD_ACTION_FOR_MODIFIED_DATE,
         UPDATE_COORD_ACTION_RERUN,
-        GET_COORD_ACTION
+        GET_COORD_ACTION,
+        GET_COORD_ACTIVE_ACTIONS_COUNT_BY_JOBID
     };
 
     private static CoordActionQueryExecutor instance = new CoordActionQueryExecutor();

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java Sat Sep 21 08:07:37 2013
@@ -17,14 +17,21 @@
  */
 package org.apache.oozie.executor.jpa;
 
+import java.sql.Timestamp;
+import java.util.ArrayList;
 import java.util.List;
 import javax.persistence.EntityManager;
 import javax.persistence.Query;
 
+import org.apache.oozie.BinaryBlob;
 import org.apache.oozie.CoordinatorJobBean;
 import org.apache.oozie.ErrorCode;
+import org.apache.oozie.StringBlob;
+import org.apache.oozie.WorkflowJobBean;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.util.DateUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -47,7 +54,13 @@ public class CoordJobQueryExecutor exten
         UPDATE_COORD_JOB_STATUS_PENDING_TIME,
         UPDATE_COORD_JOB_MATERIALIZE,
         UPDATE_COORD_JOB_CHANGE,
-        GET_COORD_JOB
+        GET_COORD_JOB,
+        GET_COORD_JOB_USER_APPNAME,
+        GET_COORD_JOB_INPUT_CHECK,
+        GET_COORD_JOB_ACTION_READY,
+        GET_COORD_JOB_ACTION_KILL,
+        GET_COORD_JOB_MATERIALIZE,
+        GET_COORD_JOB_SUSPEND_KILL
     };
 
     private static CoordJobQueryExecutor instance = new CoordJobQueryExecutor();
@@ -96,6 +109,8 @@ public class CoordJobQueryExecutor exten
                 query.setParameter("startTime", cjBean.getStartTimestamp());
                 query.setParameter("status", cjBean.getStatus().toString());
                 query.setParameter("timeUnit", cjBean.getTimeUnitStr());
+                query.setParameter("appNamespace", cjBean.getAppNamespace());
+                query.setParameter("bundleId", cjBean.getBundleId());
                 query.setParameter("id", cjBean.getId());
                 break;
             case UPDATE_COORD_JOB_STATUS:
@@ -178,6 +193,12 @@ public class CoordJobQueryExecutor exten
         Query query = em.createNamedQuery(namedQuery.name());
         switch (namedQuery) {
             case GET_COORD_JOB:
+            case GET_COORD_JOB_USER_APPNAME:
+            case GET_COORD_JOB_INPUT_CHECK:
+            case GET_COORD_JOB_ACTION_READY:
+            case GET_COORD_JOB_ACTION_KILL:
+            case GET_COORD_JOB_MATERIALIZE:
+            case GET_COORD_JOB_SUSPEND_KILL:
                 query.setParameter("id", parameters[0]);
                 break;
             default:
@@ -194,16 +215,98 @@ public class CoordJobQueryExecutor exten
         int ret = jpaService.executeUpdate(namedQuery.name(), query, em);
         return ret;
     }
+    private CoordinatorJobBean constructBean(CoordJobQuery namedQuery, Object ret) throws JPAExecutorException {
+        CoordinatorJobBean bean;
+        Object[] arr;
+        switch (namedQuery) {
+            case GET_COORD_JOB:
+                bean = (CoordinatorJobBean) ret;
+                break;
+            case GET_COORD_JOB_USER_APPNAME:
+                bean = new CoordinatorJobBean();
+                arr = (Object[]) ret;
+                bean.setUser((String) arr[0]);
+                bean.setAppName((String) arr[1]);
+                break;
+            case GET_COORD_JOB_INPUT_CHECK:
+                bean = new CoordinatorJobBean();
+                arr = (Object[]) ret;
+                bean.setUser((String) arr[0]);
+                bean.setAppName((String) arr[1]);
+                bean.setStatusStr((String) arr[2]);
+                bean.setAppNamespace((String) arr[3]);
+                break;
+            case GET_COORD_JOB_ACTION_READY:
+                bean = new CoordinatorJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setUser((String) arr[1]);
+                bean.setGroup((String) arr[2]);
+                bean.setAppName((String) arr[3]);
+                bean.setStatusStr((String) arr[4]);
+                bean.setExecution((String) arr[5]);
+                bean.setConcurrency((Integer) arr[6]);
+                break;
+            case GET_COORD_JOB_ACTION_KILL:
+                bean = new CoordinatorJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setUser((String) arr[1]);
+                bean.setGroup((String) arr[2]);
+                bean.setAppName((String) arr[3]);
+                bean.setStatusStr((String) arr[4]);
+                break;
+            case GET_COORD_JOB_MATERIALIZE:
+                bean = new CoordinatorJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setUser((String) arr[1]);
+                bean.setGroup((String) arr[2]);
+                bean.setAppName((String) arr[3]);
+                bean.setStatusStr((String) arr[4]);
+                bean.setFrequency((String) arr[5]);
+                bean.setMatThrottling((Integer) arr[6]);
+                bean.setTimeout((Integer) arr[7]);
+                bean.setTimeZone((String) arr[8]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[9]));
+                bean.setEndTime(DateUtils.toDate((Timestamp) arr[10]));
+                bean.setPauseTime(DateUtils.toDate((Timestamp) arr[11]));
+                bean.setNextMaterializedTime(DateUtils.toDate((Timestamp) arr[12]));
+                bean.setLastActionTime(DateUtils.toDate((Timestamp) arr[13]));
+                bean.setLastActionNumber((Integer) arr[14]);
+                bean.setDoneMaterialization((Integer) arr[15]);
+                bean.setBundleId((String) arr[16]);
+                bean.setConfBlob((StringBlob) arr[17]);
+                bean.setJobXmlBlob((StringBlob) arr[18]);
+                break;
+            case GET_COORD_JOB_SUSPEND_KILL:
+                bean = new CoordinatorJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setUser((String) arr[1]);
+                bean.setGroup((String) arr[2]);
+                bean.setAppName((String) arr[3]);
+                bean.setStatusStr((String) arr[4]);
+                bean.setBundleId((String) arr[5]);
+                bean.setAppNamespace((String) arr[6]);
+                bean.setDoneMaterialization((Integer) arr[7]);
+                break;
+            default:
+                throw new JPAExecutorException(ErrorCode.E0603, "QueryExecutor cannot construct job bean for "
+                        + namedQuery.name());
+        }
+        return bean;
+    }
 
     @Override
     public CoordinatorJobBean get(CoordJobQuery namedQuery, Object... parameters) throws JPAExecutorException {
         EntityManager em = jpaService.getEntityManager();
         Query query = getSelectQuery(namedQuery, em, parameters);
-        @SuppressWarnings("unchecked")
-        CoordinatorJobBean bean = (CoordinatorJobBean) jpaService.executeGet(namedQuery.name(), query, em);
-        if (bean == null) {
+        Object ret = jpaService.executeGet(namedQuery.name(), query, em);
+        if (ret == null) {
             throw new JPAExecutorException(ErrorCode.E0604, query.toString());
         }
+        CoordinatorJobBean bean = constructBean(namedQuery, ret);
         return bean;
     }
 
@@ -211,11 +314,13 @@ public class CoordJobQueryExecutor exten
     public List<CoordinatorJobBean> getList(CoordJobQuery namedQuery, Object... parameters) throws JPAExecutorException {
         EntityManager em = jpaService.getEntityManager();
         Query query = getSelectQuery(namedQuery, em, parameters);
-        @SuppressWarnings("unchecked")
-        List<CoordinatorJobBean> beanList = (List<CoordinatorJobBean>) jpaService.executeGetList(namedQuery.name(),
-                query, em);
-        if (beanList == null || beanList.size() == 0) {
-            throw new JPAExecutorException(ErrorCode.E0604, query.toString());
+        List<?> retList = (List<?>) jpaService.executeGetList(namedQuery.name(), query, em);
+        List<CoordinatorJobBean> beanList = null;
+        if (retList != null) {
+            beanList = new ArrayList<CoordinatorJobBean>();
+            for (Object ret : retList) {
+                beanList.add(constructBean(namedQuery, ret));
+            }
         }
         return beanList;
     }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java Sat Sep 21 08:07:37 2013
@@ -57,7 +57,8 @@ public class WorkflowActionQueryExecutor
         GET_ACTION_END,
         GET_ACTION_KILL,
         GET_ACTION_COMPLETED,
-        GET_RUNNING_ACTIONS
+        GET_RUNNING_ACTIONS,
+        GET_PENDING_ACTIONS
     };
 
     private static WorkflowActionQueryExecutor instance = new WorkflowActionQueryExecutor();
@@ -216,9 +217,14 @@ public class WorkflowActionQueryExecutor
                 query.setParameter("id", parameters[0]);
                 break;
             case GET_RUNNING_ACTIONS:
-                Timestamp ts = new Timestamp(System.currentTimeMillis() - (Integer)parameters[0] * 1000);
+                Timestamp ts = new Timestamp(System.currentTimeMillis() - (Integer) parameters[0] * 1000);
                 query.setParameter("lastCheckTime", ts);
                 break;
+            case GET_PENDING_ACTIONS:
+                Long minimumPendingAgeSecs = (Long) parameters[0];
+                Timestamp pts = new Timestamp(System.currentTimeMillis() - minimumPendingAgeSecs * 1000);
+                query.setParameter("pendingAge", pts);
+                break;
             default:
                 throw new JPAExecutorException(ErrorCode.E0603, "QueryExecutor cannot set parameters for "
                         + namedQuery.name());
@@ -379,6 +385,15 @@ public class WorkflowActionQueryExecutor
                 bean = new WorkflowActionBean();
                 bean.setId((String)ret);
                 break;
+            case GET_PENDING_ACTIONS:
+                bean = new WorkflowActionBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setJobId((String) arr[1]);
+                bean.setStatusStr((String) arr[2]);
+                bean.setType((String) arr[3]);
+                bean.setPendingAge(DateUtils.toDate((Timestamp) arr[4]));
+                break;
             default:
                 throw new JPAExecutorException(ErrorCode.E0603, "QueryExecutor cannot construct action bean for "
                         + namedQuery.name());

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/service/RecoveryService.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/service/RecoveryService.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/service/RecoveryService.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/service/RecoveryService.java Sat Sep 21 08:07:37 2013
@@ -51,9 +51,11 @@ import org.apache.oozie.executor.jpa.Bun
 import org.apache.oozie.executor.jpa.BundleJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordActionsGetForRecoveryJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordActionsGetReadyGroupbyJobIDJPAExecutor;
-import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowActionsGetPendingJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.util.JobUtils;
 import org.apache.oozie.util.XCallable;
 import org.apache.oozie.util.XConfiguration;
@@ -255,8 +257,8 @@ public class RecoveryService implements 
                             }
                         }
                         else if (caction.getStatus() == CoordinatorActionBean.Status.SUBMITTED) {
-                            CoordinatorJobBean coordJob = jpaService
-                                    .execute(new CoordJobGetJPAExecutor(caction.getJobId()));
+                            CoordinatorJobBean coordJob = CoordJobQueryExecutor.getInstance().get(
+                                    CoordJobQuery.GET_COORD_JOB_USER_APPNAME, caction.getJobId());
                             queueCallable(new CoordActionStartXCommand(caction.getId(), coordJob.getUser(),
                                     coordJob.getAppName(), caction.getJobId()));
 
@@ -323,7 +325,8 @@ public class RecoveryService implements 
             // queue command for action recovery
             List<WorkflowActionBean> actions = null;
             try {
-                actions = jpaService.execute(new WorkflowActionsGetPendingJPAExecutor(olderThan));
+                actions = WorkflowActionQueryExecutor.getInstance().getList(WorkflowActionQuery.GET_PENDING_ACTIONS,
+                        olderThan);
             }
             catch (JPAExecutorException ex) {
                 log.warn("Exception while reading pending actions from storage", ex);

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=1525202&r1=1525201&r2=1525202&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 Sat Sep 21 08:07:37 2013
@@ -24,6 +24,7 @@ import org.apache.oozie.CoordinatorJobBe
 import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.service.JPAService;
+import org.apache.oozie.service.SchemaService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.test.XDataTestCase;
 
@@ -46,7 +47,7 @@ public class TestCoordJobQueryExecutor e
         super.tearDown();
     }
 
-    public void testGetQuery() throws Exception {
+    public void testGetUpdateQuery() throws Exception {
         EntityManager em = jpaService.getEntityManager();
         CoordinatorJobBean cjBean = addRecordToCoordJobTable(CoordinatorJob.Status.PREP, true, true);
 
@@ -73,6 +74,8 @@ public class TestCoordJobQueryExecutor e
         assertEquals(query.getParameterValue("startTime"), cjBean.getStartTimestamp());
         assertEquals(query.getParameterValue("status"), cjBean.getStatus().toString());
         assertEquals(query.getParameterValue("timeUnit"), cjBean.getTimeUnit().toString());
+        assertEquals(query.getParameterValue("appNamespace"), cjBean.getAppNamespace());
+        assertEquals(query.getParameterValue("bundleId"), cjBean.getBundleId());
         assertEquals(query.getParameterValue("id"), cjBean.getId());
 
         // UPDATE_COORD_JOB_STATUS
@@ -165,7 +168,88 @@ public class TestCoordJobQueryExecutor e
     }
 
     public void testGet() throws Exception {
-        // TODO
+        CoordinatorJobBean bean = addRecordToCoordJobTable(CoordinatorJob.Status.SUCCEEDED, false, true);
+        bean.setAppNamespace(SchemaService.COORDINATOR_NAMESPACE_URI_1);
+        bean.setBundleId("dummy-bundleid");
+        bean.setOrigJobXml("dummy-origjobxml");
+        bean.setSlaXml("<sla></sla>");
+        CoordJobQueryExecutor.getInstance().executeUpdate(CoordJobQuery.UPDATE_COORD_JOB, bean);
+        CoordinatorJobBean retBean;
+        // GET_COORD_JOB_USER_APPNAME
+        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_INPUTCHECK
+        retBean = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_INPUT_CHECK, bean.getId());
+        assertEquals(bean.getUser(), retBean.getUser());
+        assertEquals(bean.getAppName(), retBean.getAppName());
+        assertEquals(bean.getStatusStr(), retBean.getStatusStr());
+        assertEquals(bean.getAppNamespace(), retBean.getAppNamespace());
+        assertNull(retBean.getConf());
+        assertNull(retBean.getJobXmlBlob());
+        assertNull(retBean.getOrigJobXmlBlob());
+        assertNull(retBean.getSlaXmlBlob());
+        // GET_COORD_JOB_ACTION_READY
+        retBean = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_ACTION_READY, bean.getId());
+        assertEquals(bean.getId(), retBean.getId());
+        assertEquals(bean.getUser(), retBean.getUser());
+        assertEquals(bean.getGroup(), retBean.getGroup());
+        assertEquals(bean.getAppName(), retBean.getAppName());
+        assertEquals(bean.getStatusStr(), retBean.getStatusStr());
+        assertEquals(bean.getExecution(), retBean.getExecution());
+        assertEquals(bean.getConcurrency(), retBean.getConcurrency());
+        assertNull(retBean.getConf());
+        assertNull(retBean.getJobXmlBlob());
+        assertNull(retBean.getOrigJobXmlBlob());
+        assertNull(retBean.getSlaXmlBlob());
+        // GET_COORD_JOB_ACTION_KILL
+        retBean = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_ACTION_KILL, bean.getId());
+        assertEquals(bean.getId(), retBean.getId());
+        assertEquals(bean.getUser(), retBean.getUser());
+        assertEquals(bean.getGroup(), retBean.getGroup());
+        assertEquals(bean.getAppName(), retBean.getAppName());
+        assertEquals(bean.getStatusStr(), retBean.getStatusStr());
+        assertNull(retBean.getConf());
+        assertNull(retBean.getJobXmlBlob());
+        assertNull(retBean.getOrigJobXmlBlob());
+        assertNull(retBean.getSlaXmlBlob());
+        // GET_COORD_JOB_MATERIALIZE
+        retBean = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_MATERIALIZE, bean.getId());
+        assertEquals(bean.getId(), retBean.getId());
+        assertEquals(bean.getUser(), retBean.getUser());
+        assertEquals(bean.getGroup(), retBean.getGroup());
+        assertEquals(bean.getAppName(), retBean.getAppName());
+        assertEquals(bean.getStatusStr(), retBean.getStatusStr());
+        assertEquals(bean.getFrequency(), retBean.getFrequency());
+        assertEquals(bean.getMatThrottling(), retBean.getMatThrottling());
+        assertEquals(bean.getTimeout(), retBean.getTimeout());
+        assertEquals(bean.getTimeZone(), retBean.getTimeZone());
+        assertEquals(bean.getStartTime(), retBean.getStartTime());
+        assertEquals(bean.getEndTime(), retBean.getEndTime());
+        assertEquals(bean.getPauseTime(), retBean.getPauseTime());
+        assertEquals(bean.getNextMaterializedTime(), retBean.getNextMaterializedTime());
+        assertEquals(bean.getLastActionTime(), retBean.getLastActionTime());
+        assertEquals(bean.getLastActionNumber(), retBean.getLastActionNumber());
+        assertEquals(bean.isDoneMaterialization(), retBean.isDoneMaterialization());
+        assertEquals(bean.getBundleId(), retBean.getBundleId());
+        assertEquals(bean.getConf(), retBean.getConf());
+        assertEquals(bean.getJobXml(), retBean.getJobXml());
+        assertNull(retBean.getOrigJobXmlBlob());
+        assertNull(retBean.getSlaXmlBlob());
+        // GET_COORD_JOB_SUSPEND_KILL
+        retBean = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_SUSPEND_KILL, bean.getId());
+        assertEquals(bean.getId(), retBean.getId());
+        assertEquals(bean.getUser(), retBean.getUser());
+        assertEquals(bean.getGroup(), retBean.getGroup());
+        assertEquals(bean.getAppName(), retBean.getAppName());
+        assertEquals(bean.getStatusStr(), retBean.getStatusStr());
+        assertEquals(bean.getBundleId(), retBean.getBundleId());
+        assertEquals(bean.getAppNamespace(), retBean.getAppNamespace());
+        assertEquals(bean.isDoneMaterialization(), retBean.isDoneMaterialization());
+        assertNull(retBean.getConf());
+        assertNull(retBean.getJobXmlBlob());
+        assertNull(retBean.getOrigJobXmlBlob());
+        assertNull(retBean.getSlaXmlBlob());
     }
 
     public void testGetList() throws Exception {

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsCountNotForPurgeFromParentIdJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsCountNotForPurgeFromParentIdJPAExecutor.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsCountNotForPurgeFromParentIdJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsCountNotForPurgeFromParentIdJPAExecutor.java Sat Sep 21 08:07:37 2013
@@ -120,7 +120,7 @@ public class TestCoordJobsCountNotForPur
         CoordinatorJobBean coordJob7 = addRecordToCoordJobTable(CoordinatorJob.Status.PREMATER, false, false);
         coordJob5 = TestPurgeXCommand.setLastModifiedTime(coordJob7, "2009-06-01T01:00Z");
         coordJob5.setAppName("coord7");
-        CoordJobQueryExecutor.getInstance().executeUpdate(CoordJobQuery.UPDATE_COORD_JOB, coordJob6);
+        CoordJobQueryExecutor.getInstance().executeUpdate(CoordJobQuery.UPDATE_COORD_JOB, coordJob7);
         addRecordToBundleActionTable(bundleJobId, coordJob7.getId(), coordJob7.getAppName(), 0, Job.Status.PREMATER);
         days = TestPurgeXCommand.getNumDaysToNotBePurged(coordJob6.getLastModifiedTime());
         assertEquals(7, (long) jpaService.execute(new CoordJobsCountNotForPurgeFromParentIdJPAExecutor(days, bundleJobId)));

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=1525202&r1=1525201&r2=1525202&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 Sat Sep 21 08:07:37 2013
@@ -351,16 +351,32 @@ public class TestWorkflowActionQueryExec
     }
 
     public void testGetList() throws Exception {
-      //GET_RUNNING_ACTIONS
         addRecordToWfActionTable("wrkflow","1", WorkflowAction.Status.RUNNING, true);
         addRecordToWfActionTable("wrkflow","2", WorkflowAction.Status.RUNNING, true);
         addRecordToWfActionTable("wrkflow","3", WorkflowAction.Status.RUNNING, true);
+        addRecordToWfActionTable("wrkflow","4", WorkflowAction.Status.PREP, true);
+        addRecordToWfActionTable("wrkflow","5", WorkflowAction.Status.FAILED, true);
+        addRecordToWfActionTable("wrkflow","6", WorkflowAction.Status.FAILED, false);
+        //GET_RUNNING_ACTIONS
         List<WorkflowActionBean> retList = WorkflowActionQueryExecutor.getInstance().getList(
                 WorkflowActionQuery.GET_RUNNING_ACTIONS, 0);
         assertEquals(3, retList.size());
         for(WorkflowActionBean bean : retList){
             assertTrue(bean.getId().equals("wrkflow@1") || bean.getId().equals("wrkflow@2") || bean.getId().equals("wrkflow@3"));
         }
+        //GET_PENDING_ACTIONS
+        sleep(10);
+        long olderThan = 1;
+        retList = WorkflowActionQueryExecutor.getInstance().getList(
+                WorkflowActionQuery.GET_PENDING_ACTIONS, olderThan);
+        assertEquals(2, retList.size());
+        for(WorkflowActionBean bean : retList){
+            assertTrue(bean.getId().equals("wrkflow@4") || bean.getId().equals("wrkflow@5"));
+        }
+        olderThan = 10000;
+        retList = WorkflowActionQueryExecutor.getInstance().getList(
+                WorkflowActionQuery.GET_PENDING_ACTIONS, olderThan);
+        assertEquals(0, retList.size());
 
     }
 

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/test/XDataTestCase.java Sat Sep 21 08:07:37 2013
@@ -1203,10 +1203,10 @@ public abstract class XDataTestCase exte
         action.setCred("null");
         action.setStats("dummyStats");
         if (pending) {
-            action.setPendingOnly();
+            action.setPending();
         }
         else {
-            action.resetPendingOnly();
+            action.resetPending();
         }
 
         Path inputDir = new Path(getFsTestCaseDir(), "input");

Modified: oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/trunk/release-log.txt?rev=1525202&r1=1525201&r2=1525202&view=diff
==============================================================================
--- oozie/trunk/release-log.txt (original)
+++ oozie/trunk/release-log.txt Sat Sep 21 08:07:37 2013
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1547 Change Coordinator SELECT query to fetch only necessary columns and consolidate JPA Executors (ryota)
 OOZIE-1529 Disable job DAG display for workflow having huge actions (puru via rohini)
 OOZIE-1468 Add created time column in WF_ACTIONS and SLA tables (rohini)
 OOZIE-1524 Change Workflow SELECT query to fetch only necessary columns and consolidate JPA Executors (ryota)