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/20 09:39:44 UTC

svn commit: r1524923 [1/2] - 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/command/wf/ core/src/main/java/org/apache/oozie/executor/jpa/ core/src/main/jav...

Author: ryota
Date: Fri Sep 20 07:39:43 2013
New Revision: 1524923

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

Modified:
    oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowActionBean.java
    oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowJobBean.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionStartXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/CompletedActionXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/DefinitionXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ReRunXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ResumeXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SuspendXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionsRunningGetJPAExecutor.java
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowJobQueryExecutor.java
    oozie/trunk/core/src/main/java/org/apache/oozie/service/ActionCheckerService.java
    oozie/trunk/core/src/main/java/org/apache/oozie/service/AuthorizationService.java
    oozie/trunk/core/src/main/java/org/apache/oozie/service/JPAService.java
    oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java
    oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionsRunningGetJPAExecutor.java
    oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobQueryExecutor.java
    oozie/trunk/core/src/test/java/org/apache/oozie/sla/TestSLAEventGeneration.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/WorkflowActionBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowActionBean.java?rev=1524923&r1=1524922&r2=1524923&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 Fri Sep 20 07:39:43 2013
@@ -65,15 +65,15 @@ import org.json.simple.JSONObject;
 
     @NamedQuery(name = "UPDATE_ACTION_CHECK", query = "update WorkflowActionBean a set a.userRetryCount = :userRetryCount, a.stats = :stats, a.externalChildIDs = :externalChildIDs, a.externalStatus = :externalStatus, a.statusStr = :status, a.data = :data, a.pending = :pending, a.errorCode = :errorCode, a.errorMessage = :errorMessage, a.lastCheckTimestamp = :lastCheckTime, a.retries = :retries, a.pendingAgeTimestamp = :pendingAge, a.startTimestamp = :startTime where a.id = :id"),
 
-    @NamedQuery(name = "UPDATE_ACTION_END", query = "update WorkflowActionBean a set a.stats = :stats, a.errorCode = :errorCode, a.errorMessage = :errorMessage, a.retries = :retries, a.endTimestamp = :endTime, a.statusStr = :status, a.pending = :pending, a.pendingAgeTimestamp = :pendingAge, a.signalValue = :signalValue, a.userRetryCount = :userRetryCount, a.externalStatus = :externalStatus where a.id = :id"),
+    @NamedQuery(name = "UPDATE_ACTION_END", query = "update WorkflowActionBean a set a.stats = :stats, a.errorCode = :errorCode, a.errorMessage = :errorMessage, a.retries = :retries, a.endTimestamp = :endTime, a.statusStr = :status, a.retries = :retries, a.pending = :pending, a.pendingAgeTimestamp = :pendingAge, a.signalValue = :signalValue, a.userRetryCount = :userRetryCount, a.externalStatus = :externalStatus where a.id = :id"),
 
-    @NamedQuery(name = "UPDATE_ACTION_PENDING", query = "update WorkflowActionBean a set a.pending = :pending where a.id = :id"),
+    @NamedQuery(name = "UPDATE_ACTION_PENDING", query = "update WorkflowActionBean a set a.pending = :pending, a.pendingAgeTimestamp = :pendingAge where a.id = :id"),
 
-    @NamedQuery(name = "UPDATE_ACTION_STATUS_PENDING", query = "update WorkflowActionBean a set a.statusStr = :status, a.pending = :pending where a.id = :id"),
+    @NamedQuery(name = "UPDATE_ACTION_STATUS_PENDING", query = "update WorkflowActionBean a set a.statusStr = :status, a.pending = :pending, a.pendingAgeTimestamp = :pendingAge where a.id = :id"),
 
-    @NamedQuery(name = "UPDATE_ACTION_PENDING_TRANS", query = "update WorkflowActionBean a set a.pending = :pending, a.transition = :transition where a.id = :id"),
+    @NamedQuery(name = "UPDATE_ACTION_PENDING_TRANS", query = "update WorkflowActionBean a set a.pending = :pending, a.pendingAgeTimestamp = :pendingAge, a.transition = :transition where a.id = :id"),
 
-    @NamedQuery(name = "UPDATE_ACTION_PENDING_TRANS_ERROR", query = "update WorkflowActionBean a set a.pending = :pending, a.transition = :transition, a.errorCode = :errorCode, a.errorMessage = :errorMessage where a.id = :id"),
+    @NamedQuery(name = "UPDATE_ACTION_PENDING_TRANS_ERROR", query = "update WorkflowActionBean a set a.pending = :pending, a.pendingAgeTimestamp = :pendingAge, a.transition = :transition, a.errorCode = :errorCode, a.errorMessage = :errorMessage where a.id = :id"),
 
     @NamedQuery(name = "DELETE_ACTION", query = "delete from WorkflowActionBean a where a.id = :id"),
 
@@ -83,6 +83,22 @@ import org.json.simple.JSONObject;
 
     @NamedQuery(name = "GET_ACTION", query = "select OBJECT(a) from WorkflowActionBean a where a.id = :id"),
 
+    @NamedQuery(name = "GET_ACTION_ID_TYPE", query = "select a.id, a.type from WorkflowActionBean a where a.id = :id"),
+
+    @NamedQuery(name = "GET_ACTION_FAIL", query = "select a.id, a.wfId, a.name, a.statusStr, a.pending, a.type, a.logToken, a.transition, a.errorCode, a.errorMessage from WorkflowActionBean a where a.id = :id"),
+
+    @NamedQuery(name = "GET_ACTION_SIGNAL", query = "select a.id, a.wfId, a.name, a.statusStr, a.pending, a.type, a.logToken, a.transition, a.errorCode, a.errorMessage, a.executionPath, a.signalValue, a.slaXml from WorkflowActionBean a where a.id = :id"),
+
+    @NamedQuery(name = "GET_ACTION_START", query = "select a.id, a.wfId, a.name, a.statusStr, a.pending, a.type, a.logToken, a.transition, a.retries, a.userRetryCount, a.userRetryMax, a.userRetryInterval, a.startTimestamp, a.endTimestamp, a.errorCode, a.errorMessage, a.cred, a.conf, a.slaXml from WorkflowActionBean a where a.id = :id"),
+
+    @NamedQuery(name = "GET_ACTION_CHECK", query = "select a.id, a.wfId, a.name, a.statusStr, a.pending, a.type, a.logToken, a.transition, a.retries, a.trackerUri, a.startTimestamp, a.endTimestamp, a.lastCheckTimestamp, a.errorCode, a.errorMessage, a.externalId, a.externalStatus, a.externalChildIDs, a.conf from WorkflowActionBean a where a.id = :id"),
+
+    @NamedQuery(name = "GET_ACTION_END", query = "select a.id, a.wfId, a.name, a.statusStr, a.pending, a.type, a.logToken, a.transition, a.retries, a.trackerUri, a.userRetryCount, a.userRetryMax, a.userRetryInterval, a.startTimestamp, a.endTimestamp, a.errorCode, a.errorMessage, a.externalId, a.externalStatus, a.externalChildIDs, a.conf, a.data, a.stats from WorkflowActionBean a where a.id = :id"),
+
+    @NamedQuery(name = "GET_ACTION_KILL", query = "select a.id, a.wfId, a.name, a.statusStr, a.pending, a.type, a.logToken, a.transition, a.retries, a.trackerUri, a.startTimestamp, a.endTimestamp, a.errorCode, a.errorMessage, a.externalId, a.conf, a.data from WorkflowActionBean a where a.id = :id"),
+
+    @NamedQuery(name = "GET_ACTION_COMPLETED", query = "select a.id, a.wfId, a.statusStr, a.type, a.logToken from WorkflowActionBean a where a.id = :id"),
+
     @NamedQuery(name = "GET_ACTION_FOR_UPDATE", query = "select OBJECT(a) from WorkflowActionBean a where a.id = :id"),
 
     @NamedQuery(name = "GET_ACTION_FOR_SLA", query = "select a.id, a.statusStr, a.startTimestamp, a.endTimestamp from WorkflowActionBean a where a.id = :id"),
