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/06/05 19:26:20 UTC

svn commit: r1489973 - in /oozie/trunk: ./ client/src/main/java/org/apache/oozie/client/rest/ core/src/main/java/org/apache/oozie/ core/src/main/java/org/apache/oozie/client/rest/ core/src/main/java/org/apache/oozie/command/coord/ core/src/main/java/or...

Author: ryota
Date: Wed Jun  5 17:26:19 2013
New Revision: 1489973

URL: http://svn.apache.org/r1489973
Log:
OOZIE-1357 Can't view more than 1000 actions of a coordinator and paging does not work (ryota)

Added:
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionsCountForJobIdJPAExecutor.java
Modified:
    oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java
    oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
    oozie/trunk/core/src/main/java/org/apache/oozie/BaseEngine.java
    oozie/trunk/core/src/main/java/org/apache/oozie/BundleEngine.java
    oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorEngine.java
    oozie/trunk/core/src/main/java/org/apache/oozie/DagEngine.java
    oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonCoordinatorJob.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordJobXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsSubsetJPAExecutor.java
    oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
    oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
    oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleEngineSimple.java
    oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java
    oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobGetActionsSubsetJPAExecutor.java
    oozie/trunk/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java
    oozie/trunk/release-log.txt
    oozie/trunk/webapp/src/main/webapp/oozie-console.js

