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 2014/05/14 19:58:52 UTC

[1/2] OOZIE-1791 add IGNORED status to Coordinator Job and Action (ryota)

Repository: oozie
Updated Branches:
  refs/heads/master a0aa6fbdf -> f82c12408


http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionsIgnoreXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionsIgnoreXCommand.java b/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionsIgnoreXCommand.java
new file mode 100644
index 0000000..9daa6d7
--- /dev/null
+++ b/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionsIgnoreXCommand.java
@@ -0,0 +1,169 @@
+/**
+ * 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.command.coord;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.oozie.CoordinatorActionBean;
+import org.apache.oozie.CoordinatorActionInfo;
+import org.apache.oozie.CoordinatorJobBean;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.WorkflowJobBean;
+import org.apache.oozie.client.CoordinatorAction;
+import org.apache.oozie.client.CoordinatorJob;
+import org.apache.oozie.client.CoordinatorJob.Execution;
+import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.command.CommandException;
+import org.apache.oozie.coord.CoordELFunctions;
+import org.apache.oozie.executor.jpa.CoordActionGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordActionQueryExecutor;
+import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobInsertJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
+import org.apache.oozie.executor.jpa.JPAExecutorException;
+import org.apache.oozie.executor.jpa.WorkflowJobInsertJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
+import org.apache.oozie.local.LocalOozie;
+import org.apache.oozie.service.JPAService;
+import org.apache.oozie.service.LiteWorkflowStoreService;
+import org.apache.oozie.service.SchemaService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.service.StatusTransitService;
+import org.apache.oozie.service.StoreService;
+import org.apache.oozie.service.WorkflowStoreService;
+import org.apache.oozie.store.CoordinatorStore;
+import org.apache.oozie.store.StoreException;
+import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.util.DateUtils;
+import org.apache.oozie.util.IOUtils;
+import org.apache.oozie.util.XConfiguration;
+import org.apache.oozie.util.XLog;
+import org.apache.oozie.util.XmlUtils;
+import org.apache.oozie.workflow.WorkflowApp;
+import org.apache.oozie.workflow.WorkflowInstance;
+import org.apache.oozie.workflow.WorkflowLib;
+import org.apache.oozie.workflow.lite.EndNodeDef;
+import org.apache.oozie.workflow.lite.LiteWorkflowApp;
+import org.apache.oozie.workflow.lite.LiteWorkflowInstance;
+import org.apache.oozie.workflow.lite.StartNodeDef;
+import org.jdom.Element;
+import org.jdom.JDOMException;
+
+public class TestCoordActionsIgnoreXCommand extends XDataTestCase {
+    private Services services;
+    List<CoordinatorJobBean> coordJobs;
+    List<CoordinatorActionBean> coordActions;
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+        services = new Services();
+        services.init();
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        LocalOozie.stop();
+        services.destroy();
+        super.tearDown();
+    }
+
+    public void testCoordActionsIgnore() throws Exception {
+        createDBRecords();
+
+        // positive test of single action - oozie job -ingore job1 -action 1
+        CoordinatorActionInfo retInfo = new CoordActionsIgnoreXCommand(coordJobs.get(0).getId(), "action", "1").call();
+        CoordinatorActionBean actionBean1 = CoordActionQueryExecutor.getInstance().get(
+                CoordActionQuery.GET_COORD_ACTION, coordActions.get(0).getId());
+        assertEquals(CoordinatorAction.Status.IGNORED, actionBean1.getStatus());
+        assertEquals(1, retInfo.getCoordActions().size());
+        assertEquals(actionBean1.getId(), retInfo.getCoordActions().get(0).getId());
+
+        // positive test of action range - oozie job -ignore job1 -action 2-3
+        retInfo = new CoordActionsIgnoreXCommand(coordJobs.get(0).getId(), "action", "2-3").call();
+        CoordinatorActionBean actionBean2 = CoordActionQueryExecutor.getInstance().get(
+                CoordActionQuery.GET_COORD_ACTION, coordActions.get(1).getId());
+        CoordinatorActionBean actionBean3 = CoordActionQueryExecutor.getInstance().get(
+                CoordActionQuery.GET_COORD_ACTION, coordActions.get(2).getId());
+        assertEquals(CoordinatorAction.Status.IGNORED, actionBean2.getStatus());
+        assertEquals(CoordinatorAction.Status.IGNORED, actionBean3.getStatus());
+        assertEquals(2, retInfo.getCoordActions().size());
+        String retId1 = retInfo.getCoordActions().get(0).getId();
+        String retId2 = retInfo.getCoordActions().get(1).getId();
+        assertTrue(actionBean2.getId().equals(retId1) || actionBean2.getId().equals(retId2));
+        assertTrue(actionBean3.getId().equals(retId1) || actionBean3.getId().equals(retId2));
+
+        // negative test when ignoring a coord action in RUNNING (@5 is running)
+        try {
+            new CoordActionsIgnoreXCommand(coordJobs.get(0).getId(), "action", "4-5").call();
+        }
+        catch (CommandException ex) {
+            assertEquals(ex.getErrorCode(), ErrorCode.E1024);
+            assertTrue(ex.getMessage().indexOf(
+                    "part or all actions are not eligible to ignore, check state of action number(s) [5]") > -1);
+        }
+
+        // negative test when ignore command on coordinator job in PREP
+        try {
+            new CoordActionsIgnoreXCommand(coordJobs.get(1).getId(), "action", "1").call();
+        }
+        catch (CommandException ex) {
+            assertEquals(ex.getErrorCode(), ErrorCode.E1024);
+            assertTrue(ex.getMessage().indexOf("No actions are materialized to ignore") > -1);
+        }
+    }
+
+    private void createDBRecords() throws Exception {
+        JPAService jpaService = services.get(JPAService.class);
+        coordJobs = new ArrayList<CoordinatorJobBean>();
+        coordActions = new ArrayList<CoordinatorActionBean>();
+
+        Date startTime = DateUtils.parseDateOozieTZ("2013-08-01T23:59Z");
+        Date endTime = DateUtils.parseDateOozieTZ("2013-08-02T23:59Z");
+        CoordinatorJobBean job1 = addRecordToCoordJobTable(CoordinatorJob.Status.KILLED, startTime, endTime, false,
+                true, 0);
+        CoordinatorJobBean job2 = addRecordToCoordJobTable(CoordinatorJob.Status.PREP, startTime, endTime, false, true,
+                0);
+        coordJobs.add(job1);
+        coordJobs.add(job2);
+
+        CoordinatorActionBean action1_1 = addRecordToCoordActionTable(job1.getId(), 1, CoordinatorAction.Status.FAILED,
+                "coord-action-get.xml", 0);
+        CoordinatorActionBean action1_2 = addRecordToCoordActionTable(job1.getId(), 2,
+                CoordinatorAction.Status.FAILED, "coord-action-get.xml", 0);
+        CoordinatorActionBean action1_3 = addRecordToCoordActionTable(job1.getId(), 3, CoordinatorAction.Status.KILLED,
+                "coord-action-get.xml", 0);
+        CoordinatorActionBean action1_4 = addRecordToCoordActionTable(job1.getId(), 4, CoordinatorAction.Status.KILLED,
+                "coord-action-get.xml", 0);
+        CoordinatorActionBean action1_5 = addRecordToCoordActionTable(job1.getId(), 5,
+                CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+        coordActions.add(action1_1);
+        coordActions.add(action1_2);
+        coordActions.add(action1_3);
+        coordActions.add(action1_4);
+        coordActions.add(action1_5);
+
+        action1_1.setNominalTime(DateUtils.parseDateOozieTZ("2009-12-15T02:00Z"));
+        action1_1.setExternalId(null);
+        CoordActionQueryExecutor.getInstance().executeUpdate(CoordActionQuery.UPDATE_COORD_ACTION, action1_1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java b/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
index 327ec90..bc24235 100644
--- a/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
@@ -660,6 +660,50 @@ public class TestCoordChangeXCommand extends XDataTestCase {
         }
     }
 
+    public void testCoordStatus_Ignored() throws Exception {
+        Date start = new Date();
+        Date end = new Date(start.getTime() + (5 * 60 * 60 * 1000)); // 5 hrs
+        String statusToRUNNING = "status=RUNNING";
+        String statusToIGNORED = "status=IGNORED";
+        final CoordinatorJobBean job1 = addRecordToCoordJobTableForPauseTimeTest(CoordinatorJob.Status.IGNORED, start,
+                end, end, false, false, 4);
+        final CoordinatorJobBean job2 = addRecordToCoordJobTableForPauseTimeTest(CoordinatorJob.Status.KILLED, start,
+                end, end, false, false, 4);
+        final CoordinatorJobBean job3 = addRecordToCoordJobTableForPauseTimeTest(CoordinatorJob.Status.RUNNING, start,
+                end, end, false, false, 4);
+        final CoordinatorJobBean job4 = addRecordToCoordJobTableForPauseTimeTest(CoordinatorJob.Status.FAILED, start,
+                end, end, true, false, 4);
+
+        // Status change from IGNORED to RUNNING
+        new CoordChangeXCommand(job1.getId(), statusToRUNNING).call();
+        CoordinatorJobBean coordJob = CoordJobQueryExecutor.getInstance()
+                .get(CoordJobQuery.GET_COORD_JOB, job1.getId());
+        assertEquals(coordJob.getStatus(), Job.Status.RUNNING);
+
+        // Status change from KILLED -> IGNORED
+        new CoordChangeXCommand(job2.getId(), statusToIGNORED).call();
+        coordJob = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB, job2.getId());
+        assertEquals(coordJob.getStatus(), Job.Status.IGNORED);
+
+        // Status change from RUNNING -> IGNORED
+        try {
+            new CoordChangeXCommand(job3.getId(), statusToIGNORED).call();
+        }
+        catch (CommandException ex) {
+            assertEquals(ErrorCode.E1015, ex.getErrorCode());
+            assertTrue(ex.getMessage().indexOf(
+                    "Only FAILED or KILLED non-pending job can be changed to IGNORED") > -1);
+        }
+        // Status change from FAILED -> IGNORED when coord job is pending
+        try {
+            new CoordChangeXCommand(job4.getId(), statusToIGNORED).call();
+        }
+        catch (CommandException ex) {
+            assertEquals(ErrorCode.E1015, ex.getErrorCode());
+            assertTrue(ex.getMessage().indexOf(
+                    "Only FAILED or KILLED non-pending job can be changed to IGNORED") > -1);
+        }
+    }
     // Status change from failed- successful
     public void testCoordStatus_Failed() throws Exception {
         Date start = new Date();

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleActionQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleActionQueryExecutor.java b/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleActionQueryExecutor.java
index 3ae4d6c..3fd6537 100644
--- a/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleActionQueryExecutor.java
+++ b/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleActionQueryExecutor.java
@@ -105,7 +105,7 @@ public class TestBundleActionQueryExecutor extends XDataTestCase {
         assertTrue(date.before(Calendar.getInstance().getTime()));
 
         query = BundleActionQueryExecutor.getInstance().getSelectQuery(
-                BundleActionQuery.GET_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE, em, bean.getBundleId());
+                BundleActionQuery.GET_BUNDLE_UNIGNORED_ACTION_STATUS_PENDING_FOR_BUNDLE, em, bean.getBundleId());
         assertEquals(query.getParameterValue("bundleId"), bean.getBundleId());
     }
 
@@ -123,9 +123,9 @@ public class TestBundleActionQueryExecutor extends XDataTestCase {
     public void testGet() throws Exception {
         BundleJobBean job = this.addRecordToBundleJobTable(Job.Status.RUNNING, false);
         BundleActionBean bundleAction = this.addRecordToBundleActionTable(job.getId(), "action1", 1, Job.Status.PREP);
-        // GET_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE
+        // GET_UNIGNORED_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE
         BundleActionBean retBean = BundleActionQueryExecutor.getInstance().get(
-                BundleActionQuery.GET_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE, bundleAction.getBundleId());
+                BundleActionQuery.GET_BUNDLE_UNIGNORED_ACTION_STATUS_PENDING_FOR_BUNDLE, bundleAction.getBundleId());
         assertEquals(bundleAction.getCoordId(), retBean.getCoordId());
         assertEquals(bundleAction.getStatusStr(), retBean.getStatusStr());
         assertEquals(bundleAction.getPending(), retBean.getPending());
@@ -156,9 +156,9 @@ public class TestBundleActionQueryExecutor extends XDataTestCase {
         bActions = BundleActionQueryExecutor.getInstance().getList(
                 BundleActionQuery.GET_BUNDLE_WAITING_ACTIONS_OLDER_THAN, (long) (-1000 * 60));
         assertEquals(2, bActions.size());
-        // GET_BUNDLE_ACTIONS_FOR_BUNDLE
+        // GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE
         List<BundleActionBean> retList = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
         assertEquals(3, retList.size());
         for (BundleActionBean bean : retList) {
             assertTrue(bean.getCoordName().equals("coord1") || bean.getCoordName().equals("coord2")

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java b/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java
index 9892d4b..80ab512 100644
--- a/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java
+++ b/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java
@@ -156,7 +156,7 @@ public class MockCoordinatorEngineService extends CoordinatorEngineService {
         public void change(String jobId, String changeValue) throws CoordinatorEngineException {
             did = RestConstants.JOB_ACTION_CHANGE;
             int idx = validateCoordinatorIdx(jobId);
-            started.set(idx, false);
+            started.set(idx, true);
         }
 
         @Override
@@ -165,6 +165,13 @@ public class MockCoordinatorEngineService extends CoordinatorEngineService {
         }
 
         @Override
+        public CoordinatorActionInfo ignore(String jobId, String type, String scope) throws CoordinatorEngineException {
+            did = RestConstants.JOB_ACTION_IGNORE;
+            int idx = validateCoordinatorIdx(jobId);
+            started.set(idx, true);
+            return null;
+        }
+        @Override
         public CoordinatorActionInfo reRun(String jobId, String rerunType, String scope, boolean refresh,
                 boolean noCleanup) throws BaseEngineException {
             did = RestConstants.JOB_COORD_ACTION_RERUN;

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/servlet/TestV2JobServlet.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/servlet/TestV2JobServlet.java b/core/src/test/java/org/apache/oozie/servlet/TestV2JobServlet.java
index aa5442c..7512e24 100644
--- a/core/src/test/java/org/apache/oozie/servlet/TestV2JobServlet.java
+++ b/core/src/test/java/org/apache/oozie/servlet/TestV2JobServlet.java
@@ -17,6 +17,7 @@
  */
 package org.apache.oozie.servlet;
 