@@ -93,7 +109,7 @@ import org.json.simple.JSONObject;
 
     @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_RUNNING_ACTIONS", query = "select OBJECT(a) from WorkflowActionBean a where a.pending = 1 AND a.statusStr = 'RUNNING' AND a.lastCheckTimestamp < :lastCheckTime"),
+    @NamedQuery(name = "GET_RUNNING_ACTIONS", query = "select a.id from WorkflowActionBean a where a.pending = 1 AND a.statusStr = 'RUNNING' AND a.lastCheckTimestamp < :lastCheckTime"),
 
     @NamedQuery(name = "GET_RETRY_MANUAL_ACTIONS", query = "select OBJECT(a) from WorkflowActionBean a where a.wfId = :wfId AND (a.statusStr = 'START_RETRY' OR a.statusStr = 'START_MANUAL' OR a.statusStr = 'END_RETRY' OR a.statusStr = 'END_MANUAL')") })
 @Table(name = "WF_ACTIONS")
@@ -415,6 +431,13 @@ public class WorkflowActionBean implemen
     }
 
     /**
+     * Set pending flag
+     */
+    public void setPending(int i) {
+        pending = i;
+    }
+
+    /**
      * Set a time when the action will be pending, normally a time in the
      * future.
      *

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowJobBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowJobBean.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowJobBean.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/WorkflowJobBean.java Fri Sep 20 07:39:43 2013
@@ -88,6 +88,22 @@ import org.json.simple.JSONObject;
 
     @NamedQuery(name = "GET_WORKFLOW", query = "select OBJECT(w) from WorkflowJobBean w where w.id = :id"),
 
+    @NamedQuery(name = "GET_WORKFLOW_STARTTIME", query = "select w.id, w.startTimestamp from WorkflowJobBean w where w.id = :id"),
+
+    @NamedQuery(name = "GET_WORKFLOW_USER_GROUP", query = "select w.user, w.group from WorkflowJobBean w where w.id = :id"),
+
+    @NamedQuery(name = "GET_WORKFLOW_SUSPEND", query = "select w.id, w.user, w.group, w.appName, w.statusStr, w.parentId, w.startTimestamp, w.endTimestamp, w.logToken, w.wfInstance  from WorkflowJobBean w where w.id = :id"),
+
+    @NamedQuery(name = "GET_WORKFLOW_RERUN", query = "select w.id, w.user, w.group, w.appName, w.statusStr, w.run, w.logToken, w.wfInstance from WorkflowJobBean w where w.id = :id"),
+
+    @NamedQuery(name = "GET_WORKFLOW_DEFINITION", query = "select w.id, w.user, w.group, w.appName, w.logToken, w.wfInstance from WorkflowJobBean w where w.id = :id"),
+
+    @NamedQuery(name = "GET_WORKFLOW_ACTION_OP", query = "select w.id, w.user, w.group, w.appName, w.appPath, w.statusStr, w.parentId, w.logToken, w.wfInstance, w.protoActionConf from WorkflowJobBean w where w.id = :id"),
+
+    @NamedQuery(name = "GET_WORKFLOW_KILL", query = "select w.id, w.user, w.group, w.appName, w.appPath, w.statusStr, w.parentId, w.startTimestamp, w.endTimestamp, w.logToken, w.wfInstance, w.slaXml from WorkflowJobBean w where w.id = :id"),
+
+    @NamedQuery(name = "GET_WORKFLOW_RESUME", query = "select w.id, w.user, w.group, w.appName, w.appPath, w.statusStr, w.parentId, w.startTimestamp, w.endTimestamp, w.logToken, w.wfInstance, w.protoActionConf from WorkflowJobBean w where w.id = :id"),
+
     @NamedQuery(name = "GET_WORKFLOW_FOR_UPDATE", query = "select OBJECT(w) from WorkflowJobBean w where w.id = :id"),
 
     @NamedQuery(name = "GET_WORKFLOW_FOR_SLA", query = "select w.id, w.statusStr, w.startTimestamp, w.endTimestamp from WorkflowJobBean w where w.id = :id"),

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionStartXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionStartXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionStartXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordActionStartXCommand.java Fri Sep 20 07:39:43 2013
@@ -47,7 +47,7 @@ import org.apache.oozie.executor.jpa.Bat
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.jdom.Element;
 import org.jdom.JDOMException;
@@ -188,7 +188,7 @@ public class CoordActionStartXCommand ex
                 JPAService jpaService = Services.get().get(JPAService.class);
                 if (jpaService != null) {
                     log.debug("Updating WF record for WFID :" + wfId + " with parent id: " + actionId);
-                    WorkflowJobBean wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(wfId));
+                    WorkflowJobBean wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_STARTTIME, wfId);
                     wfJob.setParentId(actionId);
                     wfJob.setLastModifiedTime(new Date());
                     BatchQueryExecutor executor = BatchQueryExecutor.getInstance();

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java Fri Sep 20 07:39:43 2013
@@ -33,10 +33,9 @@ import org.apache.oozie.client.WorkflowJ
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
@@ -87,8 +86,8 @@ public class ActionCheckXCommand extends
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(jobId));
-                this.wfAction = jpaService.execute(new WorkflowActionGetJPAExecutor(actionId));
+                this.wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_ACTION_OP, jobId);
+                this.wfAction = WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION_CHECK, actionId);
                 LogUtils.setLogInfo(wfJob, logInfo);
                 LogUtils.setLogInfo(wfAction, logInfo);
             }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java Fri Sep 20 07:39:43 2013
@@ -42,8 +42,8 @@ import org.apache.oozie.command.Precondi
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.service.ActionService;
@@ -97,8 +97,9 @@ public class ActionEndXCommand extends A
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(jobId));
-                this.wfAction = jpaService.execute(new WorkflowActionGetJPAExecutor(actionId));
+                this.wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_ACTION_OP,
+                        jobId);
+                this.wfAction = WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION_END, actionId);
                 LogUtils.setLogInfo(wfJob, logInfo);
                 LogUtils.setLogInfo(wfAction, logInfo);
             }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java Fri Sep 20 07:39:43 2013
@@ -34,8 +34,8 @@ import org.apache.oozie.command.Precondi
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.action.ActionExecutor;
@@ -95,8 +95,8 @@ public class ActionKillXCommand extends 
             jpaService = Services.get().get(JPAService.class);
 
             if (jpaService != null) {
-                this.wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(jobId));
-                this.wfAction = jpaService.execute(new WorkflowActionGetJPAExecutor(actionId));
+                this.wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_ACTION_OP, jobId);
+                this.wfAction = WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION_KILL, actionId);
                 LogUtils.setLogInfo(wfJob, logInfo);
                 LogUtils.setLogInfo(wfAction, logInfo);
             }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java Fri Sep 20 07:39:43 2013
@@ -43,9 +43,9 @@ import org.apache.oozie.command.Precondi
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.service.ActionService;
 import org.apache.oozie.service.EventHandlerService;
@@ -97,8 +97,8 @@ public class ActionStartXCommand extends
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(jobId));
-                this.wfAction = jpaService.execute(new WorkflowActionGetJPAExecutor(actionId));
+                this.wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW, jobId);
+                this.wfAction = WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION_START, actionId);
                 LogUtils.setLogInfo(wfJob, logInfo);
                 LogUtils.setLogInfo(wfAction, logInfo);
             }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/CompletedActionXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/CompletedActionXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/CompletedActionXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/CompletedActionXCommand.java Fri Sep 20 07:39:43 2013
@@ -24,7 +24,8 @@ import org.apache.oozie.WorkflowActionBe
 import org.apache.oozie.action.ActionExecutor;
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
-import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.service.ActionService;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
@@ -61,7 +62,8 @@ public class CompletedActionXCommand ext
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.wfactionBean = jpaService.execute(new WorkflowActionGetJPAExecutor(this.actionId));
+                this.wfactionBean = WorkflowActionQueryExecutor.getInstance().get(
+                        WorkflowActionQuery.GET_ACTION_COMPLETED, this.actionId);
             }
             else {
                 throw new CommandException(ErrorCode.E0610);

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/DefinitionXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/DefinitionXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/DefinitionXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/DefinitionXCommand.java Fri Sep 20 07:39:43 2013
@@ -22,7 +22,8 @@ import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.XException;
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
+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.LogUtils;
@@ -53,7 +54,7 @@ public class DefinitionXCommand extends 
             JPAService jpaService = Services.get().get(JPAService.class);
 
             if (jpaService != null) {
-                this.wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(jobId));
+                this.wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_DEFINITION, jobId);
                 LogUtils.setLogInfo(wfJob, logInfo);
             }
             else {

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java Fri Sep 20 07:39:43 2013
@@ -33,7 +33,7 @@ import org.apache.oozie.executor.jpa.Bat
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.executor.jpa.WorkflowActionsGetForJobJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.service.ActionService;
@@ -92,7 +92,7 @@ public class KillXCommand extends Workfl
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(wfId));
+                this.wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_KILL, wfId);
                 this.actionList = jpaService.execute(new WorkflowActionsGetForJobJPAExecutor(wfId));
                 LogUtils.setLogInfo(wfJob, logInfo);
             }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ReRunXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ReRunXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ReRunXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ReRunXCommand.java Fri Sep 20 07:39:43 2013
@@ -45,8 +45,8 @@ import org.apache.oozie.command.CommandE
 import org.apache.oozie.command.PreconditionException;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.WorkflowActionsGetForJobJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.service.DagXLogInfoService;
@@ -272,7 +272,7 @@ public class ReRunXCommand extends Workf
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.wfBean = jpaService.execute(new WorkflowJobGetJPAExecutor(this.jobId));
+                this.wfBean = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_RERUN, this.jobId);
                 this.actions = jpaService.execute(new WorkflowActionsGetForJobJPAExecutor(this.jobId));
             }
             else {

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ResumeXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ResumeXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ResumeXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ResumeXCommand.java Fri Sep 20 07:39:43 2013
@@ -40,7 +40,7 @@ import org.apache.oozie.executor.jpa.Bat
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.executor.jpa.WorkflowJobGetActionsJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.service.EventHandlerService;
 import org.apache.oozie.service.HadoopAccessorException;
@@ -182,7 +182,7 @@ public class ResumeXCommand extends Work
             throw new CommandException(ErrorCode.E0610);
         }
         try {
-            workflow = jpaService.execute(new WorkflowJobGetJPAExecutor(id));
+            workflow = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_RESUME, id);
         }
         catch (JPAExecutorException e) {
             throw new CommandException(e);

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java Fri Sep 20 07:39:43 2013
@@ -34,9 +34,9 @@ import org.apache.oozie.command.wf.Actio
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.service.ELService;
 import org.apache.oozie.service.EventHandlerService;
@@ -79,7 +79,6 @@ public class SignalXCommand extends Work
     private String wfJobErrorCode;
     private String wfJobErrorMsg;
 
-
     public SignalXCommand(String name, int priority, String jobId) {
         super(name, name, priority);
         this.jobId = ParamChecker.notEmpty(jobId, "jobId");
@@ -110,10 +109,10 @@ public class SignalXCommand extends Work
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(jobId));
+                this.wfJob = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW, jobId);
                 LogUtils.setLogInfo(wfJob, logInfo);
                 if (actionId != null) {
-                    this.wfAction = jpaService.execute(new WorkflowActionGetJPAExecutor(actionId));
+                    this.wfAction = WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION_SIGNAL, actionId);
                     LogUtils.setLogInfo(wfAction, logInfo);
                 }
             }
@@ -158,14 +157,14 @@ public class SignalXCommand extends Work
                 wfJob.setWorkflowInstance(workflowInstance);
                 generateEvent = true;
                 // 1. Add SLA status event for WF-JOB with status STARTED
-                SLAEventBean slaEvent = SLADbXOperations.createStatusEvent(wfJob.getSlaXml(), jobId,
-                        Status.STARTED, SlaAppType.WORKFLOW_JOB);
-                if(slaEvent != null) {
+                SLAEventBean slaEvent = SLADbXOperations.createStatusEvent(wfJob.getSlaXml(), jobId, Status.STARTED,
+                        SlaAppType.WORKFLOW_JOB);
+                if (slaEvent != null) {
                     insertList.add(slaEvent);
                 }
                 // 2. Add SLA registration events for all WF_ACTIONS
-                createSLARegistrationForAllActions(workflowInstance.getApp().getDefinition(), wfJob.getUser(), wfJob
-                        .getGroup(), wfJob.getConf());
+                createSLARegistrationForAllActions(workflowInstance.getApp().getDefinition(), wfJob.getUser(),
+                        wfJob.getGroup(), wfJob.getConf());
                 queue(new NotificationXCommand(wfJob));
             }
             else {
@@ -205,7 +204,8 @@ public class SignalXCommand extends Work
                 for (String actionToKillId : WorkflowStoreService.getActionsToKill(workflowInstance)) {
                     WorkflowActionBean actionToKill;
 
-                    actionToKill = jpaService.execute(new WorkflowActionGetJPAExecutor(actionToKillId));
+                    actionToKill = WorkflowActionQueryExecutor.getInstance().get(
+                            WorkflowActionQuery.GET_ACTION_ID_TYPE, actionToKillId);
 
                     actionToKill.setPending();
                     actionToKill.setStatus(WorkflowActionBean.Status.KILLED);
@@ -215,8 +215,8 @@ public class SignalXCommand extends Work
                 }
 
                 for (String actionToFailId : WorkflowStoreService.getActionsToFail(workflowInstance)) {
-                    WorkflowActionBean actionToFail = jpaService.execute(new WorkflowActionGetJPAExecutor(
-                            actionToFailId));
+                    WorkflowActionBean actionToFail = WorkflowActionQueryExecutor.getInstance().get(
+                            WorkflowActionQuery.GET_ACTION_FAIL, actionToFailId);
                     actionToFail.resetPending();
                     actionToFail.setStatus(WorkflowActionBean.Status.FAILED);
                     if (wfJobErrorCode != null) {
@@ -226,7 +226,7 @@ public class SignalXCommand extends Work
                     queue(new NotificationXCommand(wfJob, actionToFail));
                     SLAEventBean slaEvent = SLADbXOperations.createStatusEvent(wfAction.getSlaXml(), wfAction.getId(),
                             Status.FAILED, SlaAppType.WORKFLOW_ACTION);
-                    if(slaEvent != null) {
+                    if (slaEvent != null) {
                         insertList.add(slaEvent);
                     }
                     updateList.add(new UpdateEntry<WorkflowActionQuery>(
@@ -254,9 +254,9 @@ public class SignalXCommand extends Work
                 default: // TODO SUSPENDED
                     break;
             }
-            SLAEventBean slaEvent = SLADbXOperations.createStatusEvent(wfJob.getSlaXml(), jobId,
-                    slaStatus, SlaAppType.WORKFLOW_JOB);
-            if(slaEvent != null) {
+            SLAEventBean slaEvent = SLADbXOperations.createStatusEvent(wfJob.getSlaXml(), jobId, slaStatus,
+                    SlaAppType.WORKFLOW_JOB);
+            if (slaEvent != null) {
                 insertList.add(slaEvent);
             }
             queue(new NotificationXCommand(wfJob));
@@ -276,9 +276,8 @@ public class SignalXCommand extends Work
                         String tmpNodeConf = nodeDef.getConf();
                         String actionConf = context.getELEvaluator().evaluate(tmpNodeConf, String.class);
                         LOG.debug(
-                                "Try to resolve KillNode message for jobid [{0}], actionId [{1}], before resolve [{2}], " +
-                                "after resolve [{3}]",
-                                jobId, actionId, tmpNodeConf, actionConf);
+                                "Try to resolve KillNode message for jobid [{0}], actionId [{1}], before resolve [{2}], "
+                                        + "after resolve [{3}]", jobId, actionId, tmpNodeConf, actionConf);
                         if (wfAction.getErrorCode() != null) {
                             wfAction.setErrorInfo(wfAction.getErrorCode(), actionConf);
                         }
@@ -304,39 +303,35 @@ public class SignalXCommand extends Work
                 if (skipVar != null) {
                     skipNewAction = skipVar.equals("true");
                 }
-                try {
-                    if (skipNewAction) {
-                        WorkflowActionBean oldAction;
-
-                        oldAction = jpaService.execute(new WorkflowActionGetJPAExecutor(newAction.getId()));
 
-                        oldAction.setPending();
-                        updateList.add(new UpdateEntry<WorkflowActionQuery>(WorkflowActionQuery.UPDATE_ACTION_PENDING, oldAction));
+                if (skipNewAction) {
+                    WorkflowActionBean oldAction = new WorkflowActionBean();
+                    oldAction.setId(newAction.getId());
+                    oldAction.setPending();
+                    updateList.add(new UpdateEntry<WorkflowActionQuery>(WorkflowActionQuery.UPDATE_ACTION_PENDING,
+                            oldAction));
+                    queue(new SignalXCommand(jobId, oldAction.getId()));
+                }
+                else {
+                    try {
+                        // Make sure that transition node for a forked action
+                        // is inserted only once
+                        WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION_ID_TYPE,
+                                newAction.getId());
 
-                        queue(new SignalXCommand(jobId, oldAction.getId()));
+                        continue;
                     }
-                    else {
-                        try {
-                            // Make sure that transition node for a forked action
-                            // is inserted only once
-                            jpaService.execute(new WorkflowActionGetJPAExecutor(newAction.getId()));
-                            continue;
-                        }
-                        catch (JPAExecutorException jee) {
-                        }
-                        checkForSuspendNode(newAction);
-                        newAction.setPending();
-                        String actionSlaXml = getActionSLAXml(newAction.getName(), workflowInstance.getApp()
-                                .getDefinition(), wfJob.getConf());
-                        newAction.setSlaXml(actionSlaXml);
-                        insertList.add(newAction);
-                        LOG.debug("SignalXCommand: Name: " + newAction.getName() + ", Id: " + newAction.getId()
-                                + ", Authcode:" + newAction.getCred());
-                        queue(new ActionStartXCommand(newAction.getId(), newAction.getType()));
+                    catch (JPAExecutorException jee) {
                     }
-                }
-                catch (JPAExecutorException je) {
-                    throw new CommandException(je);
+                    checkForSuspendNode(newAction);
+                    newAction.setPending();
+                    String actionSlaXml = getActionSLAXml(newAction.getName(), workflowInstance.getApp()
+                            .getDefinition(), wfJob.getConf());
+                    newAction.setSlaXml(actionSlaXml);
+                    insertList.add(newAction);
+                    LOG.debug("SignalXCommand: Name: " + newAction.getName() + ", Id: " + newAction.getId()
+                            + ", Authcode:" + newAction.getCred());
+                    queue(new ActionStartXCommand(newAction.getId(), newAction.getType()));
                 }
             }
         }
@@ -354,11 +349,10 @@ public class SignalXCommand extends Work
         catch (JPAExecutorException je) {
             throw new CommandException(je);
         }
-        LOG.debug(
-                "Updated the workflow status to " + wfJob.getId() + "  status =" + wfJob.getStatusStr());
+        LOG.debug("Updated the workflow status to " + wfJob.getId() + "  status =" + wfJob.getStatusStr());
         if (wfJob.getStatus() != WorkflowJob.Status.RUNNING && wfJob.getStatus() != WorkflowJob.Status.SUSPENDED) {
             updateParentIfNecessary(wfJob);
-            new WfEndXCommand(wfJob).call(); //To delete the WF temp dir
+            new WfEndXCommand(wfJob).call(); // To delete the WF temp dir
         }
         LOG.debug("ENDED SignalCommand for jobid=" + jobId + ", actionId=" + actionId);
         return null;
@@ -417,11 +411,11 @@ public class SignalXCommand extends Work
                 if (eSla != null) {
                     String slaXml = resolveSla(eSla, conf);
                     eSla = XmlUtils.parseXml(slaXml);
-                    String actionId = Services.get().get(UUIDService.class).generateChildId(jobId,
-                            action.getAttributeValue("name") + "");
+                    String actionId = Services.get().get(UUIDService.class)
+                            .generateChildId(jobId, action.getAttributeValue("name") + "");
                     SLAEventBean slaEvent = SLADbXOperations.createSlaRegistrationEvent(eSla, actionId,
                             SlaAppType.WORKFLOW_ACTION, user, group);
-                    if(slaEvent != null) {
+                    if (slaEvent != null) {
                         insertList.add(slaEvent);
                     }
                 }
@@ -439,7 +433,8 @@ public class SignalXCommand extends Work
             String[] values = wfjobConf.getTrimmedStrings(OozieClient.OOZIE_SUSPEND_ON_NODES);
             if (values != null) {
                 if (values.length == 1 && values[0].equals("*")) {
-                    LOG.info("Reached suspend node at [{0}], suspending workflow [{1}]", newAction.getName(), wfJob.getId());
+                    LOG.info("Reached suspend node at [{0}], suspending workflow [{1}]", newAction.getName(),
+                            wfJob.getId());
                     queue(new SuspendXCommand(jobId));
                 }
                 else {

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SuspendXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SuspendXCommand.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SuspendXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SuspendXCommand.java Fri Sep 20 07:39:43 2013
@@ -32,7 +32,7 @@ import org.apache.oozie.executor.jpa.Bat
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.executor.jpa.WorkflowActionRetryManualGetJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.service.EventHandlerService;
 import org.apache.oozie.service.JPAService;
@@ -147,7 +147,8 @@ public class SuspendXCommand extends Wor
         try {
             jpaService = Services.get().get(JPAService.class);
             if (jpaService != null) {
-                this.wfJobBean = jpaService.execute(new WorkflowJobGetJPAExecutor(this.wfid));
+                this.wfJobBean = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_SUSPEND,
+                        this.wfid);
             }
             else {
                 throw new CommandException(ErrorCode.E0610);

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=1524923&r1=1524922&r2=1524923&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 Fri Sep 20 07:39:43 2013
@@ -17,14 +17,18 @@
  */
 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.ErrorCode;
