You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by an...@apache.org on 2012/01/06 22:58:40 UTC

svn commit: r1228455 - in /incubator/oozie/branches/3.1: ./ core/src/main/java/org/apache/oozie/ core/src/main/java/org/apache/oozie/command/wf/ core/src/test/java/org/apache/oozie/command/wf/

Author: angeloh
Date: Fri Jan  6 21:58:39 2012
New Revision: 1228455

URL: http://svn.apache.org/viewvc?rev=1228455&view=rev
Log:
OOZIE-580 use xml element to handle string escape when configure evaluator

Modified:
    incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/DagELFunctions.java
    incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java
    incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java
    incubator/oozie/branches/3.1/release-log.txt

Modified: incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/DagELFunctions.java
URL: http://svn.apache.org/viewvc/incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/DagELFunctions.java?rev=1228455&r1=1228454&r2=1228455&view=diff
==============================================================================
--- incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/DagELFunctions.java (original)
+++ incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/DagELFunctions.java Fri Jan  6 21:58:39 2012
@@ -6,9 +6,9 @@
  * 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.
@@ -27,7 +27,7 @@ import org.apache.oozie.util.PropertiesU
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.util.ParamChecker;
 import org.apache.oozie.util.XmlUtils;
-
+import org.jdom.JDOMException;
 import java.io.IOException;
 import java.io.StringReader;
 import java.util.Properties;
