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 2015/02/03 21:13:04 UTC

oozie git commit: OOZIE-2118 add createdtime option to workflow jobs query (ryota)

Repository: oozie
Updated Branches:
  refs/heads/master e589521a5 -> c6afe1c11


OOZIE-2118 add createdtime option to workflow jobs query (ryota)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/c6afe1c1
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/c6afe1c1
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/c6afe1c1

Branch: refs/heads/master
Commit: c6afe1c1135f01f6d8ffb45311a6cf0679503220
Parents: e589521
Author: egashira <ry...@yahoo.com>
Authored: Tue Feb 3 12:06:36 2015 -0800
Committer: egashira <ry...@yahoo.com>
Committed: Tue Feb 3 12:06:36 2015 -0800

----------------------------------------------------------------------
 .../java/org/apache/oozie/cli/OozieCLI.java     |   7 +-
 .../org/apache/oozie/client/OozieClient.java    |   4 +
 .../main/java/org/apache/oozie/DagEngine.java   |   3 +
 .../jpa/WorkflowsJobGetJPAExecutor.java         | 106 +++++++++-
 .../apache/oozie/util/XLogUserFilterParam.java  |   2 -
 .../org/apache/oozie/client/TestOozieCLI.java   |  11 ++
 .../jpa/TestWorkflowsJobGetJPAExecutor.java     | 196 ++++++++++++++++++-
 .../apache/oozie/servlet/TestV1JobsServlet.java |  11 ++
 docs/src/site/twiki/DG_CommandLineTool.twiki    |  17 +-
 docs/src/site/twiki/WebServicesAPI.twiki        |   4 +
 release-log.txt                                 |   1 +
 11 files changed, 352 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