+import org.apache.oozie.StringBlob;
 import org.apache.oozie.WorkflowActionBean;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.util.DateUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -44,7 +48,16 @@ public class WorkflowActionQueryExecutor
         UPDATE_ACTION_STATUS_PENDING,
         UPDATE_ACTION_PENDING_TRANS,
         UPDATE_ACTION_PENDING_TRANS_ERROR,
-        GET_ACTION
+        GET_ACTION,
+        GET_ACTION_ID_TYPE,
+        GET_ACTION_FAIL,
+        GET_ACTION_SIGNAL,
+        GET_ACTION_START,
+        GET_ACTION_CHECK,
+        GET_ACTION_END,
+        GET_ACTION_KILL,
+        GET_ACTION_COMPLETED,
+        GET_RUNNING_ACTIONS
     };
 
     private static WorkflowActionQueryExecutor instance = new WorkflowActionQueryExecutor();
@@ -106,21 +119,25 @@ public class WorkflowActionQueryExecutor
                 break;
             case UPDATE_ACTION_PENDING:
                 query.setParameter("pending", actionBean.getPending());
+                query.setParameter("pendingAge", actionBean.getPendingAgeTimestamp());
                 query.setParameter("id", actionBean.getId());
                 break;
             case UPDATE_ACTION_STATUS_PENDING:
                 query.setParameter("status", actionBean.getStatus().toString());
                 query.setParameter("pending", actionBean.getPending());