@@ -54,11 +54,19 @@ public class DagELFunctions {
     public static void configureEvaluator(ELEvaluator evaluator, WorkflowJobBean workflow, WorkflowActionBean action) {
         evaluator.setVariable(WORKFLOW, workflow);
         evaluator.setVariable(ACTION, action);
+
         for (Map.Entry<String, String> entry : workflow.getWorkflowInstance().getConf()) {
             if (ParamChecker.isValidIdentifier(entry.getKey())) {
                 String value = entry.getValue().trim();
-                // escape the characters for xml
-                value = XmlUtils.escapeCharsForXML(value);
+                try {
+                    String valueElem = "<value>"+value+"</value>";
+                    XmlUtils.parseXml(valueElem);
+                }
+                catch (JDOMException ex) {
+                    // If happens, try escaping the characters for XML. The escaping may or
+                    // may not solve the problem since the JDOMException could be for a range of issues.
+                    value = XmlUtils.escapeCharsForXML(value);
+                }
                 evaluator.setVariable(entry.getKey().trim(), value);
             }
         }

Modified: incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java?rev=1228455&r1=1228454&r2=1228455&view=diff
==============================================================================
--- incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java (original)
+++ incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/command/wf/ActionStartXCommand.java Fri Jan  6 21:58:39 2012
@@ -6,9 +6,9 @@
  * 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.
@@ -20,7 +20,6 @@ package org.apache.oozie.command.wf;
 import java.util.Date;
 
 import javax.servlet.jsp.el.ELException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.FaultInjection;
@@ -157,10 +156,8 @@ public class ActionStartXCommand extends
                 String tmpActionConf = XmlUtils.removeComments(wfAction.getConf());
                 String actionConf = context.getELEvaluator().evaluate(tmpActionConf, String.class);
                 wfAction.setConf(actionConf);
-
                 LOG.debug("Start, name [{0}] type [{1}] configuration{E}{E}{2}{E}", wfAction.getName(), wfAction
                         .getType(), actionConf);
-
             }
             catch (ELEvaluationException ex) {
                 throw new ActionExecutorException(ActionExecutorException.ErrorType.TRANSIENT, EL_EVAL_ERROR, ex

Modified: incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java?rev=1228455&r1=1228454&r2=1228455&view=diff
==============================================================================
--- incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java (original)
+++ incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java Fri Jan  6 21:58:39 2012
@@ -6,9 +6,9 @@
  * 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.
@@ -194,6 +194,60 @@ public class TestActionStartXCommand ext
         assertFalse(workDir.contains("workflow.xml"));
     }
 
+    /**
+     * Test workflow action with CDATA section and string configuration that needs to be escaped can be submitted without
+     * throwing exceptions.
+     * <p/>
+     * Escaped string needs to be 'escaped' before converting to XML Document, otherwise,
+     * exception will be thrown.
+     * @see org.apache.oozie.DagELFunctions#configureEvaluator(org.apache.oozie.util.ELEvaluator.ELEvaluator evaluator, org.apache.oozie.WorkflowJobBean, org.apache.oozie.WorkflowActionBean)
+     *
+     * @throws Exception thrown if failed to execute test case
+     */
+    public void testActionWithEscapedStringAndCDATA() throws Exception {
+        // create workflow job and action beans with escaped parameters and CDATA value
+        JPAService jpaService = Services.get().get(JPAService.class);
+        WorkflowJobBean job = this.addRecordToWfJobTableWithEscapedStringAndCDATA(WorkflowJob.Status.RUNNING, WorkflowInstance.Status.RUNNING);
+        WorkflowActionBean action = this.addRecordToWfActionTableWithEscapedStringAndCDATA(job.getId(), WorkflowAction.Status.PREP);
+        WorkflowActionGetJPAExecutor wfActionGetCmd = new WorkflowActionGetJPAExecutor(action.getId());
+
+        // start workflow action
+        new ActionStartXCommand(action.getId(), "map-reduce").call();
+        action = jpaService.execute(wfActionGetCmd);
+        assertNotNull(action.getExternalId());
+
+        ActionExecutorContext context = new ActionXCommand.ActionExecutorContext(job, action, false, false);
+        MapReduceActionExecutor actionExecutor = new MapReduceActionExecutor();
+        Configuration conf = actionExecutor.createBaseHadoopConf(context, XmlUtils.parseXml(action.getConf()));
+        String user = conf.get("user.name");
+        String group = conf.get("group.name");
+        JobClient jobClient = Services.get().get(HadoopAccessorService.class).createJobClient(user, group,
+                new JobConf(conf));
+
+        String launcherId = action.getExternalId();
+
+        // retrieve launcher job
+        final RunningJob launcherJob = jobClient.getJob(JobID.forName(launcherId));
+
+        // time out after 120 seconds unless launcher job succeeds
+        waitFor(120 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                return launcherJob.isComplete();
+            }
+        });
+        // check if launcher job succeeds
+        assertTrue(launcherJob.isSuccessful());
+        assertTrue(LauncherMapper.hasIdSwap(launcherJob));
+    }
+
+    /**
+     * Create workflow job with custom application path
+     *
+     * @param jobStatus workflow job status
+     * @param instanceStatus workflow instance status
+     * @return workflow job bean
+     * @throws Exception thrown if failed to create workflow job
+     */
     protected WorkflowJobBean addRecordToWfJobTableWithCustomAppPath(WorkflowJob.Status jobStatus, WorkflowInstance.Status instanceStatus)
     throws Exception {
         WorkflowApp app = new LiteWorkflowApp("testApp", "<workflow-app/>", new StartNodeDef("end"))
@@ -221,6 +275,15 @@ public class TestActionStartXCommand ext
         return wfBean;
     }
 
+    /**
+     * Create workflow action with custom action configuration
+     *
+     * @param wfId workflow id
+     * @param actionName action name
+     * @param status workflow action status
+     * @return workflow action bean
+     * @throws Exception thrown if failed to create workflow action
+     */
     protected WorkflowActionBean addRecordToWfActionTableWithAppPathConfig(String wfId, String actionName, WorkflowAction.Status status)
             throws Exception {
         WorkflowActionBean action = createWorkflowActionWithAppPathConfig(wfId, status);
@@ -238,6 +301,14 @@ public class TestActionStartXCommand ext
         return action;
     }
 