+import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.rest.RestConstants;
 import org.apache.oozie.client.rest.JsonTags;
 import org.json.simple.JSONObject;
@@ -147,4 +148,78 @@ public class TestV2JobServlet extends DagServletTestCase {
             }
         });
     }
+
+    public void testCoordJobIgnore() throws Exception {
+        runTest("/v2/job/*", V2JobServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+
+                MockDagEngineService.reset();
+                Map<String, String> params = new HashMap<String, String>();
+                params.put(RestConstants.ACTION_PARAM, RestConstants.JOB_ACTION_IGNORE);
+
+                // url - oozie/v2/coord_job_id?action=ignore
+                URL url = createURL(MockCoordinatorEngineService.JOB_ID + 1, params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("PUT");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                assertEquals(RestConstants.JOB_ACTION_CHANGE, MockCoordinatorEngineService.did);
+
+                MockCoordinatorEngineService.reset();
+                params = new HashMap<String, String>();
+                params.put(RestConstants.ACTION_PARAM, RestConstants.JOB_ACTION_IGNORE);
+                url = createURL(MockCoordinatorEngineService.JOB_ID
+                        + (MockCoordinatorEngineService.coordJobs.size() + 1), params);
+                conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("PUT");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                assertEquals(RestConstants.JOB_ACTION_CHANGE, MockCoordinatorEngineService.did);
+
+                return null;
+            }
+        });
+    }
+    public void testCoordActionIgnore() throws Exception {
+        runTest("/v2/job/*", V2JobServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+
+                MockDagEngineService.reset();
+                Map<String, String> params = new HashMap<String, String>();
+                params.put(RestConstants.ACTION_PARAM, RestConstants.JOB_ACTION_IGNORE);
+                params.put(RestConstants.JOB_COORD_RANGE_TYPE_PARAM, RestConstants.JOB_COORD_SCOPE_ACTION);
+                params.put(RestConstants.JOB_COORD_SCOPE_PARAM, "1");
+
+                // url - oozie/v2/coord_job_id?action=ignore
+                URL url = createURL(MockCoordinatorEngineService.JOB_ID + 1, params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("PUT");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                assertEquals(RestConstants.JOB_ACTION_IGNORE, MockCoordinatorEngineService.did);
+
+                // negative test for non-existent action
+                MockCoordinatorEngineService.reset();
+                params = new HashMap<String, String>();
+                params.put(RestConstants.ACTION_PARAM, RestConstants.JOB_ACTION_IGNORE);
+                params.put(RestConstants.JOB_COORD_RANGE_TYPE_PARAM, RestConstants.JOB_COORD_SCOPE_ACTION);
+                params.put(RestConstants.JOB_COORD_SCOPE_PARAM, "1");
+                url = createURL(MockCoordinatorEngineService.JOB_ID
+                        + (MockCoordinatorEngineService.coordJobs.size() + 1), params);
+                conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("PUT");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                assertEquals(RestConstants.JOB_ACTION_IGNORE, MockCoordinatorEngineService.did);
+
+                return null;
+            }
+        });
+    }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/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 f3231c7..cd0391c 100644
--- a/docs/src/site/twiki/DG_CommandLineTool.twiki
+++ b/docs/src/site/twiki/DG_CommandLineTool.twiki
@@ -69,6 +69,8 @@ usage:
                 -update               Update coordinator definition and properties
                 -logfilter            job log search parameter. Can be specified as -logfilter opt1=val1;opt2=val1;opt3=val1.
                                       Supported options are recent, start, end, loglevel, text, limit and debug.
+                -ignore <arg>         ignore a coordinator job or action
+                                      (requires '-action' to ignore a coordinator action, if no option given, ignore a coodinator job)
 .
       oozie jobs <OPTIONS> : jobs status
                  -auth <arg>          select authentication type [SIMPLE|KERBEROS]
@@ -352,10 +354,11 @@ Conditions and usage:
    * Repeated value names are not allowed.
    * New end time should not be before job's start time and last action time.
    * If end time is before job start time and if the job has not materialized any actions, then job status is changed to SUCCEEDED.
-   * Currently status only takes RUNNING and can be used to change the status of FAILED or KILLED coordinator job to RUNNING and resuming materialization. This status change command does not affect the status of already materialized actions in the coordinator. If there are FAILED or KILLED coordinator actions they have to be rerun separately.
+   * Currently status only takes RUNNING and can be used to change the status of FAILED, KILLED, IGNORED coordinator job to RUNNING and resuming materialization. This status change command does not affect the status of already materialized actions in the coordinator. If there are FAILED, KILLED or IGNORED coordinator actions they have to be rerun separately.
    * New concurrency value has to be a valid integer.
    * All lookahead actions which are in WAITING/READY state will be revoked according to the new pause/end time. If any action after new pause/end time is not in WAITING/READY state, an exception will be thrown.
    * Also empty string "" can be used to reset pause time to none.
+   * Endtime/concurency/pausetime of IGNORED Job cannot be changed.
 
 After the command is executed the job's end time, concurrency or pause time should be changed. If an already-succeeded job changes its end time, its status will become running.
 
@@ -409,7 +412,7 @@ $oozie job -rerun <coord_Job_id> [-nocleanup] [-refresh]
 (if neither -action nor -date is given, the exception will be thrown.)
 </verbatim>
 
-The =rerun= option reruns a terminated (=TIMEDOUT=, =SUCCEEDED=, =KILLED=, =FAILED=) coordiantor action when coordiator job
+The =rerun= option reruns a terminated (=TIMEDOUT=, =SUCCEEDED=, =KILLED=, =FAILED=, =IGNORED=) coordiantor action when coordiator job
 is not in =FAILED= or =KILLED= state.
 
 After the command is executed the rerun coordiator action will be in =WAITING= status.
@@ -755,6 +758,36 @@ $ oozie job -oozie http://localhost:11000/oozie -config job.properties -update 0
 **********************************
 </verbatim>
 