+                query.setParameter("pendingAge", actionBean.getPendingAgeTimestamp());
                 query.setParameter("id", actionBean.getId());
                 break;
             case UPDATE_ACTION_PENDING_TRANS:
                 query.setParameter("transition", actionBean.getTransition());
                 query.setParameter("pending", actionBean.getPending());
+                query.setParameter("pendingAge", actionBean.getPendingAgeTimestamp());
                 query.setParameter("id", actionBean.getId());
                 break;
             case UPDATE_ACTION_PENDING_TRANS_ERROR:
                 query.setParameter("transition", actionBean.getTransition());
                 query.setParameter("pending", actionBean.getPending());
+                query.setParameter("pendingAge", actionBean.getPendingAgeTimestamp());
                 query.setParameter("errorCode", actionBean.getErrorCode());
                 query.setParameter("errorMessage", actionBean.getErrorMessage());
                 query.setParameter("id", actionBean.getId());
@@ -166,6 +183,7 @@ public class WorkflowActionQueryExecutor
                 query.setParameter("retries", actionBean.getRetries());
                 query.setParameter("status", actionBean.getStatus().toString());
                 query.setParameter("endTime", actionBean.getEndTimestamp());
+                query.setParameter("retries", actionBean.getRetries());
                 query.setParameter("pending", actionBean.getPending());
                 query.setParameter("pendingAge", actionBean.getPendingAgeTimestamp());
                 query.setParameter("signalValue", actionBean.getSignalValue());
