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:02:27 UTC

svn commit: r1442244 - in /oozie/branches/branch-3.3: ./ client/src/main/java/org/apache/oozie/cli/ core/src/main/java/org/apache/oozie/ core/src/main/java/org/apache/oozie/command/wf/ core/src/main/java/org/apache/oozie/servlet/ core/src/test/java/org...

Author: rkanter
Date: Mon Feb  4 18:02:27 2013
New Revision: 1442244

URL: http://svn.apache.org/viewvc?rev=1442244&view=rev
Log:
OOZIE-1080 Add a dryrun option for workflows (rkanter)

Modified:
    oozie/branches/branch-3.3/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
    oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BaseEngine.java
    oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BundleEngine.java
    oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/CoordinatorEngine.java
    oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/DagEngine.java
    oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/wf/SubmitXCommand.java
    oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java
    oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java
    oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java
    oozie/branches/branch-3.3/docs/src/site/twiki/DG_CommandLineTool.twiki
    oozie/branches/branch-3.3/release-log.txt

Modified: oozie/branches/branch-3.3/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/client/src/main/java/org/apache/oozie/cli/OozieCLI.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/client/src/main/java/org/apache/oozie/cli/OozieCLI.java (original)
+++ oozie/branches/branch-3.3/client/src/main/java/org/apache/oozie/cli/OozieCLI.java Mon Feb  4 18:02:27 2013
@@ -242,8 +242,8 @@ public class OozieCLI {
         Option debug = new Option(DEBUG_OPTION, false, "Use debug mode to see debugging statements on stdout");
         Option rerun = new Option(RERUN_OPTION, true,
                 "rerun a job  (coordinator requires -action or -date, bundle requires -coordinator or -date)");
-        Option dryrun = new Option(DRYRUN_OPTION, false,
-                "Supported in Oozie-2.0 or later versions ONLY - dryrun or test run a coordinator job, job is not queued");
+        Option dryrun = new Option(DRYRUN_OPTION, false, "Dryrun a workflow (since 3.4) or coordinator (since 2.0) job without"
+                + " actually executing it");
         Option start = new Option(START_OPTION, true, "start a job");
         Option suspend = new Option(SUSPEND_OPTION, true, "suspend a job");
         Option resume = new Option(RESUME_OPTION, true, "resume a job");
@@ -745,19 +745,24 @@ public class OozieCLI {
                 wc.start(commandLine.getOptionValue(START_OPTION));
             }
             else if (options.contains(DRYRUN_OPTION)) {
-                String[] dryrunStr = wc.dryrun(getConfiguration(wc, commandLine)).split("action for new instance");
-                int arraysize = dryrunStr.length;
-                System.out.println("***coordJob after parsing: ***");
-                System.out.println(dryrunStr[0]);
-                int aLen = dryrunStr.length - 1;
-                if (aLen < 0) {
-                    aLen = 0;
-                }
-                System.out.println("***total coord actions is " + aLen + " ***");
-                for (int i = 1; i <= arraysize - 1; i++) {
-                    System.out.println(RULER);
-                    System.out.println("coordAction instance: " + i + ":");
-                    System.out.println(dryrunStr[i]);
+                String dryrunStr = wc.dryrun(getConfiguration(wc, commandLine));
+                if (dryrunStr.equals("OK")) {  // workflow
+                    System.out.println("OK");
+                } else {                        // coordinator
+                    String[] dryrunStrs = dryrunStr.split("action for new instance");
+                    int arraysize = dryrunStrs.length;
+                    System.out.println("***coordJob after parsing: ***");
+                    System.out.println(dryrunStrs[0]);
+                    int aLen = dryrunStrs.length - 1;
+                    if (aLen < 0) {
+                        aLen = 0;
+                    }
+                    System.out.println("***total coord actions is " + aLen + " ***");
+                    for (int i = 1; i <= arraysize - 1; i++) {
+                        System.out.println(RULER);
+                        System.out.println("coordAction instance: " + i + ":");
+                        System.out.println(dryrunStrs[i]);
+                    }
                 }
             }
             else if (options.contains(SUSPEND_OPTION)) {

Modified: oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BaseEngine.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BaseEngine.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BaseEngine.java (original)
+++ oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BaseEngine.java Mon Feb  4 18:02:27 2013
@@ -182,6 +182,16 @@ public abstract class BaseEngine {
      */
     public abstract String getJobIdForExternalId(String externalId) throws BaseEngineException;
 
-    public abstract String dryrunSubmit(Configuration conf, boolean startJob) throws BaseEngineException;
+    /**
+     * Dry run a job; like {@link BaseEngine#submitJob(org.apache.hadoop.conf.Configuration, boolean) but doesn't actually execute
+     * the job.
+     * <p/>
+     * It validates configuration properties.
+     *
+     * @param conf job configuration.
+     * @return the result of the dryrun
+     * @throws BaseEngineException thrown if there was a problem doing the dryrun
+     */
+    public abstract String dryRunSubmit(Configuration conf) throws BaseEngineException;
 
 }

Modified: oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BundleEngine.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BundleEngine.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BundleEngine.java (original)
+++ oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/BundleEngine.java Mon Feb  4 18:02:27 2013
@@ -88,10 +88,10 @@ public class BundleEngine extends BaseEn
     }
 
     /* (non-Javadoc)
-     * @see org.apache.oozie.BaseEngine#dryrunSubmit(org.apache.hadoop.conf.Configuration, boolean)
+     * @see org.apache.oozie.BaseEngine#dryRunSubmit(org.apache.hadoop.conf.Configuration)
      */
     @Override
-    public String dryrunSubmit(Configuration conf, boolean startJob) throws BundleEngineException {
+    public String dryRunSubmit(Configuration conf) throws BundleEngineException {
         BundleSubmitXCommand submit = new BundleSubmitXCommand(true, conf, getAuthToken());
         try {
             String jobId = submit.call();

Modified: oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/CoordinatorEngine.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/CoordinatorEngine.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/CoordinatorEngine.java (original)
+++ oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/CoordinatorEngine.java Mon Feb  4 18:02:27 2013
@@ -379,11 +379,10 @@ public class CoordinatorEngine extends B
      * (non-Javadoc)
      *
      * @see
-     * org.apache.oozie.BaseEngine#dryrunSubmit(org.apache.hadoop.conf.Configuration
-     * , boolean)
+     * org.apache.oozie.BaseEngine#dryRunSubmit(org.apache.hadoop.conf.Configuration)
      */
     @Override
-    public String dryrunSubmit(Configuration conf, boolean startJob) throws CoordinatorEngineException {
+    public String dryRunSubmit(Configuration conf) throws CoordinatorEngineException {
         try {
 			CoordSubmitXCommand submit = new CoordSubmitXCommand(true, conf,
 					getAuthToken());

Modified: oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/DagEngine.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/DagEngine.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/DagEngine.java (original)
+++ oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/DagEngine.java Mon Feb  4 18:02:27 2013
@@ -479,8 +479,16 @@ public class DagEngine extends BaseEngin
         }
     }
 
+    /* (non-Javadoc)
+     * @see org.apache.oozie.BaseEngine#dryRunSubmit(org.apache.hadoop.conf.Configuration)
+     */
     @Override
-    public String dryrunSubmit(Configuration conf, boolean startJob) throws BaseEngineException {
-        return null;
+    public String dryRunSubmit(Configuration conf) throws BaseEngineException {
+        try {
+            SubmitXCommand submit = new SubmitXCommand(true, conf, getAuthToken());
+            return submit.call();
+        } catch (CommandException ex) {
+            throw new DagEngineException(ex);
+        }
     }
 }

Modified: oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/wf/SubmitXCommand.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/wf/SubmitXCommand.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/wf/SubmitXCommand.java (original)
+++ oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/command/wf/SubmitXCommand.java Mon Feb  4 18:02:27 2013
@@ -75,12 +75,30 @@ public class SubmitXCommand extends Work
     private String authToken;
     private List<JsonBean> insertList = new ArrayList<JsonBean>();
 
+    /**
+     * Constructor to create the workflow Submit Command.
+     *
+     * @param conf : Configuration for workflow job
+     * @param authToken : To be used for authentication
+     */
     public SubmitXCommand(Configuration conf, String authToken) {
         super("submit", "submit", 1);
         this.conf = ParamChecker.notNull(conf, "conf");
         this.authToken = ParamChecker.notEmpty(authToken, "authToken");
     }
 
+    /**
+     * Constructor to create the workflow Submit Command.
+     *
+     * @param dryrun : if dryrun
+     * @param conf : Configuration for workflow job
+     * @param authToken : To be used for authentication
+     */
+    public SubmitXCommand(boolean dryrun, Configuration conf, String authToken) {
+        this(conf, authToken);
+        this.dryrun = dryrun;
+    }
+
     private static final Set<String> DISALLOWED_DEFAULT_PROPERTIES = new HashSet<String>();
     private static final Set<String> DISALLOWED_USER_PROPERTIES = new HashSet<String>();
 
@@ -178,27 +196,32 @@ public class SubmitXCommand extends Work
             Element wfElem = XmlUtils.parseXml(app.getDefinition());
             ELEvaluator evalSla = createELEvaluatorForGroup(conf, "wf-sla-submit");
             String jobSlaXml = verifySlaElements(wfElem, evalSla);
-            writeSLARegistration(jobSlaXml, workflow.getId(), workflow.getUser(), workflow.getGroup(), LOG);
-            workflow.setSlaXml(jobSlaXml);
-            // System.out.println("SlaXml :"+ slaXml);
-
-            //store.insertWorkflow(workflow);
-            insertList.add(workflow);
-            JPAService jpaService = Services.get().get(JPAService.class);
-            if (jpaService != null) {
-                try {
-                    jpaService.execute(new BulkUpdateInsertJPAExecutor(null, insertList));
+            if (!dryrun) {
+                writeSLARegistration(jobSlaXml, workflow.getId(), workflow.getUser(), workflow.getGroup(), LOG);
+                workflow.setSlaXml(jobSlaXml);
+                // System.out.println("SlaXml :"+ slaXml);
+
+                //store.insertWorkflow(workflow);
+                insertList.add(workflow);
+                JPAService jpaService = Services.get().get(JPAService.class);
+                if (jpaService != null) {
+                    try {
+                        jpaService.execute(new BulkUpdateInsertJPAExecutor(null, insertList));
+                    }
+                    catch (JPAExecutorException je) {
+                        throw new CommandException(je);
+                    }
                 }
-                catch (JPAExecutorException je) {
-                    throw new CommandException(je);
+                else {
+                    LOG.error(ErrorCode.E0610);
+                    return null;
                 }
+
+                return workflow.getId();
             }
             else {
-                LOG.error(ErrorCode.E0610);
-                return null;
+                return "OK";
             }
-
-            return workflow.getId();
         }
         catch (WorkflowException ex) {
             throw new CommandException(ex);

Modified: oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java (original)
+++ oozie/branches/branch-3.3/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java Mon Feb  4 18:02:27 2013
@@ -160,17 +160,28 @@ public class V1JobsServlet extends BaseJ
 
         try {
             String action = request.getParameter(RestConstants.ACTION_PARAM);
-            if (action != null && !action.equals(RestConstants.JOB_ACTION_START)) {
+            if (action != null && !action.equals(RestConstants.JOB_ACTION_START)
+                    && !action.equals(RestConstants.JOB_ACTION_DRYRUN)) {
                 throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0303,
                         RestConstants.ACTION_PARAM, action);
             }
             boolean startJob = (action != null);
             String user = conf.get(OozieClient.USER_NAME);
             DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(user, getAuthToken(request));
-            String id = dagEngine.submitJob(conf, startJob);
+            String id;
+            boolean dryrun = false;
+            if (action != null) {
+                dryrun = (action.equals(RestConstants.JOB_ACTION_DRYRUN));
+            }
+            if (dryrun) {
+                id = dagEngine.dryRunSubmit(conf);
+            }
+            else {
+                id = dagEngine.submitJob(conf, startJob);
+            }
             json.put(JsonTags.JOB_ID, id);
         }
-        catch (DagEngineException ex) {
+        catch (BaseEngineException ex) {
             throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ex);
         }
 
@@ -202,7 +213,7 @@ public class V1JobsServlet extends BaseJ
                 dryrun = (action.equals(RestConstants.JOB_ACTION_DRYRUN));
             }
             if (dryrun) {
-                id = coordEngine.dryrunSubmit(conf, startJob);
+                id = coordEngine.dryRunSubmit(conf);
             }
             else {
                 id = coordEngine.submitJob(conf, startJob);
@@ -240,7 +251,7 @@ public class V1JobsServlet extends BaseJ
                 dryrun = (action.equals(RestConstants.JOB_ACTION_DRYRUN));
             }
             if (dryrun) {
-                id = bundleEngine.dryrunSubmit(conf, startJob);
+                id = bundleEngine.dryRunSubmit(conf);
             }
             else {
                 id = bundleEngine.submitJob(conf, startJob);

Modified: oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java (original)
+++ oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java Mon Feb  4 18:02:27 2013
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.PrintWriter;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.oozie.ErrorCode;
 import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.local.LocalOozie;
 import org.apache.oozie.client.OozieClient;
@@ -33,6 +34,7 @@ import org.apache.oozie.store.WorkflowSt
 import org.apache.oozie.test.XDataTestCase;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.service.XLogService;
+import org.apache.oozie.util.IOUtils;
 
 public class TestSubmitXCommand extends XDataTestCase {
     @Override
@@ -167,6 +169,66 @@ public class TestSubmitXCommand extends 
         }
     }
 
+    public void testDryrunValidXml() throws Exception {
+        Configuration conf = new XConfiguration();
+        String appPath = getTestCaseDir();
+        String appXml = IOUtils.getResourceAsString("wf-schema-valid-global.xml", -1);
+        writeToFile(appXml, appPath + "/workflow.xml");
+        conf.set(OozieClient.APP_PATH, "file://" + appPath);
+        conf.set(OozieClient.USER_NAME, getTestUser());
+        SubmitXCommand sc = new SubmitXCommand(true, conf, "UNIT_TESTING");
+        assertEquals("OK", sc.call());
+    }
+
+    public void testDryrunInvalidXml() throws Exception {
+        Configuration conf = new XConfiguration();
+        String appPath = getTestCaseDir();
+        String appXml = IOUtils.getResourceAsString("wf-loop1-invalid.xml", -1);
+        writeToFile(appXml, appPath + "/workflow.xml");
+        conf.set(OozieClient.APP_PATH, "file://" + appPath);
+        conf.set(OozieClient.USER_NAME, getTestUser());
+        SubmitXCommand sc = new SubmitXCommand(true, conf, "UNIT_TESTING");
+        try {
+            sc.call();
+            fail("Should have gotten E0707 because the XML has a loop");
+        } catch (CommandException ce) {
+            assertEquals(ErrorCode.E0707, ce.getErrorCode());
+            assertEquals("E0707: Loop detected at parsing, node [a]", ce.getMessage());
+        }
+
+        conf = new XConfiguration();
+        appPath = getTestCaseDir();
+        appXml = IOUtils.getResourceAsString("wf-transition-invalid.xml", -1);
+        writeToFile(appXml, appPath + "/workflow.xml");
+        conf.set(OozieClient.APP_PATH, "file://" + appPath);
+        conf.set(OozieClient.USER_NAME, getTestUser());
+        sc = new SubmitXCommand(true, conf, "UNIT_TESTING");
+        try {
+            sc.call();
+            fail("Should have gotten E0708 because the XML has an invalid transition");
+        } catch (CommandException ce) {
+            assertEquals(ErrorCode.E0708, ce.getErrorCode());
+            assertEquals("E0708: Invalid transition, node [c] transition [f]", ce.getMessage());
+        }
+
+        conf = new XConfiguration();
+        appPath = getTestCaseDir();
+        appXml = IOUtils.getResourceAsString("wf-schema-invalid.xml", -1);
+        writeToFile(appXml, appPath + "/workflow.xml");
+        conf.set(OozieClient.APP_PATH, "file://" + appPath);
+        conf.set(OozieClient.USER_NAME, getTestUser());
+        sc = new SubmitXCommand(true, conf, "UNIT_TESTING");
+        try {
+            sc.call();
+            fail("Should have gotten E0701 because the XML has an invalid element");
+        } catch (CommandException ce) {
+            assertEquals(ErrorCode.E0701, ce.getErrorCode());
+            assertTrue(ce.getMessage().contains("XML schema error"));
+            assertTrue(ce.getMessage().contains("starting with element 'xstart'"));
+            assertTrue(ce.getMessage().contains("'{\"uri:oozie:workflow:0.1\":start}' is expected"));
+        }
+    }
+
     private void writeToFile(String appXml, String appPath) throws IOException {
         // TODO Auto-generated method stub
         File wf = new File(appPath);

Modified: oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java (original)
+++ oozie/branches/branch-3.3/core/src/test/java/org/apache/oozie/servlet/MockCoordinatorEngineService.java Mon Feb  4 18:02:27 2013
@@ -98,11 +98,11 @@ public class MockCoordinatorEngineServic
         }
 
         @Override
-        public String dryrunSubmit(Configuration conf, boolean startJob) throws CoordinatorEngineException {
+        public String dryRunSubmit(Configuration conf) throws CoordinatorEngineException {
             did = RestConstants.JOB_ACTION_DRYRUN;
             int idx = coordJobs.size();
             coordJobs.add(createDummyCoordinatorJob(idx, conf));
-            started.add(startJob);
+            started.add(false);
             return JOB_ID + idx;
         }
 

Modified: oozie/branches/branch-3.3/docs/src/site/twiki/DG_CommandLineTool.twiki
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/docs/src/site/twiki/DG_CommandLineTool.twiki?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/docs/src/site/twiki/DG_CommandLineTool.twiki (original)
+++ oozie/branches/branch-3.3/docs/src/site/twiki/DG_CommandLineTool.twiki Mon Feb  4 18:02:27 2013
@@ -34,8 +34,7 @@ usage:
                 -date <arg>           coordinator/bundle rerun on action dates (requires -rerun)
                 -definition <arg>     job definition
                 -doas <arg>           doAs user, impersonates as the specified user
-                -dryrun               Supported in Oozie-2.0 or later versions ONLY - dryrun or test
-                                      run a coordinator job, job is not queued
+                -dryrun               Dryrun a workflow (since 3.4) or coordinator (since 2.0) job without actually executing it
                 -info <arg>           info of a job
                 -kill <arg>           kill a job
                 -len <arg>            number of actions (default TOTAL ACTIONS, requires -info)
@@ -474,7 +473,7 @@ Example:
 
 <verbatim>
 
-$ oozie job -oozie http://localhost:11000/oozie job -dryrun -config job.properties
+$ oozie job -oozie http://localhost:11000/oozie -dryrun -config job.properties
 ***coordJob after parsing: ***
 <coordinator-app xmlns="uri:oozie:coordinator:0.1" name="sla_coord" frequency="20"
 start="2009-03-06T010:00Z" end="2009-03-20T11:00Z" timezone="America/Los_Angeles">
@@ -519,6 +518,30 @@ Configuration file (.xml). This file mus
 The coordinator application path must be specified in the file with the =oozie.coord.application.path= property. The
 specified path must be an HDFS path.
 
+---+++ Dryrun of Workflow Job
+
+* This feature is only supported in Oozie 3.4 or later.
+
+Example:
+
+<verbatim>
+
+$ oozie job -oozie http://localhost:11000/oozie -dryrun -config job.properties
+OK
+
+</verbatim>
+
+The =dryrun= option tests running a workflow job with given job properties and does not create the job.
+
+The parameters for the job must be provided in a file, either a Java Properties file (.properties) or a Hadoop XML
+Configuration file (.xml). This file must be specified with the <code>-config</code> option.
+
+The workflow application path must be specified in the file with the =oozie.wf.application.path= property. The
+specified path must be an HDFS path.
+
+If the workflow is accepted (i.e. Oozie is able to successfully read and parse it), it will return ="OK"=; otherwise, it will return
+an error message describing why it was rejected.
+
 ---++ Jobs Operations
 
 ---+++ Checking the Status of multiple Workflow Jobs

Modified: oozie/branches/branch-3.3/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/branches/branch-3.3/release-log.txt?rev=1442244&r1=1442243&r2=1442244&view=diff
==============================================================================
--- oozie/branches/branch-3.3/release-log.txt (original)
+++ oozie/branches/branch-3.3/release-log.txt Mon Feb  4 18:02:27 2013
@@ -1,5 +1,6 @@
 -- Oozie 3.3.2 (unreleased)
 
+OOZIE-1080 Add a dryrun option for workflows (rkanter)
 OOZIE-1062 Create a shell example
 OOZIE-1034 Allow disabling forkjoin validation just for a specific workflow
 OOZIE-1072 Oozie Coordinator Doc error in Synchronous datasets