Updated Branches: refs/heads/branch-4.0 cfba5e905 -> b2858ab6d
OOZIE-1670 Workflow kill command doesn't kill child job for map-reduce action (puru via rohini) Project: http://git-wip-us.apache.org/repos/asf/oozie/repo Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/b2858ab6 Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/b2858ab6 Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/b2858ab6 Branch: refs/heads/branch-4.0 Commit: b2858ab6d41bbe65e65148ce15d330dd37913af1 Parents: cfba5e9 Author: Rohini Palaniswamy <[email protected]> Authored: Fri Jan 17 10:50:42 2014 -0800 Committer: Rohini Palaniswamy <[email protected]> Committed: Fri Jan 17 10:50:42 2014 -0800 ---------------------------------------------------------------------- .../oozie/action/hadoop/JavaActionExecutor.java | 2 +- .../wf/TestWorkflowActionKillXCommand.java | 42 ++++++++++++++++---- release-log.txt | 1 + 3 files changed, 37 insertions(+), 8 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/oozie/blob/b2858ab6/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java index 3a64793..133d6a2 100644 --- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java +++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java @@ -1152,7 +1152,7 @@ public class JavaActionExecutor extends ActionExecutor { Element actionXml = XmlUtils.parseXml(action.getConf()); JobConf jobConf = createBaseHadoopConf(context, actionXml); jobClient = createJobClient(context, jobConf); - RunningJob runningJob = jobClient.getJob(JobID.forName(action.getExternalId())); + RunningJob runningJob = getRunningJob(context, action, jobClient); if (runningJob != null) { runningJob.killJob(); } http://git-wip-us.apache.org/repos/asf/oozie/blob/b2858ab6/core/src/test/java/org/apache/oozie/command/wf/TestWorkflowActionKillXCommand.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/oozie/command/wf/TestWorkflowActionKillXCommand.java b/core/src/test/java/org/apache/oozie/command/wf/TestWorkflowActionKillXCommand.java index da89f4b..a25a4dd 100644 --- a/core/src/test/java/org/apache/oozie/command/wf/TestWorkflowActionKillXCommand.java +++ b/core/src/test/java/org/apache/oozie/command/wf/TestWorkflowActionKillXCommand.java @@ -17,12 +17,16 @@ */ package org.apache.oozie.command.wf; +import java.io.StringReader; import java.net.URI; import java.util.Date; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.examples.SleepJob; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobID; +import org.apache.hadoop.mapred.JobStatus; import org.apache.hadoop.mapred.RunningJob; import org.apache.oozie.WorkflowActionBean; import org.apache.oozie.WorkflowJobBean; @@ -37,6 +41,9 @@ import org.apache.oozie.service.JPAService; import org.apache.oozie.service.Services; import org.apache.oozie.service.UUIDService; import org.apache.oozie.test.XDataTestCase; +import org.apache.oozie.test.XTestCase.Predicate; +import org.apache.oozie.util.XConfiguration; +import org.apache.oozie.util.XmlUtils; import org.apache.oozie.workflow.WorkflowInstance; public class TestWorkflowActionKillXCommand extends XDataTestCase { @@ -62,10 +69,10 @@ public class TestWorkflowActionKillXCommand extends XDataTestCase { * @throws Exception */ public void testWfActionKillSuccess() throws Exception { - String externalJobID = launchSleepJob(); + String externalJobID = launchSleepJob(1000); WorkflowJobBean job = this.addRecordToWfJobTable(WorkflowJob.Status.KILLED, WorkflowInstance.Status.KILLED); WorkflowActionBean action = this.addRecordToWfActionTable(job.getId(), externalJobID, "1", - WorkflowAction.Status.KILLED); + WorkflowAction.Status.KILLED,null); JPAService jpaService = Services.get().get(JPAService.class); assertNotNull(jpaService); @@ -89,10 +96,10 @@ public class TestWorkflowActionKillXCommand extends XDataTestCase { * @throws Exception */ public void testWfActionKillFailed() throws Exception { - String externalJobID = launchSleepJob(); + String externalJobID = launchSleepJob(1000); WorkflowJobBean job = this.addRecordToWfJobTable(WorkflowJob.Status.RUNNING, WorkflowInstance.Status.RUNNING); WorkflowActionBean action = this.addRecordToWfActionTable(job.getId(), externalJobID, "1", - WorkflowAction.Status.RUNNING); + WorkflowAction.Status.RUNNING,null); JPAService jpaService = Services.get().get(JPAService.class); assertNotNull(jpaService); @@ -110,8 +117,28 @@ public class TestWorkflowActionKillXCommand extends XDataTestCase { assertEquals(action.getExternalStatus(), "RUNNING"); } + public void testWfActionKillChildJob() throws Exception { + String externalJobID = launchSleepJob(1000); + String childId = launchSleepJob(1000000); + + WorkflowJobBean job = this.addRecordToWfJobTable(WorkflowJob.Status.KILLED, WorkflowInstance.Status.KILLED); + WorkflowActionBean action = this.addRecordToWfActionTable(job.getId(), externalJobID, "1", + WorkflowAction.Status.KILLED, childId); + + new ActionKillXCommand(action.getId()).call(); + JobClient jobClient = createJobClient(); + + final RunningJob mrJob = jobClient.getJob(JobID.forName(childId)); + waitFor(60 * 1000, new Predicate() { + public boolean evaluate() throws Exception { + return mrJob.isComplete(); + } + }); + assertEquals(mrJob.getJobState(), JobStatus.KILLED); + } + protected WorkflowActionBean addRecordToWfActionTable(String wfId, String externalJobID, String actionName, - WorkflowAction.Status status) throws Exception { + WorkflowAction.Status status, String childID) throws Exception { WorkflowActionBean action = new WorkflowActionBean(); action.setId(Services.get().get(UUIDService.class).generateChildId(wfId, actionName)); action.setJobId(wfId); @@ -124,6 +151,7 @@ public class TestWorkflowActionKillXCommand extends XDataTestCase { action.setPending(); action.setExternalId(externalJobID); action.setExternalStatus("RUNNING"); + action.setExternalChildIDs(childID); String actionXml = "<map-reduce>" + "<job-tracker>" + getJobTrackerUri() + "</job-tracker>" + @@ -153,14 +181,14 @@ public class TestWorkflowActionKillXCommand extends XDataTestCase { return action; } - private String launchSleepJob() throws Exception { + private String launchSleepJob(int sleep) throws Exception { JobConf jobConf = Services.get().get(HadoopAccessorService.class) .createJobConf(new URI(getNameNodeUri()).getAuthority()); JobClient jobClient = createJobClient(); SleepJob sleepjob = new SleepJob(); sleepjob.setConf(jobConf); - jobConf = sleepjob.setupJobConf(1, 1, 1000, 1, 1000, 1); + jobConf = sleepjob.setupJobConf(1, 1, sleep, 1, sleep, 1); final RunningJob runningJob = jobClient.submitJob(jobConf); return runningJob.getID().toString(); http://git-wip-us.apache.org/repos/asf/oozie/blob/b2858ab6/release-log.txt ---------------------------------------------------------------------- diff --git a/release-log.txt b/release-log.txt index 65285c9..ce706c3 100644 --- a/release-log.txt +++ b/release-log.txt @@ -1,5 +1,6 @@ -- Oozie 4.0.1 release (unreleased) +OOZIE-1670 Workflow kill command doesn't kill child job for map-reduce action (puru via rohini) OOZIE-1630 <prepare> operations fail when path doesn't have scheme (ryota) OOZIE-1627 Rerun doesn't resolve workflow app name (puru via rohini) OOZIE-1626 pig action pop-up is not working properly in UI (ryota, puru via rohini)
