You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by pu...@apache.org on 2015/04/09 01:30:51 UTC

oozie git commit: OOZIE-2139 Coord update doesn't work for job which is submitted by bundle

Repository: oozie
Updated Branches:
  refs/heads/master 3fb549f3a -> f7f7628b8


OOZIE-2139 Coord update doesn't work for job which is submitted by bundle


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

Branch: refs/heads/master
Commit: f7f7628b8c7a681461958541fd247c77169aef13
Parents: 3fb549f
Author: Purshotam Shah <pu...@yahoo-inc.com>
Authored: Wed Apr 8 16:30:44 2015 -0700
Committer: Purshotam Shah <pu...@yahoo-inc.com>
Committed: Wed Apr 8 16:30:44 2015 -0700

----------------------------------------------------------------------
 .../command/coord/CoordUpdateXCommand.java      |  23 +++-
 .../apache/oozie/servlet/BaseJobServlet.java    |   9 ++
 .../command/coord/TestCoordUpdateXCommand.java  | 109 ++++++++++++++++++-
 core/src/test/resources/bundle-submit-job.xml   |   4 +
 release-log.txt                                 |   1 +
 5 files changed, 141 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/f7f7628b/core/src/main/java/org/apache/oozie/command/coord/CoordUpdateXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordUpdateXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordUpdateXCommand.java
index 6647ef8..92ad4f4 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordUpdateXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordUpdateXCommand.java
@@ -23,6 +23,8 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.StringReader;
 import java.util.Date;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.CoordinatorJobBean;
 import org.apache.oozie.ErrorCode;