+    /**
+     * Create workflow action with custom action configuration
+     *
+     * @param wfId workflow id
+     * @param status workflow action status
+     * @return workflow action bean
+     * @throws Exception thrown if failed to create workflow action
+     */
     protected WorkflowActionBean createWorkflowActionWithAppPathConfig(String wfId, WorkflowAction.Status status)
     throws Exception {
         WorkflowActionBean action = new WorkflowActionBean();
@@ -279,6 +350,9 @@ public class TestActionStartXCommand ext
     }
 
 
+    /* (non-Javadoc)
+     * @see org.apache.oozie.test.XDataTestCase#addRecordToWfActionTable(java.lang.String, java.lang.String, org.apache.oozie.client.WorkflowAction.Status)
+     */
     @Override
     protected WorkflowActionBean addRecordToWfActionTable(String wfId, String actionName, WorkflowAction.Status status)
             throws Exception {
@@ -297,6 +371,14 @@ public class TestActionStartXCommand ext
         return action;
     }
 
+    /**
+     * Create workflow action with pending true
+     *
+     * @param wfId workflow id
+     * @param status workflow action status
+     * @return workflow action bean
+     * @throws Exception thrown if failed to create workflow action
+     */
     protected WorkflowActionBean createWorkflowActionSetPending(String wfId, WorkflowAction.Status status)
             throws Exception {
         WorkflowActionBean action = new WorkflowActionBean();
@@ -335,4 +417,115 @@ public class TestActionStartXCommand ext
         return action;
     }
 