Modified: oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java
URL: http://svn.apache.org/viewvc/oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java (original)
+++ oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java Wed Jun  5 17:26:19 2013
@@ -98,6 +98,8 @@ public interface JsonTags {
     public static final String COORDINATOR_JOB_CONSOLE_URL = "consoleUrl";
     public static final String COORDINATOR_JOB_ACTIONS = "actions";
     public static final String COORDINATOR_JOB_USER = "user";
+    public static final String COORDINATOR_JOB_NUM_ACTION = "total";
+
     @Deprecated
     public static final String COORDINATOR_JOB_GROUP = "group";
     public static final String COORDINATOR_JOB_ACL = "acl";

Modified: oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
URL: http://svn.apache.org/viewvc/oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java (original)
+++ oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java Wed Jun  5 17:26:19 2013
@@ -46,6 +46,8 @@ public interface RestConstants {
 
     public static final String LEN_PARAM = "len";
 
+    public static final String ORDER_PARAM = "order";
+
     public static final String JOB_FILTER_PARAM = "filter";
 
     public static final String JOB_RESOURCE = "/job";
@@ -61,6 +63,7 @@ public interface RestConstants {
     public static final String JOB_ACTION_KILL = "kill";
 
     public static final String JOB_ACTION_CHANGE = "change";
+
     public static final String JOB_CHANGE_VALUE = "value";
 
     public static final String JOB_ACTION_RERUN = "rerun";

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/BaseEngine.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/BaseEngine.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/BaseEngine.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/BaseEngine.java Wed Jun  5 17:26:19 2013
@@ -149,10 +149,12 @@ public abstract class BaseEngine {
      * @param filter the status filter
      * @param start starting from this index in the list of actions belonging to the job
      * @param length number of actions to be returned
+     * @param order true if actions are sorted in a descending order of nominal time, false if asc order
      * @return the coord job info.
      * @throws BaseEngineException thrown if the job info could not be obtained.
      */
-    public abstract CoordinatorJob getCoordJob(String jobId, String filter, int start, int length) throws BaseEngineException;
+    public abstract CoordinatorJob getCoordJob(String jobId, String filter, int start, int length, boolean desc)
+            throws BaseEngineException;
 
     /**
      * Return the a job definition.

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/BundleEngine.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/BundleEngine.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/BundleEngine.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/BundleEngine.java Wed Jun  5 17:26:19 2013
@@ -125,8 +125,10 @@ public class BundleEngine extends BaseEn
      * @see org.apache.oozie.BaseEngine#getCoordJob(java.lang.String, int, int)
      */
     @Override
-    public CoordinatorJob getCoordJob(String jobId, String filter, int start, int length) throws BundleEngineException {
-        throw new BundleEngineException(new XException(ErrorCode.E0301, "cannot get a coordinator job from BundleEngine"));
+    public CoordinatorJob getCoordJob(String jobId, String filter, int start, int length, boolean desc)
+            throws BundleEngineException {
+        throw new BundleEngineException(new XException(ErrorCode.E0301,
+                "cannot get a coordinator job from BundleEngine"));
     }
 
     /* (non-Javadoc)

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorEngine.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorEngine.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorEngine.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/CoordinatorEngine.java Wed Jun  5 17:26:19 2013
@@ -143,10 +143,11 @@ public class CoordinatorEngine extends B
      * @see org.apache.oozie.BaseEngine#getCoordJob(java.lang.String, java.lang.String, int, int)
      */
     @Override
-    public CoordinatorJobBean getCoordJob(String jobId, String filter, int start, int length) throws BaseEngineException {
+    public CoordinatorJobBean getCoordJob(String jobId, String filter, int start, int length, boolean desc)
+            throws BaseEngineException {
         List<String> filterList = parseStatusFilter(filter);
         try {
-            return new CoordJobXCommand(jobId, filterList, start, length)
+            return new CoordJobXCommand(jobId, filterList, start, length, desc)
                     .call();
         }
         catch (CommandException ex) {

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/DagEngine.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/DagEngine.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/DagEngine.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/DagEngine.java Wed Jun  5 17:26:19 2013
@@ -470,7 +470,7 @@ public class DagEngine extends BaseEngin
     }
 
     @Override
-    public CoordinatorJob getCoordJob(String jobId, String filter, int start, int length) throws BaseEngineException {
+    public CoordinatorJob getCoordJob(String jobId, String filter, int start, int length, boolean desc) throws BaseEngineException {
         throw new BaseEngineException(new XException(ErrorCode.E0301, "cannot get a coordinator job from DagEngine"));
     }
 

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonCoordinatorJob.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonCoordinatorJob.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonCoordinatorJob.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonCoordinatorJob.java Wed Jun  5 17:26:19 2013
@@ -130,6 +130,8 @@ public class JsonCoordinatorJob implemen
     @Transient
     private int pending = 0;
 
+    @Transient
+    private int numActions = 0;
 
     public JsonCoordinatorJob() {
         actions = new ArrayList<JsonCoordinatorAction>();
@@ -168,6 +170,7 @@ public class JsonCoordinatorJob implemen
         json.put(JsonTags.COORDINATOR_JOB_MAT_THROTTLING, getMatThrottling());
         json.put(JsonTags.COORDINATOR_ACTIONS, JsonCoordinatorAction.toJSONArray(actions, timeZoneId));
         json.put(JsonTags.TO_STRING,toString());
+        json.put(JsonTags.COORDINATOR_JOB_NUM_ACTION, numActions);
 
         return json;
     }
@@ -417,4 +420,12 @@ public class JsonCoordinatorJob implemen
         this.pending = 0;
     }
 
+    public int getNumActions() {
+        return numActions;
+    }
+
+    public void setNumActions(int numAction) {
+        this.numActions = numAction;
+    }
+
 }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordJobXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordJobXCommand.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordJobXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/coord/CoordJobXCommand.java Wed Jun  5 17:26:19 2013
@@ -17,6 +17,7 @@
  */
 package org.apache.oozie.command.coord;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
@@ -26,6 +27,7 @@ import org.apache.oozie.ErrorCode;
 import org.apache.oozie.XException;
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
+import org.apache.oozie.executor.jpa.CoordActionsCountForJobIdJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetActionsSubsetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
 import org.apache.oozie.service.JPAService;
@@ -40,6 +42,7 @@ public class CoordJobXCommand extends Co
     private final boolean getActionInfo;
     private int start = 1;
     private int len = Integer.MAX_VALUE;
+    private boolean desc = false;
     private List<String> filterList;
 
     /**
@@ -48,7 +51,7 @@ public class CoordJobXCommand extends Co
      * @param id coord jobId
      */
     public CoordJobXCommand(String id) {
-        this(id, Collections.<String>emptyList(), 1, Integer.MAX_VALUE);
+        this(id, Collections.<String>emptyList(), 1, Integer.MAX_VALUE, false);
     }
 
     /**
@@ -58,13 +61,14 @@ public class CoordJobXCommand extends Co
      * @param start starting index in the list of actions belonging to the job
      * @param length number of actions to be returned
      */
-    public CoordJobXCommand(String id, List<String> filterList, int start, int length) {
+    public CoordJobXCommand(String id, List<String> filterList, int start, int length, boolean desc) {
         super("job.info", "job.info", 1);
         this.id = ParamChecker.notEmpty(id, "id");
         this.getActionInfo = true;
         this.filterList = filterList;
         this.start = start;
         this.len = length;
+        this.desc = desc;
     }
 
     /**
@@ -120,9 +124,17 @@ public class CoordJobXCommand extends Co
             if (jpaService != null) {
                 coordJob = jpaService.execute(new CoordJobGetJPAExecutor(id));
                 if (getActionInfo) {
-                    List<CoordinatorActionBean> coordActions = jpaService
-                            .execute(new CoordJobGetActionsSubsetJPAExecutor(id, filterList, start, len));
+                    int numAction = jpaService.execute(new CoordActionsCountForJobIdJPAExecutor(id));
+                    List<CoordinatorActionBean> coordActions = null;
+                    if (len == 0) {
+                        coordActions = new ArrayList<CoordinatorActionBean>();
+                    }
+                    else {
+                        coordActions = jpaService.execute(new CoordJobGetActionsSubsetJPAExecutor(id, filterList,
+                                start, len, desc));
+                    }
                     coordJob.setActions(coordActions);
+                    coordJob.setNumActions(numAction);
                 }
             }
             else {

Added: oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionsCountForJobIdJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionsCountForJobIdJPAExecutor.java?rev=1489973&view=auto
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionsCountForJobIdJPAExecutor.java (added)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionsCountForJobIdJPAExecutor.java Wed Jun  5 17:26:19 2013
@@ -0,0 +1,56 @@
+/**
+ * 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 javax.persistence.EntityManager;
+import javax.persistence.Query;
+
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.util.ParamChecker;
+
+/**
+ * Load the number of running actions for a coordinator job.
+ */
+public class CoordActionsCountForJobIdJPAExecutor implements JPAExecutor<Integer> {
+
+    private String coordJobId = null;
+
+    public CoordActionsCountForJobIdJPAExecutor(String coordJobId) {
+        ParamChecker.notNull(coordJobId, "coordJobId");
+        this.coordJobId = coordJobId;
+    }
+
+    @Override
+    public String getName() {
+        return "CoordActionsCountJPAExecutor";
+    }
+
+    @Override
+    public Integer execute(EntityManager em) throws JPAExecutorException {
+        try {
+            Query q = em.createNamedQuery("GET_COORD_ACTIONS_COUNT_BY_JOBID");
+            q.setParameter("jobId", coordJobId);
+            Long count = (Long) q.getSingleResult();
+            return Integer.valueOf(count.intValue());
+        }
+        catch (Exception e) {
+            throw new JPAExecutorException(ErrorCode.E0603, e.getMessage(), e);
+        }
+    }
+
+}

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsSubsetJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsSubsetJPAExecutor.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsSubsetJPAExecutor.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsSubsetJPAExecutor.java Wed Jun  5 17:26:19 2013
@@ -39,6 +39,7 @@ public class CoordJobGetActionsSubsetJPA
     private String coordJobId = null;
     private int start = 1;
     private int len = 50;
+    private boolean desc = false;
     private List<String> filterList;
 
     public CoordJobGetActionsSubsetJPAExecutor(String coordJobId) {
@@ -46,12 +47,13 @@ public class CoordJobGetActionsSubsetJPA
         this.coordJobId = coordJobId;
     }
 
-    public CoordJobGetActionsSubsetJPAExecutor(String coordJobId, List<String> filterList, int start, int len) {
+    public CoordJobGetActionsSubsetJPAExecutor(String coordJobId, List<String> filterList, int start, int len, boolean desc) {
         this(coordJobId);
         ParamChecker.notNull(filterList, "filterList");
         this.filterList = filterList;
         this.start = start;
         this.len = len;
+        this.desc = desc;
     }
 
     @Override
@@ -103,6 +105,9 @@ public class CoordJobGetActionsSubsetJPA
             sbTotal.insert(offset, statusClause);
             q = em.createQuery(sbTotal.toString());
         }
+        if (desc) {
+            q = em.createQuery(q.toString().concat(" desc"));
+        }
         q.setParameter("jobId", coordJobId);
         q.setFirstResult(start - 1);
         q.setMaxResults(len);

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java Wed Jun  5 17:26:19 2013
@@ -49,7 +49,8 @@ public abstract class BaseJobServlet ext
     static {
         RESOURCES_INFO[0] = new ResourceInfo("*", Arrays.asList("PUT", "GET"), Arrays.asList(new ParameterInfo(
                 RestConstants.ACTION_PARAM, String.class, true, Arrays.asList("PUT")), new ParameterInfo(
-                RestConstants.JOB_SHOW_PARAM, String.class, false, Arrays.asList("GET"))));
+                RestConstants.JOB_SHOW_PARAM, String.class, false, Arrays.asList("GET")), new ParameterInfo(
+                        RestConstants.ORDER_PARAM, String.class, false, Arrays.asList("GET"))));
     }
 
     public BaseJobServlet(String instrumentationName) {

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java Wed Jun  5 17:26:19 2013
@@ -779,7 +779,7 @@ public class V1JobServlet extends BaseJo
      * @throws XServletException
      * @throws BaseEngineException
      */
-    private JsonBean getCoordinatorJob(HttpServletRequest request, HttpServletResponse response)
+    protected JsonBean getCoordinatorJob(HttpServletRequest request, HttpServletResponse response)
             throws XServletException, BaseEngineException {
         JsonBean jobBean = null;
         CoordinatorEngine coordEngine = Services.get().get(CoordinatorEngineService.class).getCoordinatorEngine(
@@ -788,6 +788,8 @@ public class V1JobServlet extends BaseJo
         String startStr = request.getParameter(RestConstants.OFFSET_PARAM);
         String lenStr = request.getParameter(RestConstants.LEN_PARAM);
         String filter = request.getParameter(RestConstants.JOB_FILTER_PARAM);
+        String orderStr = request.getParameter(RestConstants.ORDER_PARAM);
+        boolean order = (orderStr != null && orderStr.equals("desc")) ? true : false;
         int start = (startStr != null) ? Integer.parseInt(startStr) : 1;
         start = (start < 1) ? 1 : start;
         // Get default number of coordinator actions to be retrieved
@@ -795,7 +797,7 @@ public class V1JobServlet extends BaseJo
         int len = (lenStr != null) ? Integer.parseInt(lenStr) : 0;
         len = (len < 1) ? defaultLen : len;
         try {
-            JsonCoordinatorJob coordJob = coordEngine.getCoordJob(jobId, filter, start, len);
+            JsonCoordinatorJob coordJob = coordEngine.getCoordJob(jobId, filter, start, len, order);
             jobBean = coordJob;
         }
         catch (CoordinatorEngineException ex) {

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleEngineSimple.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleEngineSimple.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleEngineSimple.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleEngineSimple.java Wed Jun  5 17:26:19 2013
@@ -44,7 +44,7 @@ public class TestBundleEngineSimple exte
     public void testGetCoordJob4() {
         BundleEngine be = new BundleEngine();
         try {
-            CoordinatorJob cj = be.getCoordJob("foo", "filter", 0, 1);
+            CoordinatorJob cj = be.getCoordJob("foo", "filter", 0, 1, false);
             fail("Expected BundleEngineException was not thrown.");
         }
         catch (BundleEngineException bee) {

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java Wed Jun  5 17:26:19 2013
@@ -444,28 +444,28 @@ public class TestCoordinatorEngine exten
     private void _testSubsetActions(final String jobId) throws Exception {
         CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
         // Check for WAITING filter
-        CoordinatorJob job = ce.getCoordJob(jobId, "status=WAITING", 1, 2);
+        CoordinatorJob job = ce.getCoordJob(jobId, "status=WAITING", 1, 2, false);
         // As both actions are waiting, expected result size is 2
         assertEquals(job.getActions().size(), 2);
 
-        job = ce.getCoordJob(jobId, "status=RUNNING", 1, 2);
+        job = ce.getCoordJob(jobId, "status=RUNNING", 1, 2, false);
         assertEquals(job.getActions().size(), 0);
 
         //Check for actions WAITING OR RUNNING
-        job = ce.getCoordJob(jobId, "status=RUNNING;status=WAITING", 1, 2);
+        job = ce.getCoordJob(jobId, "status=RUNNING;status=WAITING", 1, 2, false);
         assertEquals(job.getActions().size(), 2);
 
         //Check without filters
-        job = ce.getCoordJob(jobId, null, 1, 2);
+        job = ce.getCoordJob(jobId, null, 1, 2, false);
         assertEquals(job.getActions().size(), 2);
 
         //Check for empty filter list
-        job = ce.getCoordJob(jobId, "", 1, 2);
+        job = ce.getCoordJob(jobId, "", 1, 2, false);
         assertEquals(job.getActions().size(), 2);
 
         //Check for missing "="
         try {
-            job = ce.getCoordJob(jobId, "statusRUNNING", 1, 2);
+            job = ce.getCoordJob(jobId, "statusRUNNING", 1, 2, false);
         }
         catch (CoordinatorEngineException ex) {
             assertEquals(ErrorCode.E0421, ex.getErrorCode());
@@ -474,7 +474,7 @@ public class TestCoordinatorEngine exten
 
         //Check for missing value after "="
         try {
-            job = ce.getCoordJob(jobId, "status=", 1, 2);
+            job = ce.getCoordJob(jobId, "status=", 1, 2, false);
         }
         catch (CoordinatorEngineException ex) {
             assertEquals(ErrorCode.E0421, ex.getErrorCode());
@@ -483,7 +483,7 @@ public class TestCoordinatorEngine exten
 
         // Check for invalid status value
         try {
-            job = ce.getCoordJob(jobId, "status=blahblah", 1, 2);
+            job = ce.getCoordJob(jobId, "status=blahblah", 1, 2, false);
         }
         catch (CoordinatorEngineException ex) {
             assertEquals(ErrorCode.E0421, ex.getErrorCode());
@@ -492,7 +492,7 @@ public class TestCoordinatorEngine exten
 
         // Check for empty status value
         try {
-            job = ce.getCoordJob(jobId, "status=\"\"", 1, 2);
+            job = ce.getCoordJob(jobId, "status=\"\"", 1, 2, false);
         }
         catch (CoordinatorEngineException ex) {
             assertEquals(ErrorCode.E0421, ex.getErrorCode());
@@ -501,7 +501,7 @@ public class TestCoordinatorEngine exten
 
         // Check for invalid filter option
         try {
-            job = ce.getCoordJob(jobId, "blahblah=blahblah", 1, 2);
+            job = ce.getCoordJob(jobId, "blahblah=blahblah", 1, 2, false);
         }
         catch (CoordinatorEngineException ex) {
             assertEquals(ErrorCode.E0421, ex.getErrorCode());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobGetActionsSubsetJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobGetActionsSubsetJPAExecutor.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobGetActionsSubsetJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobGetActionsSubsetJPAExecutor.java Wed Jun  5 17:26:19 2013
@@ -88,7 +88,7 @@ public class TestCoordJobGetActionsSubse
         JPAService jpaService = Services.get().get(JPAService.class);
         assertNotNull(jpaService);
         CoordJobGetActionsSubsetJPAExecutor actionGetCmd = new CoordJobGetActionsSubsetJPAExecutor(jobId,
-                Collections.<String> emptyList(), start, len);
+                Collections.<String> emptyList(), start, len, false);
         List<CoordinatorActionBean> actions = jpaService.execute(actionGetCmd);
         CoordinatorActionBean action = actions.get(0);
 
@@ -112,32 +112,54 @@ public class TestCoordJobGetActionsSubse
     public void testCoordActionOrderBy() throws Exception {
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
         // Add Coordinator action with nominal time: 2009-12-15T01:00Z
-        CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.WAITING, "coord-action-get.xml", 0);
+        CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.WAITING,
+                "coord-action-get.xml", 0);
         // Add Coordinator action with nominal time: 2009-02-01T23:59Z
-        CoordinatorActionBean action1 = addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.WAITING, "coord-action-for-action-input-check.xml", 0);
+        CoordinatorActionBean action1 = addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.WAITING,
+                "coord-action-for-action-input-check.xml", 0);
         // test for the expected action number
-        _testGetActionsSubsetOrderBy(job.getId(), 2, 1, 2);
+        List<CoordinatorActionBean> actions = _testGetActionsSubsetOrderBy(job.getId(), 1, 2, false);
+        assertEquals(actions.size(), 2);
+        // As actions are sorted by nominal time, the first action should be
+        // with action number 2
+        assertEquals(actions.get(0).getActionNumber(), 2);
     }
 
-   private void _testGetActionsSubsetOrderBy(String jobId, int actionNum, int start, int len) throws Exception {
+    // Check the ordering of actions by nominal time
+    public void testCoordActionOrderByDesc() throws Exception {
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
+        // Add Coordinator action with nominal time: 2009-12-15T01:00Z
+        CoordinatorActionBean action = addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.WAITING,
+                "coord-action-get.xml", 0);
+        // Add Coordinator action with nominal time: 2009-02-01T23:59Z
+        CoordinatorActionBean action1 = addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.WAITING,
+                "coord-action-for-action-input-check.xml", 0);
+        // test for the expected action number
+        List<CoordinatorActionBean> actions = _testGetActionsSubsetOrderBy(job.getId(), 1, 2, true);
+        assertEquals(actions.size(), 2);
+        // As actions are sorted by nominal time in desc order, the first action
+        // should be with action number 1
+        assertEquals(actions.get(0).getActionNumber(), 1);
+    }
+
+    private List<CoordinatorActionBean> _testGetActionsSubsetOrderBy(String jobId, int start, int len, boolean order)
+            throws Exception {
         JPAService jpaService = Services.get().get(JPAService.class);
         assertNotNull(jpaService);
-        CoordJobGetActionsSubsetJPAExecutor actionGetCmd = new CoordJobGetActionsSubsetJPAExecutor(jobId, Collections.<String>emptyList(), start, len);
+        CoordJobGetActionsSubsetJPAExecutor actionGetCmd = new CoordJobGetActionsSubsetJPAExecutor(jobId,
+                Collections.<String> emptyList(), start, len, order);
         List<CoordinatorActionBean> actions = jpaService.execute(actionGetCmd);
-        assertEquals(actions.size(), 2);
-        // As actions are sorted by nominal time, the first action should be with action number 2
-        assertEquals(actions.get(0).getActionNumber(), actionNum);
+        return actions;
+
     }
 
     // Check status filters for Coordinator actions
-    public void testCoordActionFilter() throws Exception{
+    public void testCoordActionFilter() throws Exception {
         CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
         // Add Coordinator action with nominal time: 2009-12-15T01:00Z
-        addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.RUNNING,
-                "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
         // Add Coordinator action with nominal time: 2009-02-01T23:59Z
-        addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.WAITING,
-                "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.WAITING, "coord-action-get.xml", 0);
         // Create lists for status filter
         List<String> filterList = new ArrayList<String>();
         filterList.add("RUNNING");
@@ -151,14 +173,15 @@ public class TestCoordJobGetActionsSubse
         JPAService jpaService = Services.get().get(JPAService.class);
         assertNotNull(jpaService);
         CoordJobGetActionsSubsetJPAExecutor actionGetCmd = new CoordJobGetActionsSubsetJPAExecutor(jobId, filterList,
-                start, len);
+                start, len, false);
         List<CoordinatorActionBean> actions = jpaService.execute(actionGetCmd);
-        // As actions are filtered by RUNNING status, only 1 action should be returned
+        // As actions are filtered by RUNNING status, only 1 action should be
+        // returned
         assertEquals(actions.size(), 1);
         assertEquals(actions.get(0).getActionNumber(), 1);
     }
 
-    public void testGetActionAllColumns() throws Exception{
+    public void testGetActionAllColumns() throws Exception {
         services.destroy();
         setSystemProperty(CoordActionGetForInfoJPAExecutor.COORD_GET_ALL_COLS_FOR_ACTION, "true");
         services = new Services();
@@ -178,7 +201,7 @@ public class TestCoordJobGetActionsSubse
         JPAService jpaService = Services.get().get(JPAService.class);
         assertNotNull(jpaService);
         CoordJobGetActionsSubsetJPAExecutor actionGetCmd = new CoordJobGetActionsSubsetJPAExecutor(jobId,
-                Collections.<String> emptyList(), start, len);
+                Collections.<String> emptyList(), start, len, false);
         List<CoordinatorActionBean> actions = jpaService.execute(actionGetCmd);
         CoordinatorActionBean action = actions.get(0);
 

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java Wed Jun  5 17:26:19 2013
@@ -161,7 +161,8 @@ public class MockCoordinatorEngineServic
         }
 
         @Override
-        public CoordinatorJobBean getCoordJob(String jobId, String filter, int start, int length) throws BaseEngineException {
+        public CoordinatorJobBean getCoordJob(String jobId, String filter, int start, int length, boolean desc)
+                throws BaseEngineException {
             did = RestConstants.JOB_SHOW_INFO;
             int idx = validateCoordinatorIdx(jobId);
             return (CoordinatorJobBean) coordJobs.get(idx);

Modified: oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/trunk/release-log.txt?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/release-log.txt (original)
+++ oozie/trunk/release-log.txt Wed Jun  5 17:26:19 2013
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1357 Can't view more than 1000 actions of a coordinator and paging does not work (ryota)
 OOZIE-1381 Oozie does not support access to the distributed cache file under different name node (ryota)
 OOZIE-1298 TestPartitionDependencyManagerEhcache.testEvictionOnTimeToIdle is flakey (rohini)
 OOZIE-1397 failure in running test cases (aklochkov via rohini)

Modified: oozie/trunk/webapp/src/main/webapp/oozie-console.js
URL: http://svn.apache.org/viewvc/oozie/trunk/webapp/src/main/webapp/oozie-console.js?rev=1489973&r1=1489972&r2=1489973&view=diff
==============================================================================
--- oozie/trunk/webapp/src/main/webapp/oozie-console.js (original)
+++ oozie/trunk/webapp/src/main/webapp/oozie-console.js Wed Jun  5 17:26:19 2013
@@ -1,5 +1,5 @@
 /**
- * Licensed to the Apache Software Foundation (ASF) under one
+' * 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
@@ -777,9 +777,16 @@ function coordJobDetailsPopup(response, 
     var coordJobId = jobDetails["coordJobId"];
     var appName = jobDetails["coordJobName"];
     var jobActionStatus = new Ext.data.JsonStore({
-        data: jobDetails["actions"],
-        fields: ['id', 'name', 'type', 'createdConf', 'runConf', 'actionNumber', 'createdTime', 'externalId', 'lastModifiedTime', 'nominalTime', 'status', 'missingDependencies', 'externalStatus', 'trackerUri', 'consoleUrl', 'errorCode', 'errorMessage', 'actions', 'externalChildIDs']
-
+        autoLoad: {params:{offset: 0, len: 50}},
+        totalProperty: 'total',
+        root: 'actions',
+        fields: ['id', 'name', 'type', 'createdConf', 'runConf', 'actionNumber', 'createdTime', 'externalId',
+                 'lastModifiedTime', 'nominalTime', 'status', 'missingDependencies', 'externalStatus', 'trackerUri',
+                 'consoleUrl', 'errorCode', 'errorMessage', 'actions', 'externalChildIDs'],
+        proxy: new Ext.data.HttpProxy({
+           url: getOozieBase() + 'job/' + coordJobId + "?timezone=" + getTimeZone() + "&order=desc",
+           method: 'GET'
+        })
     });
 
     var formFieldSet = new Ext.form.FieldSet({
@@ -872,7 +879,6 @@ function coordJobDetailsPopup(response, 
                     url: getOozieBase() + 'job/' + coordJobId + "?timezone=" + getTimeZone(),
                     success: function(response, request) {
                         jobDetails = eval("(" + response.responseText + ")");
-                        jobActionStatus.loadData(jobDetails["actions"]);
                         fs.getForm().setValues(jobDetails);
                     }
 
@@ -928,6 +934,7 @@ function coordJobDetailsPopup(response, 
         height: 400,
         width: 1600,
         title: 'Actions',
+        bbar: getPagingBar(jobActionStatus),
         listeners: {
             cellclick: {
                 fn: showWorkflowPopup
@@ -1389,7 +1396,7 @@ function coordJobDetailsGridWindow(coord
          icon: Ext.MessageBox.INFO
          });
          */
-        url: getOozieBase() + 'job/' + coordJobId + "?timezone=" + getTimeZone(),
+        url: getOozieBase() + 'job/' + coordJobId + "?timezone=" + getTimeZone() + "&offset=0&len=0",
         success: coordJobDetailsPopup
     });
 }