@@ -54,6 +56,9 @@ public class CoordUpdateXCommand extends CoordSubmitXCommand {
     private boolean showDiff = true;
     private boolean isConfChange = false;
 
+    //This properties are set in coord jobs by bundle. An update command should not overide it.
+    final static String[] bundleConfList = new String[] { OozieClient.BUNDLE_ID };
+
     StringBuffer diff = new StringBuffer();
     CoordinatorJobBean oldCoordJob = null;
 
@@ -116,9 +121,23 @@ public class CoordUpdateXCommand extends CoordSubmitXCommand {
         }
 
         LogUtils.setLogInfo(oldCoordJob);
-        if (!isConfChange) {
+        if (!isConfChange || StringUtils.isEmpty(conf.get(OozieClient.COORDINATOR_APP_PATH))) {
             try {
-                conf = new XConfiguration(new StringReader(oldCoordJob.getConf()));
+                XConfiguration jobConf = new XConfiguration(new StringReader(oldCoordJob.getConf()));
+
+                if (!isConfChange) {
+                    conf = jobConf;
+                }
+                else {
+                    for (String bundleConfKey : bundleConfList) {
+                        if (jobConf.get(bundleConfKey) != null) {
+                            conf.set(bundleConfKey, jobConf.get(bundleConfKey));
+                        }
+                    }
+                    if (StringUtils.isEmpty(conf.get(OozieClient.COORDINATOR_APP_PATH))) {
+                        conf.set(OozieClient.COORDINATOR_APP_PATH, jobConf.get(OozieClient.COORDINATOR_APP_PATH));
+                    }
+                }
             }
             catch (Exception e) {
                 throw new CommandException(ErrorCode.E1023, e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/oozie/blob/f7f7628b/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 a581f8b..bd9e998 100644
--- a/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
@@ -170,8 +170,17 @@ public abstract class BaseJobServlet extends JsonRestServlet {
                 conf.set(OozieClient.USER_NAME, requestUser);
             }
             if (conf.get(OozieClient.COORDINATOR_APP_PATH) != null) {
+                //If coord is submitted from bundle, user may want to update individual coord job with bundle properties
+                //If COORDINATOR_APP_PATH is set, we should check only COORDINATOR_APP_PATH path permission
+                String bundlePath = conf.get(OozieClient.BUNDLE_APP_PATH);
+                if (bundlePath != null) {
+                    conf.unset(OozieClient.BUNDLE_APP_PATH);
+                }
                 BaseJobServlet.checkAuthorizationForApp(conf);
                 JobUtils.normalizeAppPath(conf.get(OozieClient.USER_NAME), conf.get(OozieClient.GROUP_NAME), conf);
+                if (bundlePath != null) {
+                    conf.set(OozieClient.BUNDLE_APP_PATH, bundlePath);
+                }
             }
             JSONObject json = updateJob(request, response, conf);
             startCron();

http://git-wip-us.apache.org/repos/asf/oozie/blob/f7f7628b/core/src/test/java/org/apache/oozie/command/coord/TestCoordUpdateXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/coord/TestCoordUpdateXCommand.java b/core/src/test/java/org/apache/oozie/command/coord/TestCoordUpdateXCommand.java
index 89b81d5..de16d92 100644
--- a/core/src/test/java/org/apache/oozie/command/coord/TestCoordUpdateXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/coord/TestCoordUpdateXCommand.java
@@ -23,18 +23,29 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
+import java.io.UnsupportedEncodingException;
 import java.io.Writer;
+import java.util.Date;
+import java.util.List;
+import java.util.regex.Matcher;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.oozie.BundleActionBean;
+import org.apache.oozie.BundleJobBean;
 import org.apache.oozie.CoordinatorJobBean;
-import org.apache.oozie.ErrorCode;
 import org.apache.oozie.XException;
 import org.apache.oozie.client.CoordinatorAction;
+import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.rest.RestConstants;
 import org.apache.oozie.command.CommandException;
+import org.apache.oozie.command.bundle.BundleStartXCommand;
+import org.apache.oozie.command.bundle.BundleSubmitXCommand;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor.BundleActionQuery;
 import org.apache.oozie.local.LocalOozie;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
@@ -278,13 +289,105 @@ public class TestCoordUpdateXCommand extends XDataTestCase {
                 .getChildren("instance", namespace).get(0)).getText();
         assertEquals(text, "${coord:future(0, 1)}");
         new CoordActionsKillXCommand(jobId, RestConstants.JOB_COORD_SCOPE_ACTION, Integer.toString(actionNum)).call();
-        coordClient.reRunCoord(jobId, RestConstants.JOB_COORD_SCOPE_ACTION, Integer.toString(actionNum), true,
-                true);
+        coordClient.reRunCoord(jobId, RestConstants.JOB_COORD_SCOPE_ACTION, Integer.toString(actionNum), true, true);
         bean = coordClient.getCoordActionInfo(actionId);
         sleep(1000);
         assertEquals(bean.getMissingDependencies(), "!!${coord:future(0, 1)}");
     }
 
+    public void testCoordFromBundleJobChangeConf() throws Exception {
+        final XConfiguration jobConf = new XConfiguration();
+        String coordJobId = setUpBundleAndGetCoordID(jobConf);
+
+        jobConf.set("newvalue", "yes");
+        CoordUpdateXCommand update = new CoordUpdateXCommand(false, jobConf, coordJobId);
+        update.call();
+        CoordinatorJobBean job = getCoordJobs(coordJobId);
+        Configuration xConf = new XConfiguration(new StringReader(job.getConf()));
+        assertEquals(xConf.get("newvalue"), "yes");
+        /*
+         * testProperty is part of bundle.xml <property> <name>testProperty</name> <value>abc</value> </property>
+         */
+        jobConf.set("testProperty", "xyz");
+        new CoordUpdateXCommand(false, jobConf, coordJobId).call();
+        job = getCoordJobs(coordJobId);
+        xConf = new XConfiguration(new StringReader(job.getConf()));
+        assertEquals(xConf.get("testProperty"), "xyz");
+
+    }
+
+    public void testCoordFromBundleJobChangeDefinition() throws Exception {
+        final XConfiguration jobConf = new XConfiguration();
+        String coordJobId = setUpBundleAndGetCoordID(jobConf);
+
+        CoordinatorJobBean job = getCoordJobs(coordJobId);
+        Element processedJobXml = XmlUtils.parseXml(job.getJobXml());
+        Namespace namespace = processedJobXml.getNamespace();
+        String text = ((Element) processedJobXml.getChild("input-events", namespace).getChild("data-in", namespace)
+                .getChildren("instance", namespace).get(0)).getText();
+        assertEquals(text, "${coord:latest(0)}");
+
+        final Path coordPath1 = new Path(getFsTestCaseDir(), "coord1");
+        writeCoordXml(coordPath1, "coord-multiple-input-instance4.xml");
+        Configuration newConf = new Configuration();
+        newConf.set(OozieClient.USER_NAME, getTestUser());
+        CoordUpdateXCommand update = new CoordUpdateXCommand(false, newConf, coordJobId);
+        update.call();
+        job = getCoordJobs(coordJobId);
+        processedJobXml = XmlUtils.parseXml(job.getJobXml());
+        namespace = processedJobXml.getNamespace();
+        text = ((Element) processedJobXml.getChild("input-events", namespace).getChild("data-in", namespace)
+                .getChildren("instance", namespace).get(0)).getText();
+        assertEquals(text, "${coord:future(0, 1)}");
+    }
+
+    private String setUpBundleAndGetCoordID(XConfiguration jobConf) throws UnsupportedEncodingException, IOException,
+            CommandException, JPAExecutorException {
+
+        final Path coordPath1 = new Path(getFsTestCaseDir(), "coord1");
+        final Path coordPath2 = new Path(getFsTestCaseDir(), "coord2");
+        writeCoordXml(coordPath1, "coord-multiple-input-instance3.xml");
+        writeCoordXml(coordPath2, "coord-multiple-input-instance3.xml");
+
+        Path bundleAppPath = new Path(getFsTestCaseDir(), "bundle");
+        String bundleAppXml = getBundleXml("bundle-submit-job.xml");
+        assertNotNull(bundleAppXml);
+        assertTrue(bundleAppXml.length() > 0);
+
+        bundleAppXml = bundleAppXml.replaceAll("#app_path1",
+                Matcher.quoteReplacement(new Path(coordPath1.toString(), "coordinator.xml").toString()));
+        bundleAppXml = bundleAppXml.replaceAll("#app_path2",
+                Matcher.quoteReplacement(new Path(coordPath2.toString(), "coordinator.xml").toString()));
+
+        writeToFile(bundleAppXml, bundleAppPath, "bundle.xml");
+        final Path appPath = new Path(bundleAppPath, "bundle.xml");
+        jobConf.set(OozieClient.BUNDLE_APP_PATH, appPath.toString());
+        jobConf.set("appName", "test");
+
+        jobConf.set(OozieClient.USER_NAME, getTestUser());
+
+        jobConf.set("coordName1", "NAME");
+        jobConf.set("coordName2", "coord2");
+        BundleSubmitXCommand command = new BundleSubmitXCommand(jobConf);
+        final BundleJobBean bundleBean = (BundleJobBean) command.getJob();
+        bundleBean.setStartTime(new Date());
+        bundleBean.setEndTime(new Date());
+        final String jobId = command.call();
+        sleep(2000);
+        new BundleStartXCommand(jobId).call();
+        waitFor(200000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                        BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, jobId);
+                return actions.get(0).getStatus().equals(Job.Status.RUNNING);
+            }
+        });
+
+        final List<BundleActionBean> actions = BundleActionQueryExecutor.getInstance().getList(
+                BundleActionQuery.GET_BUNDLE_ACTIONS_STATUS_UNIGNORED_FOR_BUNDLE, jobId);
+        return actions.get(0).getCoordId();
+    }
+
     private CoordinatorJobBean getCoordJobs(String jobId) {
         try {
             JPAService jpaService = Services.get().get(JPAService.class);

http://git-wip-us.apache.org/repos/asf/oozie/blob/f7f7628b/core/src/test/resources/bundle-submit-job.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/bundle-submit-job.xml b/core/src/test/resources/bundle-submit-job.xml
index a9599ef..6eda402 100644
--- a/core/src/test/resources/bundle-submit-job.xml
+++ b/core/src/test/resources/bundle-submit-job.xml
@@ -34,6 +34,10 @@
                          <name>END_TIME</name>
                          <value>2009-02-20T23:59Z</value>
                      </property>
+                     <property>
+                         <name>testProperty</name>
+                         <value>abc</value>
+                     </property>
                  </configuration>
           </coordinator>
           <coordinator name='${coordName2}' critical='false'>

http://git-wip-us.apache.org/repos/asf/oozie/blob/f7f7628b/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index ae581ac..c7e0900 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.2.0 release (trunk - unreleased)
 
+OOZIE-2139 Coord update doesn't work for job which is submitted by bundle (puru)
 OOZIE-1726 Oozie does not support _HOST when configuring kerberos security (venkatnrangan via bzhang)
 OOZIE-2197 ooziedb.cmd command failed due to classpath being too long on windows (me.venkatr via bzhang)
 OOZIE-2182 SLA alert commands are not in sync with doc (puru)