@@ -185,14 +203,25 @@ public class WorkflowActionQueryExecutor
     public Query getSelectQuery(WorkflowActionQuery namedQuery, EntityManager em, Object... parameters)
             throws JPAExecutorException {
         Query query = em.createNamedQuery(namedQuery.name());
-        WorkflowActionQuery waQuery = (WorkflowActionQuery) namedQuery;
-        switch (waQuery) {
+        switch (namedQuery) {
             case GET_ACTION:
+            case GET_ACTION_ID_TYPE:
+            case GET_ACTION_FAIL:
+            case GET_ACTION_SIGNAL:
+            case GET_ACTION_START:
+            case GET_ACTION_CHECK:
+            case GET_ACTION_END:
+            case GET_ACTION_KILL:
+            case GET_ACTION_COMPLETED:
                 query.setParameter("id", parameters[0]);
                 break;
+            case GET_RUNNING_ACTIONS:
+                Timestamp ts = new Timestamp(System.currentTimeMillis() - (Integer)parameters[0] * 1000);
+                query.setParameter("lastCheckTime", ts);
+                break;
             default:
                 throw new JPAExecutorException(ErrorCode.E0603, "QueryExecutor cannot set parameters for "
-                        + waQuery.name());
+                        + namedQuery.name());
         }
         return query;
     }