+---+++ Ignore a Coordinator Job
+
+Example:
+
+<verbatim>
+$oozie job -ignore <coord_Job_id>
+</verbatim>
+
+The =ignore= option changes a coordinator job in =KILLED=, =FAILED= to =IGNORED= state.
+When a coordinator job in a bundle is in =IGNORED= state, the coordinator job doesn't impact the state of the bundle job.
+For example, when a coordinator job in a bundle failed and afterwards ignored, the bundle job becomes =SUCCEEDED= instead of =DONEWITHERROR= as long as other coordinator jobs in the bundle succeeded.
+ A ignored coordinator job can be changed to =RUNNING= using -change command.
+ Refer to the [[DG_CommandLineTool#Changing_endtimeconcurrencypausetimestatus_of_a_Coordinator_Job][Coordinator job change command]] for details.
+
+---+++ Ignore a Coordinator Action or Multiple Coordinator Actions
+
+Example:
+
+<verbatim>
+$oozie job -ignore <coord_Job_id> -action 1,3-4,7-40
+</verbatim>
+The =ignore= option changes a coordinator action(s) in terminal state (=KILLED=, =FAILED=, =TIMEDOUT=) to =IGNORED= state, while not changing the state of the coordinator job.
+When a coordinator action is in =IGNORED= state, the action doesn't impact the state of a coordinator job.
+For example, when a coordinator action failed and afterwards ignored, a coordinator job becomes =SUCCEEDED= instead of =DONEWITHERROR= as long
+ as other coordinator actions succeeded.
+
+A ignored coordinator action can be rerun using -rerun command.
+Refer to the [[DG_CoordinatorRerun][Rerunning Coordinator Actions]] for details.
+When a workflow job of a ignored coordinator action is rerun, the coordinator action becomes =RUNNING= state.
+
 ---++ Jobs Operations
 
 ---+++ Checking the Status of multiple Workflow Jobs

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/docs/src/site/twiki/WebServicesAPI.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/WebServicesAPI.twiki b/docs/src/site/twiki/WebServicesAPI.twiki
index 6730255..5769924 100644
--- a/docs/src/site/twiki/WebServicesAPI.twiki
+++ b/docs/src/site/twiki/WebServicesAPI.twiki
@@ -1682,6 +1682,26 @@ Content-Type: application/json;charset=UTF-8
 }
 </verbatim>
 
+---++++ Managing a Job
+---+++++ Ignore a Coordinator Job or Action
+
+A ignore request is done with an HTTP PUT request with a =ignore=
+
+The =type= parameter supports =action= only.
+The =scope= parameter can contain coodinator action id(s) to be ignored.
+Multiple action ids can be passed to the =scope= parameter
+
+*Request:*
+
+Ignore a coordinator job
+<verbatim>
+PUT /oozie/v2/job/job-3?action=ignore
+</verbatim>
+
+Ignore coordinator actions
+<verbatim>
+PUT /oozie/v2/job/job-3?action=ignore&type=action&scope=3-4
+</verbatim>
 </noautolink>
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index f5d0154..da86677 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1791 add IGNORED status to Coordinator Job and Action (ryota)
 OOZIE-1825 Optimize wf_jobs protoconf storage (puru via rohini)
 OOZIE-1831 Oozie upgrade fails if workflow jobs are in running or suspended state (satish.mittal via rohini)
 OOZIE-1690 TestShellActionExecutor#testEnvVar failed for Windows (omaliuvanchuk via rkanter)


[2/2] git commit: OOZIE-1791 add IGNORED status to Coordinator Job and Action (ryota)

Posted by ry...@apache.org.
OOZIE-1791 add IGNORED status to Coordinator Job and Action (ryota)


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

Branch: refs/heads/master
Commit: f82c1240846b52b7c27a26c934fb174edebe5446
Parents: a0aa6fb
Author: egashira <ry...@yahoo.com>
Authored: Wed May 14 10:53:47 2014 -0700
Committer: egashira <ry...@yahoo.com>
Committed: Wed May 14 10:53:47 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/oozie/cli/OozieCLI.java     |  26 +++
 .../apache/oozie/client/CoordinatorAction.java  |   4 +-
 .../main/java/org/apache/oozie/client/Job.java  |   3 +-
 .../org/apache/oozie/client/OozieClient.java    |  38 ++++-
 .../apache/oozie/client/rest/RestConstants.java |   2 +
 .../oozie/client/rest/TestJsonToBean.java       |   4 +-
 .../java/org/apache/oozie/BundleActionBean.java |   4 +-
 .../org/apache/oozie/CoordinatorActionBean.java |   6 +-
 .../org/apache/oozie/CoordinatorEngine.java     |  14 +-
 .../org/apache/oozie/CoordinatorJobBean.java    |   1 +
 .../main/java/org/apache/oozie/ErrorCode.java   |   2 +-
 .../oozie/command/IgnoreTransitionXCommand.java |  54 ++++++
 .../command/bundle/BundleJobChangeXCommand.java |   2 +-
 .../command/bundle/BundleJobResumeXCommand.java |   2 +-
 .../bundle/BundleJobSuspendXCommand.java        |   2 +-
 .../oozie/command/bundle/BundleJobXCommand.java |   1 -
 .../command/bundle/BundleKillXCommand.java      |   2 +-
 .../command/bundle/BundleRerunXCommand.java     |   2 +-
 .../coord/CoordActionsIgnoreXCommand.java       | 138 +++++++++++++++
 .../command/coord/CoordChangeXCommand.java      |  35 ++--
 .../oozie/command/coord/CoordKillXCommand.java  |   7 +-
 .../oozie/command/coord/CoordRerunXCommand.java |  14 +-
 .../command/coord/CoordSuspendXCommand.java     |   4 +-
 .../java/org/apache/oozie/coord/CoordUtils.java |   2 +-
 .../executor/jpa/BundleActionQueryExecutor.java |  12 +-
 .../CoordJobGetActionsStatusJPAExecutor.java    |   2 +-
 .../oozie/service/StatusTransitService.java     |   7 +-
 .../apache/oozie/servlet/BaseJobServlet.java    |  14 ++
 .../org/apache/oozie/servlet/V0JobServlet.java  |   5 +
 .../org/apache/oozie/servlet/V1JobServlet.java  |   4 +
 .../org/apache/oozie/servlet/V2JobServlet.java  |  69 ++++++++
 .../org/apache/oozie/TestCoordinatorEngine.java |   8 +-
 .../oozie/action/hadoop/TestOozieJobInfo.java   |   6 +-
 .../org/apache/oozie/client/TestOozieCLI.java   |  58 ++++++-
 .../bundle/TestBundleJobSuspendXCommand.java    |   4 +-
 .../command/bundle/TestBundleKillXCommand.java  |   8 +-
 .../command/bundle/TestBundleStartXCommand.java |  12 +-
 .../coord/TestCoordActionsIgnoreXCommand.java   | 169 +++++++++++++++++++
 .../command/coord/TestCoordChangeXCommand.java  |  44 +++++
 .../jpa/TestBundleActionQueryExecutor.java      |  10 +-
 .../servlet/MockCoordinatorEngineService.java   |   9 +-
 .../apache/oozie/servlet/TestV2JobServlet.java  |  75 ++++++++
 docs/src/site/twiki/DG_CommandLineTool.twiki    |  37 +++-
 docs/src/site/twiki/WebServicesAPI.twiki        |  20 +++
 release-log.txt                                 |   1 +
 45 files changed, 866 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/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 5260ab7..33935d3 100644
--- a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
+++ b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -117,6 +117,7 @@ public class OozieCLI {
     public static final String SQOOP_COMMAND_OPTION = "command";
     public static final String SHOWDIFF_OPTION = "diff";
     public static final String UPDATE_OPTION = "update";
+    public static final String IGNORE_OPTION = "ignore";
 
     public static final String DO_AS_OPTION = "doas";
 
@@ -322,6 +323,9 @@ public class OozieCLI {
                 "set/override value for given property").create("D");
         Option getAllWorkflows = new Option(ALL_WORKFLOWS_FOR_COORD_ACTION, false,
                 "Get workflow jobs corresponding to a coordinator action including all the reruns");
+        Option ignore = new Option(IGNORE_OPTION, true,
+                "change status of a coordinator job or action to IGNORED"
+                + " (-action required to ignore coord actions)");
 
         Option doAs = new Option(DO_AS_OPTION, true, "doAs user, impersonates as the specified user");
 
@@ -340,6 +344,7 @@ public class OozieCLI {
         actions.addOption(log);
         actions.addOption(definition);
         actions.addOption(config_content);
+        actions.addOption(ignore);
         actions.setRequired(true);
         Options jobOptions = new Options();
         jobOptions.addOption(oozie);
@@ -895,6 +900,16 @@ public class OozieCLI {
             else if (options.contains(RESUME_OPTION)) {
                 wc.resume(commandLine.getOptionValue(RESUME_OPTION));
             }
+            else if (options.contains(IGNORE_OPTION)) {
+                String ignoreScope = null;
+                if (options.contains(ACTION_OPTION)) {
+                    ignoreScope = commandLine.getOptionValue(ACTION_OPTION);
+                    if (ignoreScope == null || ignoreScope.isEmpty()) {
+                        throw new OozieCLIException("-" + ACTION_OPTION + " is empty");
+                    }
+                }
+                printCoordActionsStatus(wc.ignore(commandLine.getOptionValue(IGNORE_OPTION), ignoreScope));
+            }
             else if (options.contains(KILL_OPTION)) {
                 if (commandLine.getOptionValue(KILL_OPTION).contains("-C")
                         && (options.contains(DATE_OPTION) || options.contains(ACTION_OPTION))) {
@@ -1260,6 +1275,17 @@ public class OozieCLI {
         }
     }
 
+    private void printCoordActionsStatus(List<CoordinatorAction> actions) {
+        if (actions != null && actions.size() > 0) {
+            System.out.println("Action ID" + VERBOSE_DELIMITER + "Nominal Time" + VERBOSE_DELIMITER + "Status");
+            System.out.println(RULER);
+            for (CoordinatorAction action : actions) {
+                System.out.println(maskIfNull(action.getId()) + VERBOSE_DELIMITER
+                        + maskDate(action.getNominalTime(), null, false) + VERBOSE_DELIMITER
+                        + maskIfNull(action.getStatus().name()));
+            }
+        }
+    }
 
     @VisibleForTesting
     void printWorkflowAction(WorkflowAction action, String timeZoneId, boolean verbose) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/client/src/main/java/org/apache/oozie/client/CoordinatorAction.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/CoordinatorAction.java b/client/src/main/java/org/apache/oozie/client/CoordinatorAction.java
index a0e42cd..ba17c5b 100644
--- a/client/src/main/java/org/apache/oozie/client/CoordinatorAction.java
+++ b/client/src/main/java/org/apache/oozie/client/CoordinatorAction.java
@@ -25,7 +25,7 @@ import java.util.Date;
 
 public interface CoordinatorAction {
     /**
-     * Defines the possible stati of an application instance.
+     * Defines the possible status of an application instance.
      */
     public static enum Status {
         WAITING,
@@ -37,7 +37,7 @@ public interface CoordinatorAction {
         SUCCEEDED,
         KILLED,
         FAILED,
-        DISCARDED
+        IGNORED
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/client/src/main/java/org/apache/oozie/client/Job.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/Job.java b/client/src/main/java/org/apache/oozie/client/Job.java
index 8533165..c2e8562 100644
--- a/client/src/main/java/org/apache/oozie/client/Job.java
+++ b/client/src/main/java/org/apache/oozie/client/Job.java
@@ -27,7 +27,8 @@ public interface Job {
      * Defines the possible status of an Oozie JOB.
      */
     public static enum Status {
-        PREMATER, PREP, RUNNING, SUSPENDED, SUCCEEDED, KILLED, FAILED, PAUSED, PREPPAUSED, PREPSUSPENDED, RUNNINGWITHERROR, SUSPENDEDWITHERROR, PAUSEDWITHERROR, DONEWITHERROR
+        PREMATER, PREP, RUNNING, SUSPENDED, SUCCEEDED, KILLED, FAILED, PAUSED,PREPPAUSED,
+        PREPSUSPENDED, RUNNINGWITHERROR, SUSPENDEDWITHERROR, PAUSEDWITHERROR, DONEWITHERROR, IGNORED
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/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 211b5ad..b468186 100644
--- a/client/src/main/java/org/apache/oozie/client/OozieClient.java
+++ b/client/src/main/java/org/apache/oozie/client/OozieClient.java
@@ -788,6 +788,17 @@ public class OozieClient {
         new JobAction(jobId, RestConstants.JOB_ACTION_CHANGE, changeValue).call();
     }
 
+    /**
+     * Ignore a coordinator job.
+     *
+     * @param jobId coord job Id.
+     * @param scope list of coord actions to be ignored
+     * @throws OozieClientException thrown if the job could not be changed.
+     */
+    public List<CoordinatorAction> ignore(String jobId, String scope) throws OozieClientException {
+        return new CoordIgnore(jobId, RestConstants.JOB_COORD_SCOPE_ACTION, scope).call();
+    }
+
     private class JobInfo extends ClientCallable<WorkflowJob> {
 
         JobInfo(String jobId, int start, int len) {
@@ -1259,7 +1270,6 @@ public class OozieClient {
         }
 
         @Override
-        @SuppressWarnings("unchecked")
         protected List<WorkflowJob> call(HttpURLConnection conn) throws IOException, OozieClientException {
             conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
             if ((conn.getResponseCode() == HttpURLConnection.HTTP_OK)) {
@@ -1382,6 +1392,30 @@ public class OozieClient {
         }
     }
 
+    private class CoordIgnore extends ClientCallable<List<CoordinatorAction>> {
+        CoordIgnore(String jobId, String rerunType, String scope) {
+            super("PUT", RestConstants.JOB, notEmpty(jobId, "jobId"), prepareParams(RestConstants.ACTION_PARAM,
+                    RestConstants.JOB_ACTION_IGNORE, RestConstants.JOB_COORD_RANGE_TYPE_PARAM,
+                    rerunType, RestConstants.JOB_COORD_SCOPE_PARAM, scope));
+        }
+
+        @Override
+        protected List<CoordinatorAction> call(HttpURLConnection conn) throws IOException, OozieClientException {
+            conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+            if ((conn.getResponseCode() == HttpURLConnection.HTTP_OK)) {
+                Reader reader = new InputStreamReader(conn.getInputStream());
+                JSONObject json = (JSONObject) JSONValue.parse(reader);
+                if(json != null) {
+                    JSONArray coordActions = (JSONArray) json.get(JsonTags.COORDINATOR_ACTIONS);
+                    return JsonToBean.createCoordinatorActionList(coordActions);
+                }
+            }
+            else {
+                handleError(conn);
+            }
+            return null;
+        }
+    }
     private class CoordRerun extends ClientCallable<List<CoordinatorAction>> {
 
         CoordRerun(String jobId, String rerunType, String scope, boolean refresh, boolean noCleanup) {
@@ -1507,7 +1541,6 @@ public class OozieClient {
         }
 
         @Override
-        @SuppressWarnings("unchecked")
         protected Void call(HttpURLConnection conn) throws IOException, OozieClientException {
             conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
             if ((conn.getResponseCode() == HttpURLConnection.HTTP_OK)) {
@@ -1532,7 +1565,6 @@ public class OozieClient {
         }
 
         @Override
-        @SuppressWarnings("unchecked")
         protected String call(HttpURLConnection conn) throws IOException, OozieClientException {
             if ((conn.getResponseCode() == HttpURLConnection.HTTP_OK)) {
                 Reader reader = new InputStreamReader(conn.getInputStream());

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java b/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
index 808f9b2..97fd465 100644
--- a/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
+++ b/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
@@ -70,6 +70,8 @@ public interface RestConstants {
 
     public static final String JOB_ACTION_RERUN = "rerun";
 
+    public static final String JOB_ACTION_IGNORE = "ignore";
+
     public static final String JOB_COORD_ACTION_RERUN = "coord-rerun";
 
     public static final String JOB_COORD_UPDATE = "update";

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/client/src/test/java/org/apache/oozie/client/rest/TestJsonToBean.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/oozie/client/rest/TestJsonToBean.java b/client/src/test/java/org/apache/oozie/client/rest/TestJsonToBean.java
index 4091808..9f04db7 100644
--- a/client/src/test/java/org/apache/oozie/client/rest/TestJsonToBean.java
+++ b/client/src/test/java/org/apache/oozie/client/rest/TestJsonToBean.java
@@ -184,7 +184,7 @@ public class TestJsonToBean extends TestCase {
         json.put(JsonTags.COORDINATOR_ACTION_CREATED_TIME, CREATED_TIME);
         json.put(JsonTags.COORDINATOR_ACTION_NOMINAL_TIME, NOMINAL_TIME);
         json.put(JsonTags.COORDINATOR_ACTION_EXTERNALID, "d");
-        json.put(JsonTags.COORDINATOR_ACTION_STATUS, CoordinatorAction.Status.DISCARDED.toString());
+        json.put(JsonTags.COORDINATOR_ACTION_STATUS, CoordinatorAction.Status.IGNORED.toString());
         json.put(JsonTags.COORDINATOR_ACTION_RUNTIME_CONF, "e");
         json.put(JsonTags.COORDINATOR_ACTION_LAST_MODIFIED_TIME, LAST_MODIFIED);
         json.put(JsonTags.COORDINATOR_ACTION_MISSING_DEPS, "f");
@@ -221,7 +221,7 @@ public class TestJsonToBean extends TestCase {
         assertEquals(JsonUtils.parseDateRfc822(CREATED_TIME), action.getCreatedTime());
         assertEquals(JsonUtils.parseDateRfc822(NOMINAL_TIME), action.getNominalTime());
         assertEquals("d", action.getExternalId());
-        assertEquals(CoordinatorAction.Status.DISCARDED, action.getStatus());
+        assertEquals(CoordinatorAction.Status.IGNORED, action.getStatus());
         assertEquals("e", action.getRunConf());
         assertEquals(JsonUtils.parseDateRfc822(LAST_MODIFIED), action.getLastModifiedTime());
         assertEquals("f", action.getMissingDependencies());

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/BundleActionBean.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/BundleActionBean.java b/core/src/main/java/org/apache/oozie/BundleActionBean.java
index 607f55b..5d85a4d 100644
--- a/core/src/main/java/org/apache/oozie/BundleActionBean.java
+++ b/core/src/main/java/org/apache/oozie/BundleActionBean.java
@@ -50,9 +50,9 @@ import org.json.simple.JSONObject;
 
         @NamedQuery(name = "UPDATE_BUNDLE_ACTION_STATUS_PENDING_MODTIME_COORDID", query = "update BundleActionBean w set w.statusStr = :status, w.lastModifiedTimestamp = :lastModifiedTime, w.pending = :pending, w.coordId = :coordId where w.bundleActionId = :bundleActionId"),
 
-        @NamedQuery(name = "GET_BUNDLE_ACTIONS_FOR_BUNDLE", query = "select OBJECT(w) from BundleActionBean w where w.bundleId = :bundleId"),
+        @NamedQuery(name = "GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE", query = "select OBJECT(w) from BundleActionBean w where w.bundleId = :bundleId AND w.statusStr <> 'IGNORED'"),
 
-        @NamedQuery(name = "GET_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE", query = "select w.coordId, w.statusStr, w.pending from BundleActionBean w where w.bundleId = :bundleId"),
+        @NamedQuery(name = "GET_BUNDLE_UNIGNORED_ACTION_STATUS_PENDING_FOR_BUNDLE", query = "select w.coordId, w.statusStr, w.pending from BundleActionBean w where w.bundleId = :bundleId AND w.statusStr <> 'IGNORED'"),
 
         @NamedQuery(name = "GET_BUNDLE_ACTIONS", query = "select OBJECT(w) from BundleActionBean w"),
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/CoordinatorActionBean.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/CoordinatorActionBean.java b/core/src/main/java/org/apache/oozie/CoordinatorActionBean.java
index 03a7ed8..43d5103 100644
--- a/core/src/main/java/org/apache/oozie/CoordinatorActionBean.java
+++ b/core/src/main/java/org/apache/oozie/CoordinatorActionBean.java
@@ -122,7 +122,7 @@ import org.json.simple.JSONObject;
         // Query to maintain backward compatibility for coord job info command
         @NamedQuery(name = "GET_ALL_COLS_FOR_ACTIONS_FOR_COORD_JOB_ORDER_BY_NOMINAL_TIME", query = "select OBJECT(a) from CoordinatorActionBean a where a.jobId = :jobId order by a.nominalTimestamp"),
         // Query to retrieve action id, action status, pending status and external Id of not completed Coordinator actions
-        @NamedQuery(name = "GET_COORD_ACTIONS_NOT_COMPLETED", query = "select a.id, a.statusStr, a.pending, a.externalId, a.pushMissingDependencies, a.nominalTimestamp, a.createdTimestamp, a.jobId from CoordinatorActionBean a where a.jobId = :jobId AND a.statusStr <> 'FAILED' AND a.statusStr <> 'TIMEDOUT' AND a.statusStr <> 'SUCCEEDED' AND a.statusStr <> 'KILLED'"),
+        @NamedQuery(name = "GET_COORD_ACTIONS_NOT_COMPLETED", query = "select a.id, a.statusStr, a.pending, a.externalId, a.pushMissingDependencies, a.nominalTimestamp, a.createdTimestamp, a.jobId from CoordinatorActionBean a where a.jobId = :jobId AND a.statusStr <> 'FAILED' AND a.statusStr <> 'TIMEDOUT' AND a.statusStr <> 'SUCCEEDED' AND a.statusStr <> 'KILLED' AND a.statusStr <> 'IGNORED'"),
 
         // Query to retrieve action id, action status, pending status and external Id of running Coordinator actions
         @NamedQuery(name = "GET_COORD_ACTIONS_RUNNING", query = "select a.id, a.statusStr, a.pending, a.externalId, a.nominalTimestamp, a.createdTimestamp from CoordinatorActionBean a where a.jobId = :jobId and a.statusStr = 'RUNNING'"),
@@ -134,7 +134,7 @@ import org.json.simple.JSONObject;
         @NamedQuery(name = "GET_COORD_ACTIONS_PENDING_COUNT", query = "select count(a) from CoordinatorActionBean a where a.jobId = :jobId AND a.pending > 0"),
 
         // Query to retrieve status of Coordinator actions
-        @NamedQuery(name = "GET_COORD_ACTIONS_STATUS", query = "select a.statusStr from CoordinatorActionBean a where a.jobId = :jobId"),
+        @NamedQuery(name = "GET_COORD_ACTIONS_STATUS_UNIGNORED", query = "select a.statusStr from CoordinatorActionBean a where a.jobId = :jobId AND a.statusStr <> 'IGNORED'"),
 
         @NamedQuery(name = "GET_COORD_ACTION_FOR_COORD_JOB_BY_ACTION_NUMBER", query = "select a.id from CoordinatorActionBean a where a.jobId = :jobId AND a.actionNumber = :actionNumber"),
 
@@ -149,7 +149,7 @@ import org.json.simple.JSONObject;
 
         @NamedQuery(name = "GET_COORD_ACTIONS_FOR_RECOVERY_OLDER_THAN", query = "select a.id, a.jobId, a.statusStr, a.externalId, a.pending from CoordinatorActionBean a where a.pending > 0 AND (a.statusStr = 'SUSPENDED' OR a.statusStr = 'KILLED' OR a.statusStr = 'RUNNING') AND a.lastModifiedTimestamp <= :lastModifiedTime"),
         // Select query used by rerun, requires almost all columns so select * is used
-        @NamedQuery(name = "GET_ACTIONS_FOR_DATES", query = "select OBJECT(a) from CoordinatorActionBean a where a.jobId = :jobId AND (a.statusStr = 'TIMEDOUT' OR a.statusStr = 'SUCCEEDED' OR a.statusStr = 'KILLED' OR a.statusStr = 'FAILED') AND a.nominalTimestamp >= :startTime AND a.nominalTimestamp <= :endTime"),
+        @NamedQuery(name = "GET_ACTIONS_FOR_DATES", query = "select OBJECT(a) from CoordinatorActionBean a where a.jobId = :jobId AND (a.statusStr = 'TIMEDOUT' OR a.statusStr = 'SUCCEEDED' OR a.statusStr = 'KILLED' OR a.statusStr = 'FAILED' OR a.statusStr = 'IGNORED') AND a.nominalTimestamp >= :startTime AND a.nominalTimestamp <= :endTime"),
         // Select query used by log
         @NamedQuery(name = "GET_ACTION_IDS_FOR_DATES", query = "select a.id from CoordinatorActionBean a where a.jobId = :jobId AND (a.statusStr = 'TIMEDOUT' OR a.statusStr = 'SUCCEEDED' OR a.statusStr = 'KILLED' OR a.statusStr = 'FAILED') AND a.nominalTimestamp >= :startTime AND a.nominalTimestamp <= :endTime"),
         // Select query used by rerun, requires almost all columns so select * is used

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/CoordinatorEngine.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/CoordinatorEngine.java b/core/src/main/java/org/apache/oozie/CoordinatorEngine.java
index e627a95..1011b9e 100644
--- a/core/src/main/java/org/apache/oozie/CoordinatorEngine.java
+++ b/core/src/main/java/org/apache/oozie/CoordinatorEngine.java
@@ -42,6 +42,7 @@ import org.apache.oozie.client.rest.RestConstants;
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.coord.CoordActionInfoXCommand;
 import org.apache.oozie.util.CoordActionsInDateRange;
+import org.apache.oozie.command.coord.CoordActionsIgnoreXCommand;
 import org.apache.oozie.command.coord.CoordActionsKillXCommand;
 import org.apache.oozie.command.coord.CoordChangeXCommand;
 import org.apache.oozie.command.coord.CoordJobXCommand;
@@ -211,7 +212,18 @@ public class CoordinatorEngine extends BaseEngine {
     public void change(String jobId, String changeValue) throws CoordinatorEngineException {
         try {
             new CoordChangeXCommand(jobId, changeValue).call();
-            LOG.info("User " + user + " changed the Coordinator job " + jobId + " to " + changeValue);
+            LOG.info("User " + user + " changed the Coordinator job [" + jobId + "] to " + changeValue);
+        }
+        catch (CommandException e) {
+            throw new CoordinatorEngineException(e);
+        }
+    }
+
+    public CoordinatorActionInfo ignore(String jobId, String type, String scope) throws CoordinatorEngineException {
+        try {
+            LOG.info("User " + user + " ignore a Coordinator Action (s) [" + scope + "] of the Coordinator Job ["
+                    + jobId + "]");
+            return new CoordActionsIgnoreXCommand(jobId, type, scope).call();
         }
         catch (CommandException e) {
             throw new CoordinatorEngineException(e);

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java b/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
index 7915698..b77082c 100644
--- a/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
+++ b/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
@@ -638,6 +638,7 @@ public class CoordinatorJobBean implements Writable, CoordinatorJob, JsonBean {
             case FAILED:
             case KILLED:
             case DONEWITHERROR:
+            case IGNORED:
                 isTerminal = true;
                 break;
             default:

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/ErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/ErrorCode.java b/core/src/main/java/org/apache/oozie/ErrorCode.java
index 8b573ce..7d4af02 100644
--- a/core/src/main/java/org/apache/oozie/ErrorCode.java
+++ b/core/src/main/java/org/apache/oozie/ErrorCode.java
@@ -205,7 +205,7 @@ public enum ErrorCode {
     E1021(XLog.STD, "Coord Action Input Check Error: {0}"),
     E1022(XLog.STD, "Cannot delete running/completed coordinator action: [{0}]"),
     E1023(XLog.STD, "Coord Job update Error: [{0}]"),
-
+    E1024(XLog.STD, "Cannot run ignore command: [{0}]"),
 
     E1100(XLog.STD, "Command precondition does not hold before execution, [{0}]"),
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/IgnoreTransitionXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/IgnoreTransitionXCommand.java b/core/src/main/java/org/apache/oozie/command/IgnoreTransitionXCommand.java
new file mode 100644
index 0000000..6b96776
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/command/IgnoreTransitionXCommand.java
@@ -0,0 +1,54 @@
+/**
+ * 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.command;
+
+public abstract class IgnoreTransitionXCommand<T> extends TransitionXCommand<T> {
+
+    protected T ret;
+
+    public IgnoreTransitionXCommand(String name, String type, int priority) {
+        super(name, type, priority);
+    }
+
+    @Override
+    protected T execute() throws CommandException {
+        try {
+            transitToNext();
+            ignoreChildren();
+            updateJob();
+            performWrites();
+        }
+        finally {
+            notifyParent();
+        }
+        return ret;
+    }
+
+    public abstract void ignoreChildren() throws CommandException;
+
+    @Override
+    public void transitToNext() throws CommandException{
+    }
+    @Override
+    public void notifyParent(){
+    }
+    @Override
+    public void updateJob(){
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
index cd2c844..1962748 100644
--- a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
@@ -219,7 +219,7 @@ public class BundleJobChangeXCommand extends XCommand<Void> {
         try {
             this.bundleJob = BundleJobQueryExecutor.getInstance().get(BundleJobQuery.GET_BUNDLE_JOB, bundleJob.getId());
             this.bundleActions = BundleActionQueryExecutor.getInstance().getList(
-                    BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, bundleJob.getId());
+                    BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, bundleJob.getId());
         }
         catch (JPAExecutorException Ex) {
             throw new CommandException(ErrorCode.E1311, this.jobId);

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/bundle/BundleJobResumeXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobResumeXCommand.java b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobResumeXCommand.java
index 1a11765..8ccab0e 100644
--- a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobResumeXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobResumeXCommand.java
@@ -160,7 +160,7 @@ public class BundleJobResumeXCommand extends ResumeTransitionXCommand {
         try {
             bundleJob = BundleJobQueryExecutor.getInstance().get(BundleJobQuery.GET_BUNDLE_JOB, bundleId);
             bundleActions = BundleActionQueryExecutor.getInstance().getList(
-                    BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, bundleId);
+                    BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, bundleId);
         }
         catch (Exception Ex) {
             throw new CommandException(ErrorCode.E0604, bundleId);

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/bundle/BundleJobSuspendXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobSuspendXCommand.java b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobSuspendXCommand.java
index 58afeac..65eb272 100644
--- a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobSuspendXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobSuspendXCommand.java
@@ -114,7 +114,7 @@ public class BundleJobSuspendXCommand extends SuspendTransitionXCommand {
 
         try {
             bundleActions = BundleActionQueryExecutor.getInstance().getList(
-                    BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, bundleJob.getId());
+                    BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, bundleJob.getId());
         }
         catch (Exception Ex) {
             throw new CommandException(ErrorCode.E1311, jobId);

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/bundle/BundleJobXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobXCommand.java b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobXCommand.java
index 2ddfbdb..4b12a03 100644
--- a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobXCommand.java
@@ -88,7 +88,6 @@ public class BundleJobXCommand extends XCommand<BundleJobBean> {
             BundleJobBean bundleJob = null;
             if (jpaService != null) {
                 bundleJob = jpaService.execute(new BundleJobGetJPAExecutor(id));
-
                 List<CoordinatorJobBean> coordinators = jpaService.execute(new BundleJobGetCoordinatorsJPAExecutor(id));
                 bundleJob.setCoordJobs(coordinators);
             }

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java b/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java
index 3ab3273..c101f45 100644
--- a/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java
@@ -72,7 +72,7 @@ public class BundleKillXCommand extends KillTransitionXCommand {
         try {
             this.bundleJob = BundleJobQueryExecutor.getInstance().get(BundleJobQuery.GET_BUNDLE_JOB, jobId);
             this.bundleActions = BundleActionQueryExecutor.getInstance().getList(
-                    BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, jobId);
+                    BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, jobId);
             LogUtils.setLogInfo(bundleJob, logInfo);
             super.setJob(bundleJob);
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/bundle/BundleRerunXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/bundle/BundleRerunXCommand.java b/core/src/main/java/org/apache/oozie/command/bundle/BundleRerunXCommand.java
index 90e205a..f5f4b3f 100644
--- a/core/src/main/java/org/apache/oozie/command/bundle/BundleRerunXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/bundle/BundleRerunXCommand.java
@@ -88,7 +88,7 @@ public class BundleRerunXCommand extends RerunTransitionXCommand<Void> {
         try {
             this.bundleJob = BundleJobQueryExecutor.getInstance().get(BundleJobQuery.GET_BUNDLE_JOB, jobId);
             this.bundleActions = BundleActionQueryExecutor.getInstance().getList(
-                    BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, jobId);
+                    BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, jobId);
             LogUtils.setLogInfo(bundleJob, logInfo);
             super.setJob(bundleJob);
             prevPending = bundleJob.isPending();

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/coord/CoordActionsIgnoreXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordActionsIgnoreXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordActionsIgnoreXCommand.java
new file mode 100644
index 0000000..ed85a40
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordActionsIgnoreXCommand.java
@@ -0,0 +1,138 @@
+/**
+ * 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.command.coord;
+
+import java.util.Date;
+import java.util.List;
+
+import org.apache.oozie.CoordinatorActionBean;
+import org.apache.oozie.CoordinatorActionInfo;
+import org.apache.oozie.CoordinatorJobBean;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.client.CoordinatorAction.Status;
+import org.apache.oozie.client.CoordinatorJob;
+import org.apache.oozie.command.CommandException;
+import org.apache.oozie.command.IgnoreTransitionXCommand;
+import org.apache.oozie.command.PreconditionException;
+import org.apache.oozie.coord.CoordUtils;
+import org.apache.oozie.executor.jpa.BatchQueryExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
+import org.apache.oozie.executor.jpa.JPAExecutorException;
+import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
+import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
+import org.apache.oozie.util.LogUtils;
+import org.apache.oozie.util.ParamChecker;
+
+
+public class CoordActionsIgnoreXCommand extends IgnoreTransitionXCommand<CoordinatorActionInfo>{
+
+    CoordinatorJobBean coordJob;
+    String jobId;
+    String type;
+    String scope;
+    private List<CoordinatorActionBean> coordActions;
+
+    public CoordActionsIgnoreXCommand(String coordId, String type, String scope) {
+        super("coord_action_ignore", "coord_action_ignore", 1);
+        this.jobId = ParamChecker.notEmpty(coordId, "coordJobId");
+        this.type = ParamChecker.notEmpty(type, "type");
+        this.scope = ParamChecker.notEmpty(scope, "scope");
+    }
+
+    @Override
+    protected void verifyPrecondition() throws CommandException, PreconditionException {
+        // no actions to ignore for PREP job
+        if (coordJob.getStatus() == CoordinatorJob.Status.PREP) {
+            LOG.info("CoordActionsIgnoreXCommand is not able to run, job status=" + coordJob.getStatus() + ", jobid=" + jobId);
+            throw new PreconditionException(ErrorCode.E1024, "No actions are materialized to ignore");
+        }
+        StringBuilder ineligibleActions = new StringBuilder();
+        if (!checkAllActionsStatus(ineligibleActions)) {
+            throw new CommandException(ErrorCode.E1024,
+                    "part or all actions are not eligible to ignore, check state of action number(s) ["
+                            + ineligibleActions.toString() + "]");
+        }
+    }
+
+    @Override
+    public void ignoreChildren() throws CommandException {
+        for (CoordinatorActionBean action : coordActions) {
+            action.setStatus(Status.IGNORED);
+            action.setLastModifiedTime(new Date());
+            action.setPending(0);
+            updateList.add(new UpdateEntry<CoordActionQuery>(CoordActionQuery.UPDATE_COORD_ACTION_STATUS_PENDING_TIME,
+                    action));
+            LOG.info("Ignore coord action = [{0}], new status = [{1}]", action.getId(), action.getStatus());
+        }
+        ret = new CoordinatorActionInfo(coordActions);
+    }
+
+    private boolean checkAllActionsStatus(StringBuilder ineligibleActions)
+            throws CommandException {
+        boolean ret = true;
+        if (coordActions == null || coordActions.size() == 0) {
+            throw new CommandException(ErrorCode.E1024, "no actions are eligible to ignore");
+        }
+        for (CoordinatorActionBean action : coordActions) {
+            ParamChecker.notNull(action, "Action cannot be null");
+            if (!(action.getStatus() == Status.FAILED || action.getStatus() == Status.KILLED
+                    || action.getStatus() == Status.TIMEDOUT)) {
+                LOG.info("Cannot ignore coord action = [{0}], since its status is [{1}]", action.getId(),
+                        action.getStatus());
+                if (ineligibleActions.length() != 0) {
+                    ineligibleActions.append(",");
+                }
+                ineligibleActions.append(action.getActionNumber());
+                ret = false;
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public void performWrites() throws CommandException {
+        try {
+            BatchQueryExecutor.getInstance().executeBatchInsertUpdateDelete(null, updateList, null);
+        }
+        catch (JPAExecutorException jex) {
+            throw new CommandException(jex);
+        }
+    }
+
+    @Override
+    protected boolean isLockRequired() {
+        return true;
+    }
+
+    @Override
+    public String getEntityKey() {
+        return jobId;
+    }
+
+    @Override
+    protected void loadState() throws CommandException {
+        try{
+            coordJob = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_STATUS_PARENTID, jobId);
+            coordActions = CoordUtils.getCoordActions(type, jobId, scope, false);
+        }catch (Exception ex){
+            throw new CommandException(ErrorCode.E0603, ex.getMessage(), ex);
+        }
+        LogUtils.setLogInfo(this.coordJob, logInfo);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
index fb31e9a..ea7df17 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
@@ -201,15 +201,26 @@ public class CoordChangeXCommand extends CoordinatorXCommand<Void> {
      */
     private void checkStatusChange(CoordinatorJobBean coordJob, CoordinatorJob.Status jobStatus)
             throws CommandException {
-        if (!jobStatus.equals(CoordinatorJob.Status.RUNNING)) {
-            throw new CommandException(ErrorCode.E1015, jobStatus, " must be RUNNING");
+        if (!jobStatus.equals(CoordinatorJob.Status.RUNNING) && !jobStatus.equals(CoordinatorJob.Status.IGNORED)) {
+            throw new CommandException(ErrorCode.E1015, jobStatus, " must be RUNNING or IGNORED");
         }
 
-        if (!(coordJob.getStatus().equals(CoordinatorJob.Status.FAILED) || coordJob.getStatus().equals(
-                CoordinatorJob.Status.KILLED))) {
-            throw new CommandException(ErrorCode.E1015, jobStatus,
-                    " Only FAILED or KILLED job can be changed to RUNNING. Current job status is "
-                            + coordJob.getStatus());
+        if (jobStatus.equals(CoordinatorJob.Status.RUNNING)) {
+            if (!(coordJob.getStatus().equals(CoordinatorJob.Status.FAILED) || coordJob.getStatus().equals(
+                    CoordinatorJob.Status.KILLED) || coordJob.getStatus().equals(CoordinatorJob.Status.IGNORED))) {
+                throw new CommandException(ErrorCode.E1015, jobStatus,
+                        " Only FAILED, KILLED, IGNORED job can be changed to RUNNING. Current job status is "
+                                + coordJob.getStatus());
+            }
+        }
+        else {
+            if (!(coordJob.getStatus().equals(CoordinatorJob.Status.FAILED) || coordJob.getStatus().equals(
+                    CoordinatorJob.Status.KILLED))
+                    || coordJob.isPending()) {
+                throw new CommandException(ErrorCode.E1015, jobStatus,
+                        " Only FAILED or KILLED non-pending job can be changed to IGNORED. Current job status is "
+                                + coordJob.getStatus() + " and pending status is " + coordJob.isPending());
+            }
         }
     }
 
@@ -299,7 +310,8 @@ public class CoordChangeXCommand extends CoordinatorXCommand<Void> {
     private void check(CoordinatorJobBean coordJob, Date newEndTime, Integer newConcurrency, Date newPauseTime,
             CoordinatorJob.Status jobStatus) throws CommandException {
 
-        if (coordJob.getStatus() == CoordinatorJob.Status.KILLED) {
+        if (coordJob.getStatus() == CoordinatorJob.Status.KILLED
+                || coordJob.getStatus() == CoordinatorJob.Status.IGNORED) {
             if (jobStatus == null || (newEndTime != null || newConcurrency != null || newPauseTime != null)) {
                 throw new CommandException(ErrorCode.E1016);
             }
@@ -396,13 +408,16 @@ public class CoordChangeXCommand extends CoordinatorXCommand<Void> {
             }
             if (jobStatus != null) {
                 coordJob.setStatus(jobStatus);
-                LOG.info("Coord status is changed to RUNNING from " + prevStatus);
-                coordJob.setPending();
+                LOG.info("Coord status is changed to " + jobStatus + " from " + prevStatus);
                 if (jobStatus.equals(CoordinatorJob.Status.RUNNING)) {
+                    coordJob.setPending();
                     if (coordJob.getNextMaterializedTime() != null
                             && coordJob.getEndTime().after(coordJob.getNextMaterializedTime())) {
                         coordJob.resetDoneMaterialization();
                     }
+                } else if (jobStatus.equals(CoordinatorJob.Status.IGNORED)) {
+                    coordJob.resetPending();
+                    coordJob.setDoneMaterialization();
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
index 2e5f6b4..91e9e09 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
@@ -106,9 +106,10 @@ public class CoordKillXCommand extends KillTransitionXCommand {
         if (coordJob.getStatus() == CoordinatorJob.Status.SUCCEEDED
                 || coordJob.getStatus() == CoordinatorJob.Status.FAILED
                 || coordJob.getStatus() == CoordinatorJob.Status.DONEWITHERROR
-                || coordJob.getStatus() == CoordinatorJob.Status.KILLED) {
-            LOG.info("CoordKillXCommand not killed - job either finished SUCCEEDED, FAILED, KILLED or DONEWITHERROR, job id = "
-                    + jobId + ", status = " + coordJob.getStatus());
+                || coordJob.getStatus() == CoordinatorJob.Status.KILLED
+                || coordJob.getStatus() == CoordinatorJob.Status.IGNORED) {
+            LOG.info("CoordKillXCommand not killed - job either finished SUCCEEDED, FAILED, KILLED, DONEWITHERROR"
+                    + " or IGNORED,job id = " + jobId + ", status = " + coordJob.getStatus());
             throw new PreconditionException(ErrorCode.E1020, jobId);
         }
     }

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java
index 78a2614..8aeb803 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java
@@ -263,16 +263,22 @@ public class CoordRerunXCommand extends RerunTransitionXCommand<CoordinatorActio
     protected void verifyPrecondition() throws CommandException, PreconditionException {
         BundleStatusUpdateXCommand bundleStatusUpdate = new BundleStatusUpdateXCommand(coordJob, coordJob.getStatus());
 
-        // no actioins have been created for PREP job
-        if (coordJob.getStatus() == CoordinatorJob.Status.PREP) {
+        // no actions have been created for PREP job
+        if (coordJob.getStatus() == CoordinatorJob.Status.PREP || coordJob.getStatus() == CoordinatorJob.Status.IGNORED) {
             LOG.info("CoordRerunXCommand is not able to run, job status=" + coordJob.getStatus() + ", jobid=" + jobId);
             // Call the parent so the pending flag is reset and state transition
             // of bundle can happen
             if (coordJob.getBundleId() != null) {
                 bundleStatusUpdate.call();
             }
-            throw new CommandException(ErrorCode.E1018,
-                    "coordinator job is PREP so no actions are materialized to rerun!");
+            if (coordJob.getStatus() == CoordinatorJob.Status.PREP) {
+                throw new CommandException(ErrorCode.E1018,
+                        "coordinator job is PREP so no actions are materialized to rerun!");
+            }
+            else {
+                throw new CommandException(ErrorCode.E1018,
+                        "coordinator job is IGNORED, please change it to RUNNING before rerunning actions");
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java
index 0db2dad..b1b2fcd 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordSuspendXCommand.java
@@ -35,7 +35,6 @@ import org.apache.oozie.executor.jpa.BatchQueryExecutor;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
 import org.apache.oozie.executor.jpa.CoordJobGetActionsRunningJPAExecutor;
-import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
@@ -102,7 +101,8 @@ public class CoordSuspendXCommand extends SuspendTransitionXCommand {
         super.eagerVerifyPrecondition();
         if (coordJob.getStatus() == CoordinatorJob.Status.SUCCEEDED
                 || coordJob.getStatus() == CoordinatorJob.Status.FAILED
-                || coordJob.getStatus() == CoordinatorJob.Status.KILLED) {
+                || coordJob.getStatus() == CoordinatorJob.Status.KILLED
+                || coordJob.getStatus() == CoordinatorJob.Status.IGNORED) {
             LOG.info("CoordSuspendXCommand is not going to execute because "
                     + "job finished or failed or killed, id = " + jobId + ", status = " + coordJob.getStatus());
             throw new PreconditionException(ErrorCode.E0728, jobId, coordJob.getStatus().toString());

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/coord/CoordUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/coord/CoordUtils.java b/core/src/main/java/org/apache/oozie/coord/CoordUtils.java
index 0319942..26db068 100644
--- a/core/src/main/java/org/apache/oozie/coord/CoordUtils.java
+++ b/core/src/main/java/org/apache/oozie/coord/CoordUtils.java
@@ -152,7 +152,7 @@ public class CoordUtils {
      * @return the list of all Coordinator actions for action range
      * @throws CommandException thrown if failed to get coordinator actions by given id range
      */
-     static List<CoordinatorActionBean> getCoordActionsFromIds(String jobId, String scope) throws CommandException {
+     public static List<CoordinatorActionBean> getCoordActionsFromIds(String jobId, String scope) throws CommandException {
         JPAService jpaService = Services.get().get(JPAService.class);
         ParamChecker.notEmpty(jobId, "jobId");
         ParamChecker.notEmpty(scope, "scope");

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/executor/jpa/BundleActionQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/executor/jpa/BundleActionQueryExecutor.java b/core/src/main/java/org/apache/oozie/executor/jpa/BundleActionQueryExecutor.java
index 9c628ab..d2331e8 100644
--- a/core/src/main/java/org/apache/oozie/executor/jpa/BundleActionQueryExecutor.java
+++ b/core/src/main/java/org/apache/oozie/executor/jpa/BundleActionQueryExecutor.java
@@ -44,10 +44,10 @@ public class BundleActionQueryExecutor extends
         UPDATE_BUNDLE_ACTION_STATUS_PENDING_MODTIME,
         UPDATE_BUNDLE_ACTION_STATUS_PENDING_MODTIME_COORDID,
         GET_BUNDLE_ACTION,
-        GET_BUNDLE_ACTIONS_FOR_BUNDLE,
+        GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE,
         GET_BUNDLE_ACTIONS_BY_LAST_MODIFIED_TIME,
         GET_BUNDLE_WAITING_ACTIONS_OLDER_THAN,
-        GET_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE
+        GET_BUNDLE_UNIGNORED_ACTION_STATUS_PENDING_FOR_BUNDLE
     };
 
     private static BundleActionQueryExecutor instance = new BundleActionQueryExecutor();
@@ -108,7 +108,7 @@ public class BundleActionQueryExecutor extends
             case GET_BUNDLE_ACTION:
                 query.setParameter("bundleActionId", parameters[0]);
                 break;
-            case GET_BUNDLE_ACTIONS_FOR_BUNDLE:
+            case GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE:
                 query.setParameter("bundleId", parameters[0]);
                 break;
             case GET_BUNDLE_ACTIONS_BY_LAST_MODIFIED_TIME:
@@ -118,7 +118,7 @@ public class BundleActionQueryExecutor extends
                 Timestamp ts = new Timestamp(System.currentTimeMillis() - (Long)parameters[0] * 1000);
                 query.setParameter("lastModifiedTime", ts);
                 break;
-            case GET_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE:
+            case GET_BUNDLE_UNIGNORED_ACTION_STATUS_PENDING_FOR_BUNDLE:
                 query.setParameter("bundleId", parameters[0]);
                 break;
             default:
@@ -153,7 +153,7 @@ public class BundleActionQueryExecutor extends
         Object[] arr;
         switch (namedQuery) {
             case GET_BUNDLE_ACTION:
-            case GET_BUNDLE_ACTIONS_FOR_BUNDLE:
+            case GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE:
                 bean = (BundleActionBean) ret;
                 break;
             case GET_BUNDLE_ACTIONS_BY_LAST_MODIFIED_TIME:
@@ -169,7 +169,7 @@ public class BundleActionQueryExecutor extends
                 bean.setCoordId((String) arr[3]);
                 bean.setCoordName((String) arr[4]);
                 break;
-            case GET_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE:
+            case GET_BUNDLE_UNIGNORED_ACTION_STATUS_PENDING_FOR_BUNDLE:
                 bean = new BundleActionBean();
                 arr = (Object[]) ret;
                 bean.setCoordId((String) arr[0]);

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsStatusJPAExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsStatusJPAExecutor.java b/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsStatusJPAExecutor.java
index 2cee5f0..68a3e16 100644
--- a/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsStatusJPAExecutor.java
+++ b/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobGetActionsStatusJPAExecutor.java
@@ -48,7 +48,7 @@ public class CoordJobGetActionsStatusJPAExecutor implements JPAExecutor<List<Coo
     @Override
     public List<CoordinatorAction.Status> execute(EntityManager em) throws JPAExecutorException {
         try {
-            Query q = em.createNamedQuery("GET_COORD_ACTIONS_STATUS");
+            Query q = em.createNamedQuery("GET_COORD_ACTIONS_STATUS_UNIGNORED");
             q.setParameter("jobId", coordJobId);
             List<String> coordStatusResultList = q.getResultList();
             List<CoordinatorAction.Status> coordStatus = new ArrayList<CoordinatorAction.Status>();

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/service/StatusTransitService.java b/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
index 0789d84..b11b3d2 100644
--- a/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
+++ b/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
@@ -189,7 +189,7 @@ public class StatusTransitService implements Service {
                             Job.Status[] bundleStatus = new Job.Status[1];
                             bundleStatus[0] = bundleJob.getStatus();
                             List<BundleActionBean> bundleActions = BundleActionQueryExecutor.getInstance().getList(
-                                BundleActionQuery.GET_BUNDLE_ACTION_STATUS_PENDING_FOR_BUNDLE, jobId);
+                                BundleActionQuery.GET_BUNDLE_UNIGNORED_ACTION_STATUS_PENDING_FOR_BUNDLE, jobId);
                             HashMap<Job.Status, Integer> bundleActionStatus = new HashMap<Job.Status, Integer>();
                             boolean foundPending = false;
                             for (BundleActionBean bAction : bundleActions) {
@@ -744,10 +744,11 @@ public class StatusTransitService implements Service {
                     }
                     // Running coord job might have pending false
                     Job.Status coordJobStatus = coordJob.getStatus();
-                    if (coordJob.isPending() || coordJobStatus.equals(Job.Status.PAUSED)
+                    if ((coordJob.isPending() || coordJobStatus.equals(Job.Status.PAUSED)
                             || coordJobStatus.equals(Job.Status.RUNNING)
                             || coordJobStatus.equals(Job.Status.RUNNINGWITHERROR)
-                            || coordJobStatus.equals(Job.Status.PAUSEDWITHERROR)) {
+                            || coordJobStatus.equals(Job.Status.PAUSEDWITHERROR))
+                            && !coordJobStatus.equals(Job.Status.IGNORED)) {
                         pendingJobCheckList.add(coordJob);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java b/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
index 8941a02..6948c67 100644
--- a/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
@@ -109,6 +109,17 @@ public abstract class BaseJobServlet extends JsonRestServlet {
             startCron();
             response.setStatus(HttpServletResponse.SC_OK);
         }
+        else if (action.equals(RestConstants.JOB_ACTION_IGNORE)) {
+            stopCron();
+            JSONObject json = ignoreJob(request, response);
+            startCron();
+            if (json != null) {
+                sendJsonResponse(response, HttpServletResponse.SC_OK, json);
+            }
+            else {
+            response.setStatus(HttpServletResponse.SC_OK);
+            }
+        }
         else if (action.equals(RestConstants.JOB_ACTION_RERUN)) {
             validateContentType(request, RestConstants.XML_CONTENT_TYPE);
             Configuration conf = new XConfiguration(request.getInputStream());
@@ -162,6 +173,9 @@ public abstract class BaseJobServlet extends JsonRestServlet {
         }
     }
 
+    abstract JSONObject ignoreJob(HttpServletRequest request, HttpServletResponse response) throws XServletException,
+            IOException;
+
     /**
      * Validate the configuration user/group. <p/>
      *

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java b/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java
index 487a371..3bba16f 100644
--- a/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java
@@ -220,4 +220,9 @@ public class V0JobServlet extends BaseJobServlet {
             IOException {
         throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0302, "Not supported in v0");
     }
+
+    @Override
+    protected JSONObject ignoreJob(HttpServletRequest request, HttpServletResponse response) throws XServletException, IOException {
+        throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0302, "Not supported in v0");
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java b/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
index e03d7a7..8a34763 100644
--- a/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
@@ -177,6 +177,10 @@ public class V1JobServlet extends BaseJobServlet {
             changeCoordinatorJob(request, response);
         }
     }
+    @Override
+    protected JSONObject ignoreJob(HttpServletRequest request, HttpServletResponse response) throws XServletException, IOException {
+        throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0302, "Not supported in v1");
+    }
 
     /*
      * protected method to reRun a job

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java b/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java
index e961f30..371cbb4 100644
--- a/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java
@@ -18,19 +18,26 @@
 package org.apache.oozie.servlet;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.oozie.CoordinatorActionBean;
+import org.apache.oozie.CoordinatorActionInfo;
 import org.apache.oozie.CoordinatorEngine;
 import org.apache.oozie.CoordinatorEngineException;
 import org.apache.oozie.DagEngine;
 import org.apache.oozie.DagEngineException;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.rest.JsonBean;
 import org.apache.oozie.client.rest.JsonTags;
 import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.command.CommandException;
 import org.apache.oozie.service.CoordinatorEngineService;
 import org.apache.oozie.service.DagEngineService;
 import org.apache.oozie.service.Services;
@@ -119,4 +126,66 @@ public class V2JobServlet extends V1JobServlet {
         return json;
     }
 
+    /**
+     * Ignore a coordinator job
+     * @param request request object
+     * @param response response object
+     * @throws XServletException
+     * @throws IOException
+     */
+    @Override
+    protected JSONObject ignoreJob(HttpServletRequest request, HttpServletResponse response) throws XServletException, IOException {
+        String jobId = getResourceName(request);
+        if (jobId.endsWith("-W")) {
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0302, "Workflow Ignore Not supported");
+        } else if (jobId.endsWith("-B")) {
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0302, "Bundle Ignore Not supported");
+        } else {
+            return ignoreCoordinatorJob(request, response);
+        }
+
+    }
+
+    /**
+     * Ignore a coordinator job/action
+     *
+     * @param request servlet request
+     * @param response servlet response
+     * @throws XServletException
+     */
+    @SuppressWarnings("unchecked")
+    private JSONObject ignoreCoordinatorJob(HttpServletRequest request, HttpServletResponse response)
+            throws XServletException {
+        JSONObject json = null;
+        CoordinatorEngine coordEngine = Services.get().get(CoordinatorEngineService.class).getCoordinatorEngine(
+                getUser(request));
+        String jobId = getResourceName(request);
+        String type = request.getParameter(RestConstants.JOB_COORD_RANGE_TYPE_PARAM);
+        String scope = request.getParameter(RestConstants.JOB_COORD_SCOPE_PARAM);
+        String changeValue = "status=" + CoordinatorAction.Status.IGNORED;
+        List<CoordinatorActionBean> coordActions = new ArrayList<CoordinatorActionBean>();
+        try {
+            if (type != null && !type.equals(RestConstants.JOB_COORD_SCOPE_ACTION)) {
+                throw new CommandException(ErrorCode.E1024, "Currently ignore only support -action option");
+            }
+            CoordinatorActionInfo coordInfo = null;
+            if(scope == null || scope.isEmpty()) {
+                coordEngine.change(jobId, changeValue);
+            } else{
+                coordInfo = coordEngine.ignore(jobId, type, scope);
+            }
+            if(coordInfo != null) {
+                coordActions = coordInfo.getCoordActions();
+                json = new JSONObject();
+                json.put(JsonTags.COORDINATOR_ACTIONS, CoordinatorActionBean.toJSONArray(coordActions, "GMT"));
+            }
+            return json;
+        }
+        catch (CommandException ex) {
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ex);
+        }
+        catch (CoordinatorEngineException ex) {
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ex);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java b/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java
index ea2ca27..e15e567 100644
--- a/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java
+++ b/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java
@@ -500,7 +500,9 @@ public class TestCoordinatorEngine extends XTestCase {
         }
         catch (CoordinatorEngineException ex) {
             assertEquals(ErrorCode.E0421, ex.getErrorCode());
-            assertEquals("E0421: Invalid job filter [status=blahblah], invalid status value [blahblah]. Valid status values are: [WAITING READY SUBMITTED RUNNING SUSPENDED TIMEDOUT SUCCEEDED KILLED FAILED DISCARDED ]", ex.getMessage());
+            assertEquals("E0421: Invalid job filter [status=blahblah], invalid status value [blahblah]."
+                    + " Valid status values are: [WAITING READY SUBMITTED RUNNING SUSPENDED TIMEDOUT "
+                    + "SUCCEEDED KILLED FAILED IGNORED ]", ex.getMessage());
         }
 
         // Check for empty status value
@@ -509,7 +511,9 @@ public class TestCoordinatorEngine extends XTestCase {
         }
         catch (CoordinatorEngineException ex) {
             assertEquals(ErrorCode.E0421, ex.getErrorCode());
-            assertEquals("E0421: Invalid job filter [status=\"\"], invalid status value [\"\"]. Valid status values are: [WAITING READY SUBMITTED RUNNING SUSPENDED TIMEDOUT SUCCEEDED KILLED FAILED DISCARDED ]", ex.getMessage());
+            assertEquals("E0421: Invalid job filter [status=\"\"], invalid status value [\"\"]. "
+                    + "Valid status values are: [WAITING READY SUBMITTED RUNNING SUSPENDED TIMEDOUT "
+                    + "SUCCEEDED KILLED FAILED IGNORED ]", ex.getMessage());
         }
 
         // Check for invalid filter option

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/action/hadoop/TestOozieJobInfo.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/action/hadoop/TestOozieJobInfo.java b/core/src/test/java/org/apache/oozie/action/hadoop/TestOozieJobInfo.java
index 5aa9e6f..b1f4609 100644
--- a/core/src/test/java/org/apache/oozie/action/hadoop/TestOozieJobInfo.java
+++ b/core/src/test/java/org/apache/oozie/action/hadoop/TestOozieJobInfo.java
@@ -112,18 +112,18 @@ public class TestOozieJobInfo extends XDataTestCase {
         new BundleStartXCommand(job.getId()).call();
         sleep(2000);
         List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
         assertEquals(1, actions.size());
         final String bundleID = job.getId();
         waitFor(200000, new Predicate() {
             public boolean evaluate() throws Exception {
                 List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                        BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, bundleID);
+                        BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, bundleID);
                 return actions.get(0).getStatus().equals(Job.Status.RUNNING);
             }
         });
 
-        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE,
+        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE,
                 job.getId());
         final String cordID = actions.get(0).getCoordId();
         waitFor(200000, new Predicate() {

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/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 76c7a29..8a85cd2 100644
--- a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
+++ b/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
@@ -28,6 +28,7 @@ import java.util.concurrent.Callable;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.oozie.CoordinatorJobBean;
 import org.apache.oozie.cli.OozieCLI;
 import org.apache.oozie.client.rest.RestConstants;
 import org.apache.oozie.servlet.DagServletTestCase;
@@ -57,7 +58,7 @@ public class TestOozieCLI extends DagServletTestCase {
     static final String VERSION = "/v" + OozieClient.WS_PROTOCOL_VERSION;
     static final String[] END_POINTS = {"/versions", VERSION + "/jobs", VERSION + "/job/*", VERSION + "/admin/*"};
     static final Class[] SERVLET_CLASSES = { HeaderTestingVersionServlet.class, V1JobsServlet.class,
-            V1JobServlet.class, V1AdminServlet.class, V2JobServlet.class, V2AdminServlet.class };
+            V2JobServlet.class, V2AdminServlet.class, V2JobServlet.class, V2AdminServlet.class };
 
     @Override
     protected void setUp() throws Exception {
@@ -690,6 +691,61 @@ public class TestOozieCLI extends DagServletTestCase {
         });
     }
 
+    public void testCoordJobIgnore() throws Exception {
+        runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                String oozieUrl = getContextURL();
+                String[] args = new String[]{"job", "-oozie", oozieUrl, "-ignore", MockCoordinatorEngineService.JOB_ID + "1"};
+                assertEquals(0, new OozieCLI().run(args));
+                assertEquals(RestConstants.JOB_ACTION_CHANGE, MockCoordinatorEngineService.did);
+                assertTrue(MockCoordinatorEngineService.started.get(1));
+
+                // negative test for "oozie job -ignore <non-existent coord>"
+                MockCoordinatorEngineService.reset();
+                args = new String[] {
+                        "job","-oozie",oozieUrl,"ignore",
+                        MockDagEngineService.JOB_ID + (MockCoordinatorEngineService.coordJobs.size() + 1)};
+                assertEquals(-1, new OozieCLI().run(args));
+                assertNull(MockCoordinatorEngineService.did);
+                assertFalse(MockCoordinatorEngineService.started.get(1));
+                return null;
+            }
+        });
+    }
+
+    public void testCoordActionsIgnore() throws Exception {
+        runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                String oozieUrl = getContextURL();
+                String[] args = new String[]{"job", "-oozie", oozieUrl, "-ignore",
+                        MockCoordinatorEngineService.JOB_ID + "1", "-action", "1"};
+                assertEquals(0, new OozieCLI().run(args));
+                assertEquals(RestConstants.JOB_ACTION_IGNORE, MockCoordinatorEngineService.did);
+                assertTrue(MockCoordinatorEngineService.started.get(1));
+
+                // negative test for "oozie job -ignore <non-existent coord> -action 1"
+                MockCoordinatorEngineService.reset();
+                args = new String[]{"job", "-oozie", oozieUrl, "ignore",
+                        MockDagEngineService.JOB_ID + (MockCoordinatorEngineService.coordJobs.size() + 1), "-action", "1" };
+                assertEquals(-1, new OozieCLI().run(args));
+                assertNull(MockCoordinatorEngineService.did);
+                assertFalse(MockCoordinatorEngineService.started.get(1));
+
+                // negative test for "oozie job -ignore <id> -action (action is empty)"
+                MockCoordinatorEngineService.reset();
+                args = new String[]{"job", "-oozie", oozieUrl, "-ignore",
+                        MockCoordinatorEngineService.JOB_ID, "-action", ""};
+                assertEquals(-1, new OozieCLI().run(args));
+                assertNull(MockCoordinatorEngineService.did);
+                assertFalse(MockCoordinatorEngineService.started.get(1));
+
+                return null;
+            }
+        });
+    }
+
     public void testJobStatus() throws Exception {
         runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
             @Override

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java
index 225d48e..ae4717d 100644
--- a/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java
@@ -154,7 +154,7 @@ public class TestBundleJobSuspendXCommand extends XDataTestCase {
         sleep(2000);
 
         List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
 
         assertEquals(2, actions.size());
         assertNotNull(actions.get(0).getCoordId());
@@ -165,7 +165,7 @@ public class TestBundleJobSuspendXCommand extends XDataTestCase {
         job = jpaService.execute(bundleJobGetCmd);
         assertEquals(Job.Status.SUSPENDED, job.getStatus());
 
-        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE,
+        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE,
                 job.getId());
 
         assertEquals(true, actions.get(0).isPending());

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java
index 498ac0d..d4c43e5 100644
--- a/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java
@@ -119,7 +119,7 @@ public class TestBundleKillXCommand extends XDataTestCase {
         sleep(2000);
 
         List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
 
         assertEquals(2, actions.size());
         assertNotNull(actions.get(0).getCoordId());
@@ -130,7 +130,7 @@ public class TestBundleKillXCommand extends XDataTestCase {
         job = jpaService.execute(bundleJobGetCmd);
         assertEquals(Job.Status.KILLED, job.getStatus());
 
-        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE,
+        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE,
                 job.getId());
 
         assertEquals(true, actions.get(0).isPending());
@@ -168,7 +168,7 @@ public class TestBundleKillXCommand extends XDataTestCase {
         assertEquals(Job.Status.KILLED, job.getStatus());
 
         actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
         for (BundleActionBean action : actions) {
             assertEquals(0, action.getPending());
             assertEquals(CoordinatorJobBean.Status.KILLED, action.getStatus());
@@ -194,7 +194,7 @@ public class TestBundleKillXCommand extends XDataTestCase {
         job = jpaService.execute(bundleJobGetCmd);
         assertEquals(Job.Status.KILLED, job.getStatus());
         actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
         for (BundleActionBean action : actions) {
             assertEquals(0, action.getPending());
             assertEquals(CoordinatorJobBean.Status.KILLED, action.getStatus());

http://git-wip-us.apache.org/repos/asf/oozie/blob/f82c1240/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java
index 7d6542a..8531471 100644
--- a/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java
@@ -80,7 +80,7 @@ public class TestBundleStartXCommand extends XDataTestCase {
         sleep(2000);
 
         List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
 
         assertEquals(2, actions.size());
         assertEquals(true, actions.get(0).isCritical());
@@ -130,20 +130,20 @@ public class TestBundleStartXCommand extends XDataTestCase {
         sleep(2000);
 
         List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
         assertEquals(2, actions.size());
 
         final String jobId = job.getId();
         waitFor(200000, new Predicate() {
             public boolean evaluate() throws Exception {
                 List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                        BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, jobId);
+                        BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, jobId);
                 return actions.get(0).getStatus().equals(Job.Status.RUNNING)
                         && actions.get(1).getStatus().equals(Job.Status.RUNNING);
             }
         });
 
-        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE,
+        actions = BundleActionQueryExecutor.getInstance().getList(BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE,
                 job.getId());
         assertEquals(Job.Status.RUNNING, actions.get(0).getStatus());
         assertEquals(true, actions.get(0).isCritical());
@@ -175,7 +175,7 @@ public class TestBundleStartXCommand extends XDataTestCase {
         sleep(2000);
 
         List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
 
         assertEquals(2, actions.size());
         assertEquals(true, actions.get(0).isCritical());
@@ -265,7 +265,7 @@ public class TestBundleStartXCommand extends XDataTestCase {
         assertEquals(job.getStatus(), Job.Status.RUNNING);
         sleep(2000);
         List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
-                BundleActionQuery.GET_BUNDLE_ACTIONS_FOR_BUNDLE, job.getId());
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, job.getId());
         assertNull(actions.get(0).getCoordId());
         assertEquals(Job.Status.FAILED, actions.get(0).getStatus());
         Runnable runnable = new StatusTransitRunnable();