+    /**
+     * Create workflow job with action configuration with CDATA section and escaped string as value in parameter.
+     *
+     * @param jobStatus workflow job status
+     * @param instanceStatus workflow instance status
+     * @return workflow job bean
+     * @throws Exception thrown if failed to create workflow job
+     */
+    private WorkflowJobBean addRecordToWfJobTableWithEscapedStringAndCDATA(WorkflowJob.Status jobStatus,
+            WorkflowInstance.Status instanceStatus) throws Exception {
+        WorkflowApp app = new LiteWorkflowApp("testApp", "<workflow-app/>", new StartNodeDef("end"))
+                .addNode(new EndNodeDef("end"));
+        Configuration conf = new Configuration();
+        Path appUri = new Path(getAppPath(), "workflow.xml");
+        conf.set(OozieClient.APP_PATH, appUri.toString());
+        conf.set(OozieClient.LOG_TOKEN, "testToken");
+        conf.set(OozieClient.USER_NAME, getTestUser());
+        conf.set(OozieClient.GROUP_NAME, getTestGroup());
+        // The properties should not be escaped here. It will be escaped when set to configuration.
+        conf.set("testAmpSign", "http://test.apache.com/a-webservices?urlSigner=signUrl&namespace=nova.proxy");
+        conf.set("testCDATA",
+                        "<![CDATA[?redirect=http%3A%2F%2Ftest.apache.com%2Fa-webservices%2Fv1%2FurlSigner%2FsignUrl&amp;namespace=nova.proxy&amp;keyDBHash=Vsy6n_C7K6NG0z4R2eBlKg--]]>");
+        injectKerberosInfo(conf);
+        WorkflowJobBean wfBean = createWorkflow(app, conf, "auth", jobStatus, instanceStatus);
+
+        try {
+            JPAService jpaService = Services.get().get(JPAService.class);
+            assertNotNull(jpaService);
+            WorkflowJobInsertJPAExecutor wfInsertCmd = new WorkflowJobInsertJPAExecutor(wfBean);
+            jpaService.execute(wfInsertCmd);
+        }
+        catch (JPAExecutorException je) {
+            je.printStackTrace();
+            fail("Unable to insert the test wf job record to table");
+        }
+        return wfBean;
+    }
+
+    /**
+     * Create workflow action with action configuration with CDATA section and escaped string as value in parameter.
+     *
+     * @param wfId workflow job id
+     * @param status action status
+     * @return workflow action bean
+     * @throws Exception thrown if failed to create workflow action
+     */
+    private WorkflowActionBean addRecordToWfActionTableWithEscapedStringAndCDATA(String wfId,
+            WorkflowAction.Status status) throws Exception {
+        WorkflowActionBean action = createWorkflowActionSetPendingWithEscapedStringAndCDATA(wfId, status);
+        try {
+            JPAService jpaService = Services.get().get(JPAService.class);
+            assertNotNull(jpaService);
+            WorkflowActionInsertJPAExecutor actionInsertCmd = new WorkflowActionInsertJPAExecutor(action);
+            jpaService.execute(actionInsertCmd);
+        }
+        catch (JPAExecutorException ce) {
+            ce.printStackTrace();
+            fail("Unable to insert the test wf action record to table");
+        }
+        return action;
+    }
+
+    /**
+     * Create workflow action with action configuration with CDATA section and escaped string as value in parameter.
+     *
+     * @param wfId workflow job id
+     * @param status action status
+     * @return workflow action bean
+     * @throws Exception thrown if failed to create workflow action
+     */
+    private WorkflowActionBean createWorkflowActionSetPendingWithEscapedStringAndCDATA(String wfId,
+            WorkflowAction.Status status) throws Exception {
+        WorkflowActionBean action = new WorkflowActionBean();
+        String actionname = "testAction";
+        action.setName(actionname);
+        action.setCred("null");
+        action.setId(Services.get().get(UUIDService.class).generateChildId(wfId, actionname));
+        action.setJobId(wfId);
+        action.setType("map-reduce");
+        action.setTransition("transition");
+        action.setStatus(status);
+        action.setStartTime(new Date());
+        action.setEndTime(new Date());
+        action.setLastCheckTime(new Date());
+        action.setPending();
+
+        Path inputDir = new Path(getFsTestCaseDir(), "input");
+        Path outputDir = new Path(getFsTestCaseDir(), "output");
+
+        FileSystem fs = getFileSystem();
+        Writer w = new OutputStreamWriter(fs.create(new Path(inputDir, "data.txt")));
+        w.write("dummy\n");
+        w.write("dummy\n");
+        w.close();
+
+        String actionXml = "<map-reduce>" + "<job-tracker>" + getJobTrackerUri() + "</job-tracker>" + "<name-node>"
+                + getNameNodeUri() + "</name-node>" + "<configuration>"
+                + "<property><name>mapred.mapper.class</name><value>" + MapperReducerForTest.class.getName()
+                + "</value></property>" + "<property><name>mapred.reducer.class</name><value>"
+                + MapperReducerForTest.class.getName() + "</value></property>"
+                + "<property><name>mapred.input.dir</name><value>" + inputDir.toString() + "</value></property>"
+                + "<property><name>mapred.output.dir</name><value>" + outputDir.toString() + "</value></property>"
+                + "<property><name>test.ampsign</name><value>${testAmpSign}</value></property>"
+                + "<property><name>test.cdata</name><value>${testCDATA}</value></property>" + "</configuration>"
+                + "</map-reduce>";
+        action.setConf(actionXml);
+
+        return action;
+    }
+
+
 }

Modified: incubator/oozie/branches/3.1/release-log.txt
URL: http://svn.apache.org/viewvc/incubator/oozie/branches/3.1/release-log.txt?rev=1228455&r1=1228454&r2=1228455&view=diff
==============================================================================
--- incubator/oozie/branches/3.1/release-log.txt (original)
+++ incubator/oozie/branches/3.1/release-log.txt Fri Jan  6 21:58:39 2012
@@ -1,5 +1,6 @@
--- Oozie 3.1.1 release
+-- Oozie 3.1.2 release
 
+OOZIE-580 use xml element to handle string escape when configure evaluator
 OOZIE-585 Coordinator job fail to retrieve log with date range and action range.
 OOZIE-553 Ability to view log for coordinator actions that ran in a date range.
 OOZIE-581 Fix unit test failure in TestStatusTransitService.java.