@@ -205,15 +234,167 @@ public class WorkflowActionQueryExecutor
         return ret;
     }
 
+    private WorkflowActionBean constructBean(WorkflowActionQuery namedQuery, Object ret) throws JPAExecutorException {
+        WorkflowActionBean bean;
+        Object[] arr;
+        switch (namedQuery) {
+            case GET_ACTION:
+                bean = (WorkflowActionBean) ret;
+                break;
+            case GET_ACTION_ID_TYPE:
+                bean = new WorkflowActionBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setType((String) arr[1]);
+                break;
+            case GET_ACTION_FAIL:
+                bean = new WorkflowActionBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setJobId((String) arr[1]);
+                bean.setName((String) arr[2]);
+                bean.setStatusStr((String) arr[3]);
+                bean.setPending((Integer) arr[4]);
+                bean.setType((String) arr[5]);
+                bean.setLogToken((String) arr[6]);
+                bean.setTransition((String) arr[7]);
+                bean.setErrorInfo((String) arr[8], (String) arr[9]);
+                break;
+            case GET_ACTION_SIGNAL:
+                bean = new WorkflowActionBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setJobId((String) arr[1]);
+                bean.setName((String) arr[2]);
+                bean.setStatusStr((String) arr[3]);
+                bean.setPending((Integer) arr[4]);
+                bean.setType((String) arr[5]);
+                bean.setLogToken((String) arr[6]);
+                bean.setTransition((String) arr[7]);
+                bean.setErrorInfo((String) arr[8], (String) arr[9]);
+                bean.setExecutionPath((String) arr[10]);
+                bean.setSignalValue((String) arr[11]);
+                bean.setSlaXmlBlob((StringBlob) arr[12]);
+                break;
+            case GET_ACTION_START:
+                bean = new WorkflowActionBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setJobId((String) arr[1]);
+                bean.setName((String) arr[2]);
+                bean.setStatusStr((String) arr[3]);
+                bean.setPending((Integer) arr[4]);
+                bean.setType((String) arr[5]);
+                bean.setLogToken((String) arr[6]);
+                bean.setTransition((String) arr[7]);
+                bean.setRetries((Integer) arr[8]);
+                bean.setUserRetryCount((Integer) arr[9]);
+                bean.setUserRetryMax((Integer) arr[10]);
+                bean.setUserRetryInterval((Integer) arr[11]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[12]));
+                bean.setEndTime(DateUtils.toDate((Timestamp) arr[13]));
+                bean.setErrorInfo((String) arr[14], (String) arr[15]);
+                bean.setCred((String) arr[16]);
+                bean.setConfBlob((StringBlob) arr[17]);
+                bean.setSlaXmlBlob((StringBlob) arr[18]);
+                break;
+            case GET_ACTION_CHECK:
+                bean = new WorkflowActionBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setJobId((String) arr[1]);
+                bean.setName((String) arr[2]);
+                bean.setStatusStr((String) arr[3]);
+                bean.setPending((Integer) arr[4]);
+                bean.setType((String) arr[5]);
+                bean.setLogToken((String) arr[6]);
+                bean.setTransition((String) arr[7]);
+                bean.setRetries((Integer) arr[8]);
+                bean.setTrackerUri((String) arr[9]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[10]));
+                bean.setEndTime(DateUtils.toDate((Timestamp) arr[11]));
+                bean.setLastCheckTime(DateUtils.toDate((Timestamp) arr[12]));
+                bean.setErrorInfo((String) arr[13], (String) arr[14]);
+                bean.setExternalId((String) arr[15]);
+                bean.setExternalStatus((String) arr[16]);
+                bean.setExternalChildIDsBlob((StringBlob) arr[17]);
+                bean.setConfBlob((StringBlob) arr[18]);
+                break;
+            case GET_ACTION_END:
+                bean = new WorkflowActionBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setJobId((String) arr[1]);
+                bean.setName((String) arr[2]);
+                bean.setStatusStr((String) arr[3]);
+                bean.setPending((Integer) arr[4]);
+                bean.setType((String) arr[5]);
+                bean.setLogToken((String) arr[6]);
+                bean.setTransition((String) arr[7]);
+                bean.setRetries((Integer) arr[8]);
+                bean.setTrackerUri((String) arr[9]);
+                bean.setUserRetryCount((Integer) arr[10]);
+                bean.setUserRetryMax((Integer) arr[11]);
+                bean.setUserRetryInterval((Integer) arr[12]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[13]));
+                bean.setEndTime(DateUtils.toDate((Timestamp) arr[14]));
+                bean.setErrorInfo((String) arr[15], (String) arr[16]);
+                bean.setExternalId((String) arr[17]);
+                bean.setExternalStatus((String) arr[18]);
+                bean.setExternalChildIDsBlob((StringBlob) arr[19]);
+                bean.setConfBlob((StringBlob) arr[20]);
+                bean.setDataBlob((StringBlob) arr[21]);
+                bean.setStatsBlob((StringBlob) arr[22]);
+                break;
+            case GET_ACTION_KILL:
+                bean = new WorkflowActionBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setJobId((String) arr[1]);
+                bean.setName((String) arr[2]);
+                bean.setStatusStr((String) arr[3]);
+                bean.setPending((Integer) arr[4]);
+                bean.setType((String) arr[5]);
+                bean.setLogToken((String) arr[6]);
+                bean.setTransition((String) arr[7]);
+                bean.setRetries((Integer) arr[8]);
+                bean.setTrackerUri((String) arr[9]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[10]));
+                bean.setEndTime(DateUtils.toDate((Timestamp) arr[11]));
+                bean.setErrorInfo((String) arr[12], (String) arr[13]);
+                bean.setExternalId((String) arr[14]);
+                bean.setConfBlob((StringBlob) arr[15]);
+                bean.setDataBlob((StringBlob) arr[16]);
+                break;
+            case GET_ACTION_COMPLETED:
+                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.setLogToken((String) arr[4]);
+                break;
+            case GET_RUNNING_ACTIONS:
+                bean = new WorkflowActionBean();
+                bean.setId((String)ret);
+                break;
+            default:
+                throw new JPAExecutorException(ErrorCode.E0603, "QueryExecutor cannot construct action bean for "
+                        + namedQuery.name());
+        }
+        return bean;
+    }
+
     @Override
     public WorkflowActionBean get(WorkflowActionQuery namedQuery, Object... parameters) throws JPAExecutorException {
         EntityManager em = jpaService.getEntityManager();
         Query query = getSelectQuery(namedQuery, em, parameters);
-        WorkflowActionBean bean = null;
-        bean = (WorkflowActionBean) jpaService.executeGet(namedQuery.name(), query, em);
-        if (bean == null) {
+        Object ret = jpaService.executeGet(namedQuery.name(), query, em);
+        if (ret == null) {
             throw new JPAExecutorException(ErrorCode.E0605, query.toString());
         }
+        WorkflowActionBean bean = constructBean(namedQuery, ret);
         return bean;
     }
 
@@ -222,10 +403,13 @@ public class WorkflowActionQueryExecutor
             throws JPAExecutorException {
         EntityManager em = jpaService.getEntityManager();
         Query query = getSelectQuery(namedQuery, em, parameters);
-        List<WorkflowActionBean> beanList = (List<WorkflowActionBean>) jpaService.executeGetList(namedQuery.name(),
-                query, em);
-        if (beanList == null || beanList.size() == 0) {
-            throw new JPAExecutorException(ErrorCode.E0605, query.toString());
+        List<?> retList = (List<?>) jpaService.executeGetList(namedQuery.name(), query, em);
+        List<WorkflowActionBean> beanList = null;
+        if (retList != null) {
+            beanList = new ArrayList<WorkflowActionBean>();
+            for (Object ret : retList) {
+                beanList.add(constructBean(namedQuery, ret));
+            }
         }
         return beanList;
     }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionsRunningGetJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionsRunningGetJPAExecutor.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionsRunningGetJPAExecutor.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionsRunningGetJPAExecutor.java Fri Sep 20 07:39:43 2013
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.oozie.executor.jpa;
-
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.List;
-
-import javax.persistence.EntityManager;
-import javax.persistence.Query;
-
-import org.apache.oozie.ErrorCode;
-import org.apache.oozie.WorkflowActionBean;
-
-/**
- * JPA Executor to get running workflow actions
- */
-public class WorkflowActionsRunningGetJPAExecutor implements JPAExecutor<List<WorkflowActionBean>> {
-
-    private final long checkAgeSecs;
-
-    public WorkflowActionsRunningGetJPAExecutor(long checkAgeSecs) {
-        this.checkAgeSecs = checkAgeSecs;
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.oozie.executor.jpa.JPAExecutor#execute(javax.persistence.EntityManager)
-     */
-    @Override
-    @SuppressWarnings("unchecked")
-    public List<WorkflowActionBean> execute(EntityManager em) throws JPAExecutorException {
-        List<WorkflowActionBean> actions;
-        try {
-            Timestamp ts = new Timestamp(System.currentTimeMillis() - checkAgeSecs * 1000);
-            Query q = em.createNamedQuery("GET_RUNNING_ACTIONS");
-            q.setParameter("lastCheckTime", ts);
-            actions = q.getResultList();
-         }
-        catch (Exception e) {
-            throw new JPAExecutorException(ErrorCode.E0605, "null", e);
-        }
-        return actions;
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.oozie.executor.jpa.JPAExecutor#getName()
-     */
-    @Override
-    public String getName() {
-        return "WorkflowActionsRunningGetJPAExecutor";
-    }
-
-}

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowJobQueryExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowJobQueryExecutor.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowJobQueryExecutor.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowJobQueryExecutor.java Fri Sep 20 07:39:43 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.ErrorCode;
+import org.apache.oozie.StringBlob;
+import org.apache.oozie.WorkflowActionBean;
 import org.apache.oozie.WorkflowJobBean;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.util.DateUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -43,7 +50,14 @@ public class WorkflowJobQueryExecutor ex
         UPDATE_WORKFLOW_STATUS_INSTANCE_MOD_START_END,
         UPDATE_WORKFLOW_RERUN,
         GET_WORKFLOW,
-        DELETE_WORKFLOW
+        GET_WORKFLOW_STARTTIME,
+        GET_WORKFLOW_USER_GROUP,
+        GET_WORKFLOW_SUSPEND,
+        GET_WORKFLOW_ACTION_OP,
+        GET_WORKFLOW_RERUN,
+        GET_WORKFLOW_DEFINITION,
+        GET_WORKFLOW_KILL,
+        GET_WORKFLOW_RESUME
     };
 
     private static WorkflowJobQueryExecutor instance = new WorkflowJobQueryExecutor();
@@ -154,6 +168,14 @@ public class WorkflowJobQueryExecutor ex
         Query query = em.createNamedQuery(namedQuery.name());
         switch (namedQuery) {
             case GET_WORKFLOW:
+            case GET_WORKFLOW_STARTTIME:
+            case GET_WORKFLOW_USER_GROUP:
+            case GET_WORKFLOW_SUSPEND:
+            case GET_WORKFLOW_ACTION_OP:
+            case GET_WORKFLOW_RERUN:
+            case GET_WORKFLOW_DEFINITION:
+            case GET_WORKFLOW_KILL:
+            case GET_WORKFLOW_RESUME:
                 query.setParameter("id", parameters[0]);
                 break;
             default:
@@ -171,14 +193,123 @@ public class WorkflowJobQueryExecutor ex
         return ret;
     }
 
+    private WorkflowJobBean constructBean(WorkflowJobQuery namedQuery, Object ret) throws JPAExecutorException {
+        WorkflowJobBean bean;
+        Object[] arr;
+        switch (namedQuery) {
+            case GET_WORKFLOW:
+                bean = (WorkflowJobBean) ret;
+                break;
+            case GET_WORKFLOW_STARTTIME:
+                bean = new WorkflowJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[1]));
+                break;
+            case GET_WORKFLOW_USER_GROUP:
+                bean = new WorkflowJobBean();
+                arr = (Object[]) ret;
+                bean.setUser((String) arr[0]);
+                bean.setGroup((String) arr[1]);
+                break;
+            case GET_WORKFLOW_SUSPEND:
+                bean = new WorkflowJobBean();
+                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.setParentId((String) arr[5]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[6]));
+                bean.setEndTime(DateUtils.toDate((Timestamp) arr[7]));
+                bean.setLogToken((String) arr[8]);
+                bean.setWfInstanceBlob((BinaryBlob) (arr[9]));
+                break;
+            case GET_WORKFLOW_ACTION_OP:
+                bean = new WorkflowJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setUser((String) arr[1]);
+                bean.setGroup((String) arr[2]);
+                bean.setAppName((String) arr[3]);
+                bean.setAppPath((String) arr[4]);
+                bean.setStatusStr((String) arr[5]);
+                bean.setParentId((String) arr[6]);
+                bean.setLogToken((String) arr[7]);
+                bean.setWfInstanceBlob((BinaryBlob) (arr[8]));
+                bean.setProtoActionConfBlob((StringBlob) arr[9]);
+                break;
+            case GET_WORKFLOW_RERUN:
+                bean = new WorkflowJobBean();
+                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.setRun((Integer) arr[5]);
+                bean.setLogToken((String) arr[6]);
+                bean.setWfInstanceBlob((BinaryBlob) (arr[7]));
+                break;
+            case GET_WORKFLOW_DEFINITION:
+                bean = new WorkflowJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setUser((String) arr[1]);
+                bean.setGroup((String) arr[2]);
+                bean.setAppName((String) arr[3]);
+                bean.setLogToken((String) arr[4]);
+                bean.setWfInstanceBlob((BinaryBlob) (arr[5]));
+                break;
+            case GET_WORKFLOW_KILL:
+                bean = new WorkflowJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setUser((String) arr[1]);
+                bean.setGroup((String) arr[2]);
+                bean.setAppName((String) arr[3]);
+                bean.setAppPath((String) arr[4]);
+                bean.setStatusStr((String) arr[5]);
+                bean.setParentId((String) arr[6]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[7]));
+                bean.setEndTime(DateUtils.toDate((Timestamp) arr[8]));
+                bean.setLogToken((String) arr[9]);
+                bean.setWfInstanceBlob((BinaryBlob) (arr[10]));
+                bean.setSlaXmlBlob((StringBlob) arr[11]);
+                break;
+            case GET_WORKFLOW_RESUME:
+                bean = new WorkflowJobBean();
+                arr = (Object[]) ret;
+                bean.setId((String) arr[0]);
+                bean.setUser((String) arr[1]);
+                bean.setGroup((String) arr[2]);
+                bean.setAppName((String) arr[3]);
+                bean.setAppPath((String) arr[4]);
+                bean.setStatusStr((String) arr[5]);
+                bean.setParentId((String) arr[6]);
+                bean.setStartTime(DateUtils.toDate((Timestamp) arr[7]));
+                bean.setEndTime(DateUtils.toDate((Timestamp) arr[8]));
+                bean.setLogToken((String) arr[9]);
+                bean.setWfInstanceBlob((BinaryBlob) (arr[10]));
+                bean.setProtoActionConfBlob((StringBlob) arr[11]);
+                break;
+            default:
+                throw new JPAExecutorException(ErrorCode.E0603, "QueryExecutor cannot construct job bean for "
+                        + namedQuery.name());
+        }
+        return bean;
+    }
+
     @Override
     public WorkflowJobBean get(WorkflowJobQuery namedQuery, Object... parameters) throws JPAExecutorException {
         EntityManager em = jpaService.getEntityManager();
         Query query = getSelectQuery(namedQuery, em, parameters);
-        WorkflowJobBean bean = (WorkflowJobBean) 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());
         }
