You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by rk...@apache.org on 2013/02/04 19:03:47 UTC

svn commit: r1442272 - in /oozie/branches/branch-3.3: core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java release-log.txt

Author: rkanter
Date: Mon Feb  4 18:03:46 2013
New Revision: 1442272

URL: http://svn.apache.org/viewvc?rev=1442272&view=rev
Log:
OOZIE-945 BundleSubmitXCommand.submit() doesn't properly remove comments (jaoki via rkanter)

Added:
    oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java
Modified:
    oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java
    oozie/branches/branch-3.3/release-log.txt

Modified: oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java?rev=1442272&r1=1442271&r2=1442272&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java (original)
+++ oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java Mon Feb  4 18:03:46 2013
@@ -133,7 +133,7 @@ public class BundleSubmitXCommand extend
 
             ParameterVerifier.verifyParameters(conf, XmlUtils.parseXml(bundleBean.getOrigJobXml()));
             
-            XmlUtils.removeComments(this.bundleBean.getOrigJobXml().toString());
+            String jobXmlWithNoComment = XmlUtils.removeComments(this.bundleBean.getOrigJobXml().toString());
             // Resolving all variables in the job properties.
             // This ensures the Hadoop Configuration semantics is preserved.
             XConfiguration resolvedVarsConf = new XConfiguration();
@@ -142,7 +142,7 @@ public class BundleSubmitXCommand extend
             }
             conf = resolvedVarsConf;
 
-            String resolvedJobXml = resolvedVars(bundleBean.getOrigJobXml(), conf);
+            String resolvedJobXml = resolvedVars(jobXmlWithNoComment, conf);
 
             //verify the uniqueness of coord names
             verifyCoordNameUnique(resolvedJobXml);

Added: oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java?rev=1442272&view=auto
==============================================================================
--- oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java (added)
+++ oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java Mon Feb  4 18:03:46 2013
@@ -0,0 +1,92 @@
+/**
+ * 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.bundle;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.Date;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.oozie.BundleJobBean;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.client.Job;
+import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.command.CommandException;
+import org.apache.oozie.service.JPAService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.util.XConfiguration;
+
+public class TestBundleSubmitXCommand extends XDataTestCase {
+    
+    private Services services;
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+        services = new Services();
+        services.init();
+        cleanUpDBTables();
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        services.destroy();
+        super.tearDown();
+    }
+
+    /**
+     * https://issues.apache.org/jira/browse/OOZIE-945
+     * 
+     * @throws Exception
+     */
+    public void testJobXmlCommentRemoved() throws Exception {
+        // this retrieves bundle-submit-job.xml
+        BundleJobBean job = this.addRecordToBundleJobTable(Job.Status.PREP, false);
+
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        Configuration jobConf = null;
+        try {
+            jobConf = new XConfiguration(new StringReader(job.getConf()));
+        }
+        catch (IOException ioe) {
+            log.warn("Configuration parse error. read from DB :" + job.getConf(), ioe);
+            throw new CommandException(ErrorCode.E1005, ioe);
+        }
+
+        Path appPath = new Path(jobConf.get(OozieClient.BUNDLE_APP_PATH), "bundle.xml");
+        jobConf.set(OozieClient.BUNDLE_APP_PATH, appPath.toString());
+
+        BundleSubmitXCommand command = new BundleSubmitXCommand(true, jobConf, CREATE_TIME);
+        BundleJobBean bundleBean = (BundleJobBean)command.getJob();
+        bundleBean.setStartTime(new Date());
+        bundleBean.setEndTime(new Date());
+        command.call();
+
+        // result includes bundle-submit-job.xml file instead of jobId since this is a dryRun mode
+        String result = command.submit();
+        // bundle-submit-job.xml contains the Apache license but this result should not contain the comment block
+        assertTrue("submit result should not contain <!-- ", !result.contains("<!--"));
+        assertTrue("submit result should not contain --> ", !result.contains("-->"));
+        
+    }
+
+}

Modified: oozie/branches/branch-3.3/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/release-log.txt?rev=1442272&r1=1442271&r2=1442272&view=diff
==============================================================================
--- oozie/branches/branch-3.3/release-log.txt (original)
+++ oozie/branches/branch-3.3/release-log.txt Mon Feb  4 18:03:46 2013
@@ -1,5 +1,6 @@
 -- Oozie 3.3.2 (unreleased)
 
+OOZIE-945 BundleSubmitXCommand.submit() doesn't properly remove comments (jaoki via rkanter)
 OOZIE-1171 HostnameFilter should handle hostname resolution failures and continue processing (tucu via rkanter)
 OOZIE-1053 Oozie Web-console clicking on Bundle's coord jobs does not open them up (ryota via mona)
 OOZIE-1166 Print a more helpful message when ProxyUserService is configured wrong (rkanter)