index 0c28732..6690869 100644
--- a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
+++ b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -414,8 +414,11 @@ public class OozieCLI {
         Option jobtype = new Option(JOBTYPE_OPTION, true,
                 "job type ('Supported in Oozie-2.0 or later versions ONLY - 'coordinator' or 'bundle' or 'wf'(default))");
         Option len = new Option(LEN_OPTION, true, "number of jobs (default '100')");
-        Option filter = new Option(FILTER_OPTION, true, "user=<U>\\;name=<N>\\;group=<G>\\;status=<S>\\;frequency=<F>\\;unit=<M> " +
-                        "(Valid unit values are 'months', 'days', 'hours' or 'minutes'.)");
+        Option filter = new Option(FILTER_OPTION, true,
+                "user=<U>\\;name=<N>\\;group=<G>\\;status=<S>\\;frequency=<F>\\;unit=<M>" +
+                        "\\;startcreatedtime=<SC>\\;endcreatedtime=<EC> " +
+                        "(valid unit values are 'months', 'days', 'hours' or 'minutes'. " +
+                        "startcreatedtime, endcreatedtime: time of format yyyy-MM-dd'T'HH:mm'Z')");
         Option localtime = new Option(LOCAL_TIME_OPTION, false, "use local time (same as passing your time zone to -" +
                 TIME_ZONE_OPTION + "). Overrides -" + TIME_ZONE_OPTION + " option");
         Option timezone = new Option(TIME_ZONE_OPTION, true,

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/client/src/main/java/org/apache/oozie/client/OozieClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/OozieClient.java b/client/src/main/java/org/apache/oozie/client/OozieClient.java
index 29b1131..e4c93cd 100644
--- a/client/src/main/java/org/apache/oozie/client/OozieClient.java
+++ b/client/src/main/java/org/apache/oozie/client/OozieClient.java
@@ -151,6 +151,10 @@ public class OozieClient {
 
     public static final String FILTER_SLA_NOMINAL_END = "nominal_end";
 
+    public static final String FILTER_CREATED_TIME_START = "startcreatedtime";
+
+    public static final String FILTER_CREATED_TIME_END = "endcreatedtime";
+
     public static final String CHANGE_VALUE_ENDTIME = "endtime";
 
     public static final String CHANGE_VALUE_PAUSETIME = "pausetime";

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/core/src/main/java/org/apache/oozie/DagEngine.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/DagEngine.java b/core/src/main/java/org/apache/oozie/DagEngine.java
index cd90390..50aef2f 100644
--- a/core/src/main/java/org/apache/oozie/DagEngine.java
+++ b/core/src/main/java/org/apache/oozie/DagEngine.java
@@ -448,6 +448,8 @@ public class DagEngine extends BaseEngine {
         FILTER_NAMES.add(OozieClient.FILTER_GROUP);
         FILTER_NAMES.add(OozieClient.FILTER_STATUS);
         FILTER_NAMES.add(OozieClient.FILTER_ID);
+        FILTER_NAMES.add(OozieClient.FILTER_CREATED_TIME_START);
+        FILTER_NAMES.add(OozieClient.FILTER_CREATED_TIME_END);
     }
 
     /**
@@ -468,6 +470,7 @@ public class DagEngine extends BaseEngine {
                     if (pair.length != 2) {
                         throw new DagEngineException(ErrorCode.E0420, filter, "elements must be name=value pairs");
                     }
+                    pair[0] = pair[0].toLowerCase();
                     if (!FILTER_NAMES.contains(pair[0])) {
                         throw new DagEngineException(ErrorCode.E0420, filter, XLog
                                 .format("invalid name [{0}]", pair[0]));

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java b/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
index 5c0c0f9..00d2263 100644
--- a/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
+++ b/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
@@ -18,18 +18,24 @@
 
 package org.apache.oozie.executor.jpa;
 
+import java.io.IOException;
 import java.sql.Timestamp;
+import java.text.ParseException;
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.List;
 import java.util.Map;
 
 import javax.persistence.EntityManager;
 import javax.persistence.Query;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.oozie.ErrorCode;
 import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.WorkflowsInfo;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.WorkflowJob.Status;
+import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.util.XLog;
 import org.apache.openjpa.persistence.OpenJPAPersistence;
 import org.apache.openjpa.persistence.OpenJPAQuery;
@@ -69,7 +75,7 @@ public class WorkflowsJobGetJPAExecutor implements JPAExecutor<WorkflowsInfo> {
     public WorkflowsInfo execute(EntityManager em) throws JPAExecutorException {
         List<String> orArray = new ArrayList<String>();
         List<String> colArray = new ArrayList<String>();
-        List<String> valArray = new ArrayList<String>();
+        List<Object> valArray = new ArrayList<Object>();
         StringBuilder sb = new StringBuilder("");
         boolean isStatus = false;
         boolean isAppName = false;
@@ -210,10 +216,68 @@ public class WorkflowsJobGetJPAExecutor implements JPAExecutor<WorkflowsInfo> {
                             colArray.add(colVar);
                         }
                     }
+                    if (entry.getKey().equalsIgnoreCase(OozieClient.FILTER_CREATED_TIME_START)) {
+                        List<String> values = filter.get(OozieClient.FILTER_CREATED_TIME_START);
+                        colName = "createdTimestampStart";
+                        if (values.size() > 1) {
+                            throw new JPAExecutorException(ErrorCode.E0302,
+                                    "cannot specify multiple startcreatedtime");
+                        }
+                        colVar = colName;
+                        colVar = colVar + index;
+                        if (!isEnabled) {
+                            sb.append(seletStr).append(" where w.createdTimestamp >= :" + colVar);
+                            isEnabled = true;
+                        }
+                        else {
+                            sb.append(" and w.createdTimestamp >= :" + colVar);
+                        }
+                        index++;
+                        Date createdTime = null;
+                        try {
+                            createdTime = parseCreatedTimeString(values.get(0));
+                        }
+                        catch (Exception e) {
+                            throw new JPAExecutorException(ErrorCode.E0302, e.getMessage());
+                        }
+                        Timestamp createdTimeStamp = new Timestamp(createdTime.getTime());
+                        valArray.add(createdTimeStamp);
+                        orArray.add(colName);
+                        colArray.add(colVar);
+
+                    }
+                    if (entry.getKey().equalsIgnoreCase(OozieClient.FILTER_CREATED_TIME_END)) {
+                        List<String> values = filter.get(OozieClient.FILTER_CREATED_TIME_END);
+                        colName = "createdTimestampEnd";
+                        if (values.size() > 1) {
+                            throw new JPAExecutorException(ErrorCode.E0302,
+                                    "cannot specify multiple endcreatedtime");
+                        }
+                        colVar = colName;
+                        colVar = colVar + index;
+                        if (!isEnabled) {
+                            sb.append(seletStr).append(" where w.createdTimestamp <= :" + colVar);
+                            isEnabled = true;
+                        }
+                        else {
+                            sb.append(" and w.createdTimestamp <= :" + colVar);
+                        }
+                        index++;
+                        Date createdTime = null;
+                        try {
+                            createdTime = parseCreatedTimeString(values.get(0));
+                        }
+                        catch (Exception e) {
+                            throw new JPAExecutorException(ErrorCode.E0302, e.getMessage());
+                        }
+                        Timestamp createdTimeStamp = new Timestamp(createdTime.getTime());
+                        valArray.add(createdTimeStamp);
+                        orArray.add(colName);
+                        colArray.add(colVar);
+                    }
                 }
             }
         }
-
         int realLen = 0;
 
         Query q = null;
@@ -232,6 +296,7 @@ public class WorkflowsJobGetJPAExecutor implements JPAExecutor<WorkflowsInfo> {
                 q.setFirstResult(start - 1);
                 q.setMaxResults(len);
                 qTotal = em.createQuery(sbTotal.toString().replace(seletStr, countStr));
+
                 for (int i = 0; i < orArray.size(); i++) {
                     q.setParameter(colArray.get(i), valArray.get(i));
                     qTotal.setParameter(colArray.get(i), valArray.get(i));
@@ -267,6 +332,43 @@ public class WorkflowsJobGetJPAExecutor implements JPAExecutor<WorkflowsInfo> {
         return "WorkflowsJobGetJPAExecutor";
     }
 
+    private Date parseCreatedTimeString(String time) throws Exception{
+        Date createdTime = null;
+        int offset = 0;
+        if (Character.isLetter(time.charAt(time.length() - 1))) {
+            switch (time.charAt(time.length() - 1)) {
+                case 'd':
+                    offset = Integer.parseInt(time.substring(0, time.length() - 1));
+                    if(offset > 0) {
+                        throw new IllegalArgumentException("offset must be minus from currentTime");
+                    }
+                    createdTime = org.apache.commons.lang.time.DateUtils.addDays(new Date(), offset);
+                    break;
+                case 'h':
+                    offset =  Integer.parseInt(time.substring(0, time.length() - 1));
+                    if(offset > 0) {
+                        throw new IllegalArgumentException("offset must be minus from currentTime");
+                    }
+                    createdTime = org.apache.commons.lang.time.DateUtils.addHours(new Date(), offset);
+                    break;
+                case 'm':
+                    offset =  Integer.parseInt(time.substring(0, time.length() - 1));
+                    if(offset > 0) {
+                        throw new IllegalArgumentException("offset must be minus from currentTime");
+                    }
+                    createdTime = org.apache.commons.lang.time.DateUtils.addMinutes(new Date(), offset);
+                    break;
+                case 'Z':
+                    createdTime = DateUtils.parseDateUTC(time);
+                    break;
+                default:
+                    throw new IllegalArgumentException("Unsupported time format " + time);
+            }
+        } else {
+            throw new IllegalArgumentException("the format of createdTime is wrong: " + time);
+        }
+        return createdTime;
+    }
     private WorkflowJobBean getBeanForWorkflowFromArray(Object[] arr) {
 
         WorkflowJobBean wfBean = new WorkflowJobBean();

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java b/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java
index a00a79f..bd7bbfd 100644
--- a/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java
+++ b/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java
@@ -276,10 +276,8 @@ public class XLogUserFilterParam {
 
         if (Character.isLetter(offset.charAt(offset.length() - 1))) {
             switch (offset.charAt(offset.length() - 1)) {
-                case 'H':
                 case 'h':
                     return Integer.parseInt(offset.substring(0, offset.length() - 1)) * 60;
-                case 'M':
                 case 'm':
                     return Integer.parseInt(offset.substring(0, offset.length() - 1));
                 default:

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java b/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
index ac54b09..f81c1ad 100644
--- a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
+++ b/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
@@ -814,6 +814,17 @@ public class TestOozieCLI extends DagServletTestCase {
                 args = new String[]{"jobs", "-jobtype", "coord",  "-filter", "status=FAILED", "-oozie", oozieUrl};
                 assertEquals(0, new OozieCLI().run(args));
                 assertEquals(RestConstants.JOBS_FILTER_PARAM, MockDagEngineService.did);
+
+                args = new String[] { "jobs", "-filter",
+                        "startcreatedtime=2014-04-01T00:00Z;endcreatedtime=2014-05-01T00:00Z", "-oozie", oozieUrl };
+                assertEquals(0, new OozieCLI().run(args));
+                assertEquals(RestConstants.JOBS_FILTER_PARAM, MockDagEngineService.did);
+
+                args = new String[] { "jobs", "-filter",
+                        "startcreatedtime=-10d;endcreatedtime=-20m", "-oozie", oozieUrl };
+                assertEquals(0, new OozieCLI().run(args));
+                assertEquals(RestConstants.JOBS_FILTER_PARAM, MockDagEngineService.did);
+
                 return null;
             }
         });

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowsJobGetJPAExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowsJobGetJPAExecutor.java b/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowsJobGetJPAExecutor.java
index 6250255..46b0d05 100644
--- a/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowsJobGetJPAExecutor.java
+++ b/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowsJobGetJPAExecutor.java
@@ -19,19 +19,22 @@
 package org.apache.oozie.executor.jpa;
 
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.oozie.ErrorCode;
 import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.WorkflowsInfo;
+import org.apache.oozie.XException;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.WorkflowJob;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.test.XDataTestCase;
-import org.apache.oozie.workflow.WorkflowException;
+import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.workflow.WorkflowInstance;
 
 public class TestWorkflowsJobGetJPAExecutor extends XDataTestCase {
@@ -103,4 +106,195 @@ public class TestWorkflowsJobGetJPAExecutor extends XDataTestCase {
         assertEquals(wfBean.getExternalId(), retBean.getExternalId());
         assertEquals(wfBean.getParentId(), retBean.getParentId());
     }
+
+    public void testWfJobsGetWithCreatedTime() throws Exception {
+        JPAService jpaService = Services.get().get(JPAService.class);
+        Date createdTime1 = DateUtils.parseDateUTC("2012-01-01T10:00Z");
+        Date createdTime2 = DateUtils.parseDateUTC("2012-02-01T10:00Z");
+        Date createdTime3 = DateUtils.parseDateUTC("2012-03-01T10:00Z");
+        Date createdTime4 = DateUtils.parseDateUTC("2012-04-01T10:00Z");
+        Date createdTime5 = org.apache.commons.lang.time.DateUtils.addHours(new Date(), -5);
+        WorkflowJobBean wrkJob = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        wrkJob.setCreatedTime(createdTime1);
+        WorkflowJobQueryExecutor.getInstance().executeUpdate(WorkflowJobQuery.UPDATE_WORKFLOW, wrkJob);
+        wrkJob = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        wrkJob.setCreatedTime(createdTime2);
+        WorkflowJobQueryExecutor.getInstance().executeUpdate(WorkflowJobQuery.UPDATE_WORKFLOW, wrkJob);
+        wrkJob = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        wrkJob.setCreatedTime(createdTime3);
+        WorkflowJobQueryExecutor.getInstance().executeUpdate(WorkflowJobQuery.UPDATE_WORKFLOW, wrkJob);
+        wrkJob = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        wrkJob.setCreatedTime(createdTime4);
+        WorkflowJobQueryExecutor.getInstance().executeUpdate(WorkflowJobQuery.UPDATE_WORKFLOW, wrkJob);
+        WorkflowJobQueryExecutor.getInstance().executeUpdate(WorkflowJobQuery.UPDATE_WORKFLOW, wrkJob);
+        wrkJob = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        wrkJob.setCreatedTime(createdTime5);
+        WorkflowJobQueryExecutor.getInstance().executeUpdate(WorkflowJobQuery.UPDATE_WORKFLOW, wrkJob);
+
+        // filter 2012-02-01T00:00Z <= createdTimeStamp <= 2012-03-02T00:00Z
+        Map<String, List<String>> filter = new HashMap<String, List<String>>();
+        List<String> timeList = new ArrayList<String>();
+        timeList.add("2012-02-01T00:00Z");
+        filter.put(OozieClient.FILTER_CREATED_TIME_START, timeList);
+        timeList = new ArrayList<String>();
+        timeList.add("2012-03-02T00:00Z");
+        filter.put(OozieClient.FILTER_CREATED_TIME_END, timeList);
+        WorkflowsJobGetJPAExecutor wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        WorkflowsInfo wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(2, wfInfo.getWorkflows().size());
+        for(WorkflowJobBean jobBean: wfInfo.getWorkflows()) {
+            assertTrue(jobBean.getCreatedTime().equals(createdTime2) || jobBean.getCreatedTime().equals(createdTime3));
+        }
+
+        //createdTime specified in wrong format
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("2012-02-01T00:00"); // not correct ISO8601_UTC_MASK format
+        filter.put(OozieClient.FILTER_CREATED_TIME_START, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        try{
+            wfInfo = jpaService.execute(wfGetCmd);
+        } catch (XException e) {
+            assertEquals(ErrorCode.E0302, e.getErrorCode());
+        }
+
+        // filter 2012-04-01T00:00Z <= createdTimeStamp
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("2012-04-01T00:00Z");
+        filter.put(OozieClient.FILTER_CREATED_TIME_START, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(2, wfInfo.getWorkflows().size());
+        for(WorkflowJobBean jobBean: wfInfo.getWorkflows()) {
+            assertTrue(jobBean.getCreatedTime().equals(createdTime4) || jobBean.getCreatedTime().equals(createdTime5));
+        }
+
+        // filter 2012-02-01T00:00Z >= createdTimeStamp
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("2012-02-01T00:00Z");
+        filter.put(OozieClient.FILTER_CREATED_TIME_END, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(1, wfInfo.getWorkflows().size());
+        assertTrue(wfInfo.getWorkflows().get(0).getCreatedTime().equals(createdTime1));
+
+        // when createdTimeStamp specified multiple times
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("2012-02-01T00:00Z");
+        timeList.add("2012-04-01T00:00Z");
+        filter.put(OozieClient.FILTER_CREATED_TIME_END, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        try{
+            wfInfo = jpaService.execute(wfGetCmd);
+        } catch (XException e) {
+            assertEquals(ErrorCode.E0302, e.getErrorCode());
+        }
+
+        // filter createdTimeStamp > currentTime - 2 days
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("-2d");
+        filter.put(OozieClient.FILTER_CREATED_TIME_START, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(1, wfInfo.getWorkflows().size());
+        assertTrue(wfInfo.getWorkflows().get(0).getCreatedTime().equals(createdTime5));
+
+        // filter createdTimeStamp > currentTime - 10 hour
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("-10h");
+        filter.put(OozieClient.FILTER_CREATED_TIME_START, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(1, wfInfo.getWorkflows().size());
+        assertTrue(wfInfo.getWorkflows().get(0).getCreatedTime().equals(createdTime5));
+
+        // filter createdTimeStamp > currentTime - 600 min
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("-600m");
+        filter.put(OozieClient.FILTER_CREATED_TIME_START, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(1, wfInfo.getWorkflows().size());
+        assertTrue(wfInfo.getWorkflows().get(0).getCreatedTime().equals(createdTime5));
+
+        // filter createdTimeStamp < currentTime - 3 days
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("-3d");
+        filter.put(OozieClient.FILTER_CREATED_TIME_END, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(4, wfInfo.getWorkflows().size());
+        for (WorkflowJobBean jobBean : wfInfo.getWorkflows()) {
+            assertTrue(jobBean.getCreatedTime().equals(createdTime1) || jobBean.getCreatedTime().equals(createdTime2)
+                    || jobBean.getCreatedTime().equals(createdTime3) || jobBean.getCreatedTime().equals(createdTime4));
+        }
+
+        // filter createdTimeStamp < currentTime - 2 hours
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("-2h");
+        filter.put(OozieClient.FILTER_CREATED_TIME_END, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(5, wfInfo.getWorkflows().size());
+        for (WorkflowJobBean jobBean : wfInfo.getWorkflows()) {
+            assertTrue(jobBean.getCreatedTime().equals(createdTime1) || jobBean.getCreatedTime().equals(createdTime2)
+                    || jobBean.getCreatedTime().equals(createdTime3) || jobBean.getCreatedTime().equals(createdTime4)
+                    || jobBean.getCreatedTime().equals(createdTime5));
+        }
+
+        // filter createdTimeStamp < currentTime - 60 min
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("-60m");
+        filter.put(OozieClient.FILTER_CREATED_TIME_END, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        wfInfo = jpaService.execute(wfGetCmd);
+        assertNotNull(wfInfo);
+        assertEquals(5, wfInfo.getWorkflows().size());
+        for (WorkflowJobBean jobBean : wfInfo.getWorkflows()) {
+            assertTrue(jobBean.getCreatedTime().equals(createdTime1) || jobBean.getCreatedTime().equals(createdTime2)
+                    || jobBean.getCreatedTime().equals(createdTime3) || jobBean.getCreatedTime().equals(createdTime4)
+                    || jobBean.getCreatedTime().equals(createdTime5));
+        }
+
+        // when offset is in wrong format
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("-5M");
+        filter.put(OozieClient.FILTER_CREATED_TIME_END, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        try{
+            wfInfo = jpaService.execute(wfGetCmd);
+        } catch (XException e) {
+            assertEquals(ErrorCode.E0302, e.getErrorCode());
+        }
+
+     // when offset is in wrong format
+        filter = new HashMap<String, List<String>>();
+        timeList = new ArrayList<String>();
+        timeList.add("5D");
+        filter.put(OozieClient.FILTER_CREATED_TIME_END, timeList);
+        wfGetCmd = new WorkflowsJobGetJPAExecutor(filter, 1, 10);
+        try{
+            wfInfo = jpaService.execute(wfGetCmd);
+        } catch (XException e) {
+            assertEquals(ErrorCode.E0302, e.getErrorCode());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java b/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java
index e84c0f0..fb481e3 100644
--- a/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java
+++ b/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java
@@ -235,6 +235,17 @@ public class TestV1JobsServlet extends DagServletTestCase {
                 obj = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
                 assertNull(obj.get(JsonTags.JOB_ID));
 
+                params = new HashMap<String, String>();
+                params.put(RestConstants.JOBS_FILTER_PARAM,
+                        "startCreatedTime=2000-01-01T00:00Z;endCreatedTime=2100-12-31T00:00Z");
+                url = createURL("", params);
+                conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("GET");
+                assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                assertTrue(conn.getHeaderField("content-type").startsWith(RestConstants.JSON_CONTENT_TYPE));
+                json = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
+                array = (JSONArray) json.get(JsonTags.WORKFLOWS_JOBS);
+                assertEquals(MockDagEngineService.INIT_WF_COUNT, array.size());
                 return null;
             }
         });

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/docs/src/site/twiki/DG_CommandLineTool.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/DG_CommandLineTool.twiki b/docs/src/site/twiki/DG_CommandLineTool.twiki
index e801e31..0f1768b 100644
--- a/docs/src/site/twiki/DG_CommandLineTool.twiki
+++ b/docs/src/site/twiki/DG_CommandLineTool.twiki
@@ -95,7 +95,7 @@ usage:
       oozie jobs <OPTIONS> : jobs status
                  -auth <arg>          select authentication type [SIMPLE|KERBEROS]
                  -doas <arg>          doAs user, impersonates as the specified user.
-                 -filter <arg>        user=<U>\;name=<N>\;group=<G>\;status=<S>\;...
+                 -filter <arg>        user=<U>\;name=<N>\;group=<G>\;status=<S>\;frequency=<F>\;unit=<M>\;startcreatedtime=<SC>\;endcreatedtime=<EC>
                  -jobtype <arg>       job type ('Supported in Oozie-2.0 or later versions ONLY - coordinator' or 'wf' (default))
                  -len <arg>           number of jobs (default '100')
                  -localtime           use local time (same as passing your time zone to -timezone). Overrides -timezone option
@@ -924,12 +924,14 @@ Valid filter names are:
    * user: the user that submitted the job.
    * group: the group for the job.
    * status: the status of the job.
-   * frequency: the frequency of the Coordinator job.
-   * unit: the time unit. It can take one of the following four values: months, days, hours or minutes. Time unit should be added only when frequency is specified.
+   * startcreatedtime: the start of time window in specifying createdtime range filter.
+   * endcreatedtime: the end of time window in specifying createdtime range filter
 
 The query will do an AND among all the filter names. The query will do an OR among all the filter values for the same
 name. Multiple values must be specified as different name value pairs.
 
+startCreatedTime and endCreatedTime should be specified either in *ISO8601 (UTC)* format (*yyyy-MM-dd'T'HH:mm'Z'*) or a offset value in days or hours from the current time. for example, -2d means the current time - 2 days. -3h means the current time - 3 hours, -5m means the current time - 5 minutes
+
 ---+++ Checking the Status of multiple Coordinator Jobs
 
 * This feature is only supported in Oozie 2.0 or later.
@@ -949,6 +951,15 @@ Job ID
 
 The =jobtype= option specified the job type to display, default value is 'wf'. To see the coordinator jobs, value is 'coordinator'.
 
+Valid filter names are:
+
+   * name: the workflow application name from the workflow definition.
+   * user: the user that submitted the job.
+   * group: the group for the job.
+   * status: the status of the job.
+   * frequency: the frequency of the Coordinator job.
+   * unit: the time unit. It can take one of the following four values: months, days, hours or minutes. Time unit should be added only when frequency is specified.
+
 ---+++ Checking the Status of multiple Bundle Jobs
 
 * This feature is only supported in Oozie 3.0 or later.

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/docs/src/site/twiki/WebServicesAPI.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/WebServicesAPI.twiki b/docs/src/site/twiki/WebServicesAPI.twiki
index fcefb4a..c301b78 100644
--- a/docs/src/site/twiki/WebServicesAPI.twiki
+++ b/docs/src/site/twiki/WebServicesAPI.twiki
@@ -1596,6 +1596,8 @@ Valid filter names are:
    * user: the user that submitted the job
    * group: the group for the job
    * status: the status of the job
+   * startCreatedTime : the start of the window about workflow job's created time
+   * endCreatedTime : the end of above window
 
 The query will do an AND among all the filter names.
 
@@ -1607,6 +1609,8 @@ Additionally the =offset= and =len= parameters can be used for pagination. The s
 Moreover, the =jobtype= parameter could be used to determine what type of job is looking for.
 The valid values of job type are: =wf=, =coordinator= or =bundle=.
 
+startCreatedTime and endCreatedTime should be specified either in *ISO8601 (UTC)* format (*yyyy-MM-dd'T'HH:mm'Z'*) or a offset value in days or hours from the current time. for example, -2d means the current time - 2 days. -3h means the current time - 3 hours. -5m means the current time - 5 minutes
+
 ---++++ Jobs information using Bulk API
 
 A HTTP GET request retrieves a bulk response for all actions, corresponding to a particular bundle, that satisfy user specified criteria.

http://git-wip-us.apache.org/repos/asf/oozie/blob/c6afe1c1/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index c4a2984..99b3f7f 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.2.0 release (trunk - unreleased)
 
+OOZIE-2118 add createdtime option to workflow jobs query (ryota)
 OOZIE-2110 cancel delegation token of launcher jobs that stay till child jobs finish (ryota)
 OOZIE-2119 Distcp action fails when -D option in arguments (ryota)
 OOZIE-2112 Child Job URL doesn't show properly with Hive on Tez (ryota)