+        WorkflowJobBean bean = constructBean(namedQuery, ret);
         return bean;
     }
 
@@ -186,10 +317,13 @@ public class WorkflowJobQueryExecutor ex
     public List<WorkflowJobBean> getList(WorkflowJobQuery namedQuery, Object... parameters) throws JPAExecutorException {
         EntityManager em = jpaService.getEntityManager();
         Query query = getSelectQuery(namedQuery, em, parameters);
-        List<WorkflowJobBean> beanList = (List<WorkflowJobBean>) 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<WorkflowJobBean> beanList = null;
+        if (retList != null) {
+            beanList = new ArrayList<WorkflowJobBean>();
+            for (Object ret : retList) {
+                beanList.add(constructBean(namedQuery, ret));
+            }
         }
         return beanList;
     }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/service/ActionCheckerService.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/service/ActionCheckerService.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/service/ActionCheckerService.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/service/ActionCheckerService.java Fri Sep 20 07:39:43 2013
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.oozie.CoordinatorActionBean;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.WorkflowActionBean;
 import org.apache.oozie.command.CommandException;
@@ -29,7 +28,8 @@ import org.apache.oozie.command.coord.Co
 import org.apache.oozie.command.wf.ActionCheckXCommand;
 import org.apache.oozie.executor.jpa.CoordActionsRunningGetJPAExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowActionsRunningGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
 import org.apache.oozie.util.XCallable;
 import org.apache.oozie.util.XLog;
 
@@ -111,8 +111,8 @@ public class ActionCheckerService implem
 
             List<WorkflowActionBean> actions;
             try {
-                actions = jpaService
-                        .execute(new WorkflowActionsRunningGetJPAExecutor(actionCheckDelay));
+                actions = WorkflowActionQueryExecutor.getInstance().getList(WorkflowActionQuery.GET_RUNNING_ACTIONS,
+                        actionCheckDelay);
             }
             catch (JPAExecutorException je) {
                 throw new CommandException(je);

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/service/AuthorizationService.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/service/AuthorizationService.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/service/AuthorizationService.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/service/AuthorizationService.java Fri Sep 20 07:39:43 2013
@@ -38,7 +38,8 @@ import org.apache.oozie.client.XOozieCli
 import org.apache.oozie.executor.jpa.BundleJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.util.ConfigUtils;
 import org.apache.oozie.util.Instrumentation;
 import org.apache.oozie.util.XLog;
@@ -433,7 +434,7 @@ public class AuthorizationService implem
                     JPAService jpaService = Services.get().get(JPAService.class);
                     if (jpaService != null) {
                         try {
-                            jobBean = jpaService.execute(new WorkflowJobGetJPAExecutor(jobId));
+                            jobBean = WorkflowJobQueryExecutor.getInstance().get(WorkflowJobQuery.GET_WORKFLOW_USER_GROUP, jobId);
                         }
                         catch (JPAExecutorException je) {
                             throw new AuthorizationException(je);

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/service/JPAService.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/service/JPAService.java?rev=1524923&r1=1524922&r2=1524923&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/service/JPAService.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/service/JPAService.java Fri Sep 20 07:39:43 2013
@@ -439,7 +439,7 @@ public class JPAService implements Servi
      * @return list containing results that match the query
      * @throws JPAExecutorException
      */
-    public List executeGetList(String namedQueryName, Query query, EntityManager em) throws JPAExecutorException {
+    public List<?> executeGetList(String namedQueryName, Query query, EntityManager em) throws JPAExecutorException {
         Instrumentation.Cron cron = new Instrumentation.Cron();
         try {
 
@@ -450,7 +450,7 @@ public class JPAService implements Servi
 
             cron.start();
             em.getTransaction().begin();
-            List resultList = null;
+            List<?> resultList = null;
             try {
                 resultList = query.getResultList();
             }