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 2015/06/24 01:53:03 UTC

oozie git commit: OOZIE-2159 'oozie validate' command should be moved server-side (seoeun25 via rkanter)

Repository: oozie
Updated Branches:
  refs/heads/master 168de6607 -> cc94ad8e5


OOZIE-2159 'oozie validate' command should be moved server-side (seoeun25 via rkanter)


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

Branch: refs/heads/master
Commit: cc94ad8e56c47a5b647f5da3a3dd9e8c5200fb30
Parents: 168de66
Author: Robert Kanter <rk...@cloudera.com>
Authored: Tue Jun 23 16:52:09 2015 -0700
Committer: Robert Kanter <rk...@cloudera.com>
Committed: Tue Jun 23 16:52:09 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/oozie/cli/OozieCLI.java     |  42 +-
 .../org/apache/oozie/client/OozieClient.java    |  64 ++
 .../org/apache/oozie/client/rest/JsonTags.java  |   3 +
 .../apache/oozie/client/rest/RestConstants.java |   7 +
 .../org/apache/oozie/cli/TestValidation.java    |  34 +-
 .../apache/oozie/servlet/V2ValidateServlet.java | 150 ++++
 .../org/apache/oozie/client/TestOozieCLI.java   |  25 +-
 .../oozie/servlet/TestV2ValidateServlet.java    | 676 +++++++++++++++++++
 distro/src/main/tomcat/ssl-web.xml              |  12 +
 docs/src/site/twiki/DG_CommandLineTool.twiki    |  23 +-
 docs/src/site/twiki/WebServicesAPI.twiki        |  69 ++
 release-log.txt                                 |   1 +
 webapp/src/main/webapp/WEB-INF/web.xml          |  12 +
 13 files changed, 1102 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
index c381432..48bac7d 100644
--- a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
+++ b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -514,6 +514,19 @@ public class OozieCLI {
     }
 
     /**
+     * Create option for command line option 'validate'
+     *
+     * @return validate options
+     */
+    protected Options createValidateOptions() {
+        Option oozie = new Option(OOZIE_OPTION, true, "Oozie URL");
+        Options validateOption = new Options();
+        validateOption.addOption(oozie);
+        addAuthOptions(validateOption);
+        return validateOption;
+    }
+
+    /**
      * Create option for command line option 'pig' or 'hive'
      * @return pig or hive options
      */
@@ -652,7 +665,7 @@ public class OozieCLI {
         parser.addCommand(JOB_CMD, "", "job operations", createJobOptions(), false);
         parser.addCommand(JOBS_CMD, "", "jobs status", createJobsOptions(), false);
         parser.addCommand(ADMIN_CMD, "", "admin operations", createAdminOptions(), false);
-        parser.addCommand(VALIDATE_CMD, "", "validate a workflow XML file", new Options(), true);
+        parser.addCommand(VALIDATE_CMD, "", "validate a workflow, coordinator, bundle XML file", createValidateOptions(), true);
         parser.addCommand(SLA_CMD, "", "sla operations (Deprecated with Oozie 4.0)", createSlaOptions(), false);
         parser.addCommand(PIG_CMD, "-X ", "submit a pig job, everything after '-X' are pass-through parameters to pig, any '-D' "
                 + "arguments after '-X' are put in <configuration>", createScriptLanguageOptions(PIG_CMD), true);
@@ -1985,6 +1998,33 @@ public class OozieCLI {
         if (args.length != 1) {
             throw new OozieCLIException("One file must be specified");
         }
+        try {
+            XOozieClient wc = createXOozieClient(commandLine);
+            String result = wc.validateXML(args[0].toString());
+            if (result == null) {
+                // TODO This is only for backward compatibility. Need to remove after 4.2.0 higher version.
+                System.out.println("Using client-side validation. Check out Oozie server version.");
+                validateCommandV41(commandLine);
+                return;
+            }
+            System.out.println(result);
+        } catch (OozieClientException e) {
+            throw new OozieCLIException(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Validate on client-side. This is only for backward compatibility. Need to removed after <tt>4.2.0</tt> higher version.
+     * @param commandLine
+     * @throws OozieCLIException
+     */
+    @Deprecated
+    @VisibleForTesting
+    void validateCommandV41(CommandLine commandLine) throws OozieCLIException {
+        String[] args = commandLine.getArgs();
+        if (args.length != 1) {
+            throw new OozieCLIException("One file must be specified");
+        }
         File file = new File(args[0]);
         if (file.exists()) {
             try {

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/client/src/main/java/org/apache/oozie/client/OozieClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/OozieClient.java b/client/src/main/java/org/apache/oozie/client/OozieClient.java
index 6b7e750..67a62c6 100644
--- a/client/src/main/java/org/apache/oozie/client/OozieClient.java
+++ b/client/src/main/java/org/apache/oozie/client/OozieClient.java
@@ -35,6 +35,8 @@ import javax.xml.transform.TransformerFactory;
 import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.stream.StreamResult;
 import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
@@ -2023,6 +2025,42 @@ public class OozieClient {
         }
     }
 
+    private class ValidateXML extends ClientCallable<String> {
+
+        String file = null;
+
+        ValidateXML(String file, String user) {
+            super("POST", RestConstants.VALIDATE, "",
+                    prepareParams(RestConstants.FILE_PARAM, file, RestConstants.USER_PARAM, user));
+            this.file = file;
+        }
+
+        @Override
+        protected String call(HttpURLConnection conn) throws IOException, OozieClientException {
+            conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+            if (file.startsWith("/")) {
+                FileInputStream fi = new FileInputStream(new File(file));
+                byte[] buffer = new byte[1024];
+                int n = 0;
+                while (-1 != (n = fi.read(buffer))) {
+                    conn.getOutputStream().write(buffer, 0, n);
+                }
+            }
+            if ((conn.getResponseCode() == HttpURLConnection.HTTP_OK)) {
+                Reader reader = new InputStreamReader(conn.getInputStream());
+                JSONObject json = (JSONObject) JSONValue.parse(reader);
+                return (String) json.get(JsonTags.VALIDATE);
+            }
+            else if ((conn.getResponseCode() == HttpURLConnection.HTTP_NOT_FOUND)) {
+                return null;
+            }
+            else {
+                handleError(conn);
+            }
+            return null;
+        }
+    }
+
 
     private  class UpdateSharelib extends ClientCallable<String> {
 
@@ -2123,6 +2161,32 @@ public class OozieClient {
     }
 
     /**
+     * Return the workflow application is valid.
+     *
+     * @param file local file or hdfs file.
+     * @return the workflow application is valid.
+     * @throws OozieClientException throw if it the workflow application's validation could not be retrieved.
+     */
+    public String validateXML(String file) throws OozieClientException {
+        String fileName = file;
+        if (file.startsWith("file://")) {
+            fileName = file.substring(7, file.length());
+        }
+        if (!fileName.contains("://")) {
+            File f = new File(fileName);
+            if (!f.isFile()) {
+                throw new OozieClientException("File error", "File does not exist : " + f.getAbsolutePath());
+            }
+            fileName = f.getAbsolutePath();
+        }
+        String user = USER_NAME_TL.get();
+        if (user == null) {
+            user = System.getProperty("user.name");
+        }
+        return new ValidateXML(fileName, user).call();
+    }
+
+    /**
      * Return the info of the coordinator jobs that match the filter.
      *
      * @param filter job filter. Refer to the {@link OozieClient} for the filter syntax.

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java b/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java
index ab6699f..b3b148a 100644
--- a/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java
+++ b/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java
@@ -244,4 +244,7 @@ public interface JsonTags {
     public static final String COORD_UPDATE_DIFF = "diff";
 
     public static final String STATUS = "status";
+
+    public static final String VALIDATE = "validate";
+
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java b/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
index 2434b19..3c69319 100644
--- a/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
+++ b/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
@@ -208,4 +208,11 @@ public interface RestConstants {
     public static final String SLA_MAX_DURATION = "sla-max-duration";
 
     public static final String JOB_COORD_SCOPE_ACTION_LIST = "action-list";
+
+    public static final String VALIDATE = "validate";
+
+    public static final String FILE_PARAM = "file";
+
+    public static final String USER_PARAM = "user";
+
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/client/src/test/java/org/apache/oozie/cli/TestValidation.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/oozie/cli/TestValidation.java b/client/src/test/java/org/apache/oozie/cli/TestValidation.java
index 76c1445..82d4430 100644
--- a/client/src/test/java/org/apache/oozie/cli/TestValidation.java
+++ b/client/src/test/java/org/apache/oozie/cli/TestValidation.java
@@ -19,7 +19,11 @@
 package org.apache.oozie.cli;
 
 import junit.framework.TestCase;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.ParseException;
 
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
 import java.net.URL;
 import java.net.URI;
 import java.io.File;
@@ -35,11 +39,37 @@ public class TestValidation extends TestCase {
 
     public void testValid() throws Exception {
         String[] args = new String[]{"validate", getPath("valid.xml")};
-        assertEquals(0, new OozieCLI().run(args));
+        assertTrue(captureOutput(args).contains("Valid workflow-app"));
     }
 
     public void testInvalid() throws Exception {
         String[] args = new String[]{"validate", getPath("invalid.xml")};
-        assertEquals(-1, new OozieCLI().run(args));
+        assertTrue(captureOutput(args).contains("Invalid app definition"));
+    }
+
+    private String captureOutput(String[] args) throws ParseException {
+        OozieCLI cli = new OozieCLI();
+        CLIParser parser = cli.getCLIParser();
+        CLIParser.Command command = parser.parse(args);
+        PrintStream original = System.out;
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        PrintStream ps = new PrintStream(baos);
+        String outStr = null;
+        System.out.flush();
+        try {
+            System.setOut(ps);
+            cli.validateCommandV41(command.getCommandLine());
+            System.out.flush();
+            outStr = baos.toString();
+        } catch (OozieCLIException e) {
+            outStr = e.getMessage();
+        } finally {
+            System.setOut(original);
+            if (outStr != null) {
+                System.out.print(outStr);
+            }
+            System.out.flush();
+        }
+        return outStr;
     }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java b/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java
new file mode 100644
index 0000000..dbb3d49
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java
@@ -0,0 +1,150 @@
+/**
+ * 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.servlet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.cli.OozieCLIException;
+import org.apache.oozie.client.rest.JsonTags;
+import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.service.HadoopAccessorService;
+import org.apache.oozie.service.SchemaService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.util.IOUtils;
+import org.json.simple.JSONObject;
+import org.xml.sax.SAXException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.Validator;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.net.URI;
+import java.util.Arrays;
+
+public class V2ValidateServlet extends JsonRestServlet {
+    private static final String INSTRUMENTATION_NAME = "v2validate";
+
+    private static final ResourceInfo RESOURCE_INFO =
+            new ResourceInfo("", Arrays.asList("POST"), Arrays.asList(
+                    new ParameterInfo(RestConstants.FILE_PARAM, String.class, true, Arrays.asList("POST")),
+                    new ParameterInfo(RestConstants.USER_PARAM, String.class, true, Arrays.asList("POST"))));
+
+
+    public V2ValidateServlet() {
+        super(INSTRUMENTATION_NAME, RESOURCE_INFO);
+    }
+
+    /**
+     * Validate workflow definition.
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
+        validateContentType(request, RestConstants.XML_CONTENT_TYPE);
+
+        String file = request.getParameter(RestConstants.FILE_PARAM);
+        String user = request.getParameter(RestConstants.USER_PARAM);
+
+        stopCron();
+
+        StringWriter stringWriter = new StringWriter();
+        if (file.startsWith("hdfs://")) {
+            try {
+                URI uri = new URI(file);
+                HadoopAccessorService has = Services.get().get(HadoopAccessorService.class);
+                Configuration fsConf = has.createJobConf(uri.getAuthority());
+                FileSystem fs = has.createFileSystem(user, uri, fsConf);
+
+                Path path = new Path(uri.getPath());
+                IOUtils.copyCharStream(new InputStreamReader(fs.open(path)), stringWriter);
+
+            } catch (Exception e) {
+                throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0505,
+                        "File does not exist, "+ file);
+            }
+        }
+        else {
+            IOUtils.copyCharStream(new InputStreamReader(request.getInputStream()), stringWriter);
+        }
+        try {
+            validate(stringWriter.toString());
+        } catch (Exception e) {
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0701,
+                    file + ", " + e.toString());
+        }
+
+        JSONObject json = createJSON("Valid workflow-app");
+        startCron();
+        sendJsonResponse(response, HttpServletResponse.SC_OK, json);
+    }
+
+    private void validate(String xml) throws Exception{
+        SchemaService schemaService = Services.get().get(SchemaService.class);
+        Schema[] schemas = {schemaService.getSchema(SchemaService.SchemaName.WORKFLOW),
+                schemaService.getSchema(SchemaService.SchemaName.COORDINATOR),
+                schemaService.getSchema(SchemaService.SchemaName.BUNDLE),
+                schemaService.getSchema(SchemaService.SchemaName.SLA_ORIGINAL)};
+
+        Exception exception = null;
+        for (int i = 0; i < schemas.length; i++) {
+            try{
+                validateSchema(schemas[i], new StringReader(xml));
+                exception = null;
+                break;
+            } catch (SAXException e) {
+                if (i == 0) {
+                    exception = e;
+                }
+                // Check the root element declaration(workflow-app, coordinator-app, bundle-app).
+                // If invalid, move to next schema validation.
+                if (!e.getMessage().contains("cvc-elt.1.a")) {
+                    exception = e;
+                    break;
+                }
+            } catch (Exception e) {
+                exception = e;
+                break;
+            }
+        }
+        if (exception !=  null) {
+            throw exception;
+        }
+    }
+
+    private void validateSchema(Schema schema, Reader src) throws SAXException, IOException, OozieCLIException{
+            Validator validator = schema.newValidator();
+            validator.validate(new StreamSource(src));
+    }
+
+    private JSONObject createJSON(String content) {
+        JSONObject jsonObject = new JSONObject();
+        jsonObject.put(JsonTags.VALIDATE, content);
+        return jsonObject;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java b/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
index b654e48..57e3433 100644
--- a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
+++ b/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
@@ -31,6 +31,7 @@ import java.util.concurrent.Callable;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.oozie.BaseEngine;
 import org.apache.oozie.BuildInfo;
 import org.apache.oozie.cli.CLIParser;
 import org.apache.oozie.cli.OozieCLI;
@@ -47,6 +48,7 @@ import org.apache.oozie.servlet.V1JobServlet;
 import org.apache.oozie.servlet.V1JobsServlet;
 import org.apache.oozie.servlet.V2AdminServlet;
 import org.apache.oozie.servlet.V2JobServlet;
+import org.apache.oozie.servlet.V2ValidateServlet;
 import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XConfiguration;
 
@@ -60,13 +62,15 @@ public class TestOozieCLI extends DagServletTestCase {
         new V1AdminServlet();
         new V2AdminServlet();
         new V2JobServlet();
+        new V2ValidateServlet();
     }
 
     static final boolean IS_SECURITY_ENABLED = false;
     static final String VERSION = "/v" + OozieClient.WS_PROTOCOL_VERSION;
-    static final String[] END_POINTS = {"/versions", VERSION + "/jobs", VERSION + "/job/*", VERSION + "/admin/*"};
+    static final String[] END_POINTS = {"/versions", VERSION + "/jobs", VERSION + "/job/*", VERSION + "/admin/*",
+            VERSION + "/validate/*"};
     static final Class[] SERVLET_CLASSES = { HeaderTestingVersionServlet.class, V1JobsServlet.class,
-            V2JobServlet.class, V2AdminServlet.class, V2JobServlet.class, V2AdminServlet.class };
+            V2JobServlet.class, V2AdminServlet.class, V2ValidateServlet.class, V2JobServlet.class, V2AdminServlet.class};
 
     @Override
     protected void setUp() throws Exception {
@@ -1155,8 +1159,8 @@ public class TestOozieCLI extends DagServletTestCase {
         runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
             @Override
             public Void call() throws Exception {
-                String validFileName = "./test-workflow-app.xml";
-                String invalidFileName = "./test-invalid-workflow-app.xml";
+                String validFileName = "test-workflow-app.xml";
+                String invalidFileName = "test-invalid-workflow-app.xml";
 
                 String validContent = "<workflow-app xmlns=\"uri:oozie:workflow:0.2\" name=\"no-op-wf\"> "+
                         " <start to=\"end\"/> <end name=\"end\"/> </workflow-app>";
@@ -1167,20 +1171,21 @@ public class TestOozieCLI extends DagServletTestCase {
                 validfile.delete();
                 invalidfile.delete();
 
+                String oozieUrl = getContextURL();
 
-                IOUtils.copyCharStream(new StringReader(validContent), new FileWriter(validfile));
-                String [] args = new String[] { "validate", validFileName };
+                IOUtils.copyCharStream(new StringReader(validContent), new  FileWriter(validfile));
+                String [] args = new String[] { "validate", "-oozie", oozieUrl, validfile.getAbsolutePath() };
                 String out = runOozieCLIAndGetStdout(args);
                 assertTrue(out.contains("Valid"));
 
                 IOUtils.copyCharStream(new StringReader(invalidContent), new FileWriter(invalidfile));
-                args = new String[] { "validate", invalidFileName };
+                args = new String[] { "validate", "-oozie", oozieUrl, invalidfile.getAbsolutePath() };
                 out = runOozieCLIAndGetStderr(args);
-                assertTrue(out.contains("Invalid"));
+                assertTrue(out.contains("XML schema error"));
 
                 return null;
-          }
-      });
+            }
+        });
     }
 
    /**

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/core/src/test/java/org/apache/oozie/servlet/TestV2ValidateServlet.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/servlet/TestV2ValidateServlet.java b/core/src/test/java/org/apache/oozie/servlet/TestV2ValidateServlet.java
new file mode 100644
index 0000000..2eb6c01
--- /dev/null
+++ b/core/src/test/java/org/apache/oozie/servlet/TestV2ValidateServlet.java
@@ -0,0 +1,676 @@
+/**
+ * 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.servlet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.oozie.client.rest.JsonTags;
+import org.apache.oozie.client.rest.RestConstants;
+import org.json.simple.JSONObject;
+import org.json.simple.JSONValue;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+public class TestV2ValidateServlet extends DagServletTestCase {
+
+    static {
+        new V2ValidateServlet();
+    }
+
+    private static final boolean IS_SECURITY_ENABLED = false;
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+    }
+
+    public void testValidateWF() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "workflow.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>\n" +
+                        "<workflow-app xmlns=\"uri:oozie:workflow:0.3\" name=\"test\">\n" +
+                        "    <start to=\"shell-1\"/>\n" +
+                        "    <action name=\"shell-1\">\n" +
+                        "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                        "            <job-tracker>${jobTracker}</job-tracker>\n" +
+                        "            <name-node>${nameNode}</name-node>\n" +
+                        "            <exec>script-outstream.sh</exec>\n" +
+                        "            <argument></argument>\n" +
+                        "            <file>script-outstream.sh</file>\n" +
+                        "            <capture-output/>\n" +
+                        "        </shell>\n" +
+                        "        <ok to=\"end\"/>\n" +
+                        "        <error to=\"fail\"/>\n" +
+                        "    </action>\n" +
+                        "    <kill name=\"fail\">\n" +
+                        "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                        "    </kill>\n" +
+                        "    <end name=\"end\"/>\n" +
+                        "</workflow-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                JSONObject obj = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
+                assertEquals("Valid workflow-app", obj.get(JsonTags.VALIDATE));
+
+                return null;
+            }
+        });
+    }
+
+    public void testValidateWFonHDFS() throws Exception {
+        String xml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>\n" +
+                "<workflow-app xmlns=\"uri:oozie:workflow:0.3\" name=\"test\">\n" +
+                "    <start to=\"shell-1\"/>\n" +
+                "    <action name=\"shell-1\">\n" +
+                "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                "            <job-tracker>${jobTracker}</job-tracker>\n" +
+                "            <name-node>${nameNode}</name-node>\n" +
+                "            <exec>script-outstream.sh</exec>\n" +
+                "            <argument></argument>\n" +
+                "            <file>script-outstream.sh</file>\n" +
+                "            <capture-output/>\n" +
+                "        </shell>\n" +
+                "        <ok to=\"end\"/>\n" +
+                "        <error to=\"fail\"/>\n" +
+                "    </action>\n" +
+                "    <kill name=\"fail\">\n" +
+                "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                "    </kill>\n" +
+                "    <end name=\"end\"/>\n" +
+                "</workflow-app>";
+        final Path path = new Path(getFsTestCaseDir(), "workflow.xml");
+        OutputStreamWriter writer = new OutputStreamWriter(getFileSystem().create(path));
+        writer.write(xml.toCharArray());
+        writer.flush();
+        writer.close();
+
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", path.toString());
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+
+                assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                JSONObject obj = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
+                assertEquals("Valid workflow-app", obj.get(JsonTags.VALIDATE));
+
+                return null;
+            }
+        });
+    }
+
+    public void testValidateWFNegative() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "workflow.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<workflow-app xmlns=\"uri:oozie:workflow:0.3\" name=\"test\">\n" +
+                        "    <start to=\"shell-1\"/>\n" +
+                        "    <action name=\"shell-1\">\n" +
+                        "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                        "            <name-node2>${nameNode}</name-node2>\n" +
+                        "            <exec>script-outstream.sh</exec>\n" +
+                        "            <argument></argument>\n" +
+                        "            <file>script-outstream.sh</file>\n" +
+                        "            <capture-output/>\n" +
+                        "        </shell>\n" +
+                        "        <ok to=\"end\"/>\n" +
+                        "        <error to=\"fail\"/>\n" +
+                        "    </action>\n" +
+                        "    <kill name=\"fail\">\n" +
+                        "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                        "    </kill>\n" +
+                        "    <end name=\"end\"/>\n" +
+                        "</workflow-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("Invalid content was found starting with element 'name-node2'"));
+                return null;
+            }
+        });
+    }
+
+    public void testValidateWFNegative2() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "workflow.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<workflow-app xmlns=\"uri:oozie:workflow:0.3\" name=\"test\">\n" +
+                        "    <start to=\"shell-1\"/>\n" +
+                        "    <action name=\"shell-1\">\n" +
+                        "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                        "            <name-node>${nameNode}</name-node>\n" +
+                        "            <exec>script-outstream.sh</exec>\n" +
+                        "            <argument></argument>\n" +
+                        "            <file>script-outstream.sh</file>\n" +
+                        "            <capture-output/>\n" +
+                        "        </shell>\n" +
+                        "        <ok to=\"end\"/>\n" +
+                        "        <error to=\"fail\"/>\n" +
+                        "    </action>\n" +
+                        "    <kill-invalid name=\"fail\">\n" +
+                        "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                        "    </kill-invalid>\n" +
+                        "    <end name=\"end\"/>\n" +
+                        "</workflow-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("Invalid content was found starting with element 'kill-invalid'"));
+                return null;
+            }
+        });
+    }
+
+    public void testValidateWFNegative3() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "workflow.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<workflow-app-invalid xmlns=\"uri:oozie:workflow:0.3\" name=\"test\">\n" +
+                        "    <start to=\"shell-1\"/>\n" +
+                        "    <action name=\"shell-1\">\n" +
+                        "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                        "            <name-node>${nameNode}</name-node>\n" +
+                        "            <exec>script-outstream.sh</exec>\n" +
+                        "            <argument></argument>\n" +
+                        "            <file>script-outstream.sh</file>\n" +
+                        "            <capture-output/>\n" +
+                        "        </shell>\n" +
+                        "        <ok to=\"end\"/>\n" +
+                        "        <error to=\"fail\"/>\n" +
+                        "    </action>\n" +
+                        "    <kill name=\"fail\">\n" +
+                        "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                        "    </kill>\n" +
+                        "    <end name=\"end\"/>\n" +
+                        "</workflow-app-invalid>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("Cannot find the declaration of element 'workflow-app-invalid"));
+                return null;
+            }
+        });
+    }
+
+    public void testValidateWFNegative4() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "workflow.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<workflow-app xmlns=\"uri:oozie:workflow:0.3\" name=\"test\">\n" +
+                        "    <start to=\"shell-1\"/>\n" +
+                        "    <start to=\"shell-1\"/>\n" +
+                        "    <action name=\"shell-1\">\n" +
+                        "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                        "            <name-node>${nameNode}</name-node>\n" +
+                        "            <exec>script-outstream.sh</exec>\n" +
+                        "            <argument></argument>\n" +
+                        "            <file>script-outstream.sh</file>\n" +
+                        "            <capture-output/>\n" +
+                        "        </shell>\n" +
+                        "        <ok to=\"end\"/>\n" +
+                        "        <error to=\"fail\"/>\n" +
+                        "    </action>\n" +
+                        "    <kill name=\"fail\">\n" +
+                        "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                        "    </kill>\n" +
+                        "    <end name=\"end\"/>\n" +
+                        "</workflow-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("cvc-complex-type.2.4.a: " +
+                        "Invalid content was found starting with element 'start'"));
+                return null;
+            }
+        });
+    }
+
+    public void testValidateWFonHDFSNegative() throws Exception {
+        String xml = "<workflow-app xmlns=\"uri:oozie:workflow:0.3\" name=\"test\">\n" +
+                "    <start to=\"shell-1\"/>\n" +
+                "    <action name=\"shell-1\">\n" +
+                "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                "            <name-node2>${nameNode}</name-node2>\n" +
+                "            <exec>script-outstream.sh</exec>\n" +
+                "            <argument></argument>\n" +
+                "            <file>script-outstream.sh</file>\n" +
+                "            <capture-output/>\n" +
+                "        </shell>\n" +
+                "        <ok to=\"end\"/>\n" +
+                "        <error to=\"fail\"/>\n" +
+                "    </action>\n" +
+                "    <kill name=\"fail\">\n" +
+                "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                "    </kill>\n" +
+                "    <end name=\"end\"/>\n" +
+                "</workflow-app>";
+        final Path path = new Path(getFsTestCaseDir(), "workflow.xml");
+        OutputStreamWriter writer = new OutputStreamWriter(getFileSystem().create(path));
+        writer.write(xml.toCharArray());
+        writer.flush();
+        writer.close();
+
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", path.toString());
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("Invalid content was found starting with element 'name-node2'"));
+                return null;
+            }
+        });
+    }
+
+    public void testValidateCoordinator() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "coordinator.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<coordinator-app name=\"coord-simple\" frequency=\"${coord:minutes(1)}\"\n" +
+                        "                 start=\"${startTime}\" end=\"${endTime}\"\n" +
+                        "                 timezone=\"Asia/Seoul\"\n" +
+                        "                 xmlns=\"uri:oozie:coordinator:0.1\">\n" +
+                        "    <action>\n" +
+                        "        <workflow>\n" +
+                        "            <app-path>${nameNode}/user/seoeun/workflow-ndap/apps/v40/shell-outstream</app-path>\n" +
+                        "        </workflow>\n" +
+                        "    </action>\n" +
+                        "</coordinator-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                JSONObject obj = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
+                assertEquals("Valid workflow-app", obj.get(JsonTags.VALIDATE));
+
+                return null;
+
+            }
+        });
+    }
+
+    public void testValidateCoordinatorNegative1() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "coordinator.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<coordinator-app name=\"coord-simple\" frequency=\"${coord:minutes(1)}\"\n" +
+                        "                 start=\"${startTime}\" end=\"${endTime}\"\n" +
+                        "                 timezone=\"Asia/Seoul\"\n" +
+                        "                 xmlns=\"uri:oozie:coordinator:0.1\">\n" +
+                        "    <action>\n" +
+                        "        <workflow>\n" +
+                        "            <app-path>${nameNode}/user/seoeun/workflow-ndap/apps/v40/shell-outstream</app-path>\n" +
+                        "            <action name=\"shell-1\">\n" +
+                        "                <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                        "                <job-tracker>${jobTracker}</job-tracker>\n" +
+                        "                <name-node>${nameNode}</name-node>\n" +
+                        "                <exec>script-outstream.sh</exec>\n" +
+                        "                </shell>\n" +
+                        "                <ok to=\"end\"/>\n" +
+                        "                <error to=\"fail\"/>\n" +
+                        "            </action>\n" +
+                        "        </workflow>\n" +
+                        "    </action>\n" +
+                        "</coordinator-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("Invalid content was found starting with element 'action'"));
+                return null;
+
+            }
+        });
+    }
+
+    public void testValidateCoordinatorNegative2() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "coordinator.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<coordinator-app-invalid name=\"coord-simple\" frequency=\"${coord:minutes(1)}\"\n" +
+                        "                 start=\"${startTime}\" end=\"${endTime}\"\n" +
+                        "                 timezone=\"Asia/Seoul\"\n" +
+                        "                 xmlns=\"uri:oozie:coordinator:0.1\">\n" +
+                        "    <action>\n" +
+                        "        <workflow>\n" +
+                        "            <app-path>${nameNode}/user/seoeun/workflow-ndap/apps/v40/shell-outstream</app-path>\n" +
+                        "        </workflow>\n" +
+                        "    </action>\n" +
+                        "</coordinator-app-invalid>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("Cannot find the declaration of element 'coordinator-app-invalid'"));
+                return null;
+
+            }
+        });
+    }
+
+    public void testValidateBundle() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "bundle.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<bundle-app name='test_bundle' xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance' "
+                        + "xmlns='uri:oozie:bundle:0.1'> "
+                        + "<controls> <kick-off-time>2009-02-02T00:00Z</kick-off-time> </controls> "
+                        + "<coordinator name='c12'> "
+                        + "<app-path>#app_path1</app-path>"
+                        + "<configuration> "
+                        + "<property> <name>START_TIME</name> <value>2009-02-01T00:00Z</value> </property> </configuration> "
+                        + "</coordinator></bundle-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                JSONObject obj = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
+                assertEquals("Valid workflow-app", obj.get(JsonTags.VALIDATE));
+
+                return null;
+
+            }
+        });
+    }
+
+    public void testValidateBundleNegative1() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "bundle.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<bundle-app name='test_bundle' xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance' "
+                        + "xmlns='uri:oozie:bundle:0.1'> "
+                        + "<controls> <kick-off-time>2009-02-02T00:00Z</kick-off-time> </controls> "
+                        + "</bundle-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("cvc-complex-type.2.4.b: The content of element 'bundle-app' is not " +
+                        "complete. One of '{\"uri:oozie:bundle:0.1\":coordinator}' is expected"));
+                return null;
+
+            }
+        });
+    }
+
+    public void testValidateBundleNegative2() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "bundle.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<bundle-app-invalid name='test_bundle' xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance' "
+                        + "xmlns='uri:oozie:bundle:0.1'> "
+                        + "<controls> <kick-off-time>2009-02-02T00:00Z</kick-off-time> </controls> "
+                        + "<coordinator name='c12'> "
+                        + "<app-path>#app_path1</app-path>"
+                        + "<configuration> "
+                        + "<property> <name>START_TIME</name> <value>2009-02-01T00:00Z</value> </property> </configuration> "
+                        + "</coordinator></bundle-app-invalid>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("cvc-elt.1.a: Cannot find the declaration of element 'bundle-app-invalid"));
+                return null;
+
+            }
+        });
+    }
+
+    public void testValidateSla() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "workflow.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<workflow-app xmlns=\"uri:oozie:workflow:0.5\" xmlns:sla=\"uri:oozie:sla:0.2\" name=\"test\">\n" +
+                        "    <start to=\"shell-1\"/>\n" +
+                        "    <action name=\"shell-1\">\n" +
+                        "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                        "            <job-tracker>${jobTracker}</job-tracker>\n" +
+                        "            <name-node>${nameNode}</name-node>\n" +
+                        "            <exec>script-outstream.sh</exec>\n" +
+                        "            <argument></argument>\n" +
+                        "            <file>script-outstream.sh</file>\n" +
+                        "            <capture-output/>\n" +
+                        "        </shell>\n" +
+                        "        <ok to=\"end\"/>\n" +
+                        "        <error to=\"fail\"/>\n" +
+                        "        <sla:info>\n" +
+                        "            <sla:nominal-time>${nominal_time}</sla:nominal-time>\n" +
+                        "            <sla:should-start>${10 * MINUTES}</sla:should-start>\n" +
+                        "            <sla:should-end>${30 * MINUTES}</sla:should-end>\n" +
+                        "            <sla:max-duration>${30 * MINUTES}</sla:max-duration>\n" +
+                        "            <sla:alert-events>start_miss,end_met,end_miss</sla:alert-events>\n" +
+                        "            <sla:alert-contact>joe@example.com</sla:alert-contact>\n" +
+                        "        </sla:info>\n" +
+                        "    </action>\n" +
+                        "    <kill name=\"fail\">\n" +
+                        "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                        "    </kill>\n" +
+                        "    <end name=\"end\"/>\n" +
+                        "</workflow-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                JSONObject obj = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
+                assertEquals("Valid workflow-app", obj.get(JsonTags.VALIDATE));
+
+                return null;
+
+            }
+        });
+    }
+
+    public void testValidateSlaNegative() throws Exception {
+        runTest("/v2/validate", V2ValidateServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                Map<String, String> params = new HashMap<String, String>();
+                params.put("file", "workflow.xml");
+                params.put("user", getTestUser());
+                URL url = createURL("", params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("POST");
+                conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+                conn.setDoOutput(true);
+                String xml = "<workflow-app xmlns=\"uri:oozie:workflow:0.5\" xmlns:sla=\"uri:oozie:sla:0.2\" name=\"test\">\n" +
+                        "    <start to=\"shell-1\"/>\n" +
+                        "    <action name=\"shell-1\">\n" +
+                        "        <shell xmlns=\"uri:oozie:shell-action:0.3\">\n" +
+                        "            <job-tracker>${jobTracker}</job-tracker>\n" +
+                        "            <name-node>${nameNode}</name-node>\n" +
+                        "            <exec>script-outstream.sh</exec>\n" +
+                        "            <argument></argument>\n" +
+                        "            <file>script-outstream.sh</file>\n" +
+                        "            <capture-output/>\n" +
+                        "        </shell>\n" +
+                        "        <ok to=\"end\"/>\n" +
+                        "        <error to=\"fail\"/>\n" +
+                        "        <sla:info>\n" +
+                        "            <sla:app-name>${nominal_time}</sla:app-name>\n" +
+                        "            <sla:nominal-time>${nominal_time}</sla:nominal-time>\n" +
+                        "            <sla:should-start>${10 * MINUTES}</sla:should-start>\n" +
+                        "            <sla:should-end>${30 * MINUTES}</sla:should-end>\n" +
+                        "            <sla:max-duration>${30 * MINUTES}</sla:max-duration>\n" +
+                        "            <sla:alert-events>start_miss,end_met,end_miss</sla:alert-events>\n" +
+                        "            <sla:alert-contact>joe@example.com</sla:alert-contact>\n" +
+                        "        </sla:info>\n" +
+                        "    </action>\n" +
+                        "    <kill name=\"fail\">\n" +
+                        "        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>\n" +
+                        "    </kill>\n" +
+                        "    <end name=\"end\"/>\n" +
+                        "</workflow-app>";
+                writeXML(conn.getOutputStream(), xml);
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                String error = conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE);
+                String message = conn.getHeaderField(RestConstants.OOZIE_ERROR_MESSAGE);
+                assertEquals("E0701", error);
+                assertEquals(true, message.contains("Invalid content was found starting with element 'sla:app-name'"));
+                return null;
+
+            }
+        });
+    }
+
+
+    private void writeXML(OutputStream outputStream, String xml) throws IOException {
+        outputStream.write(xml.getBytes());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/distro/src/main/tomcat/ssl-web.xml
----------------------------------------------------------------------
diff --git a/distro/src/main/tomcat/ssl-web.xml b/distro/src/main/tomcat/ssl-web.xml
index 103e12d..cf0d621 100644
--- a/distro/src/main/tomcat/ssl-web.xml
+++ b/distro/src/main/tomcat/ssl-web.xml
@@ -122,6 +122,13 @@
         <load-on-startup>1</load-on-startup>
     </servlet>
 
+    <servlet>
+        <servlet-name>validate</servlet-name>
+        <display-name>WS API for Workflow Applications</display-name>
+        <servlet-class>org.apache.oozie.servlet.V2ValidateServlet</servlet-class>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+
     <!-- servlet-mapping -->
     <servlet-mapping>
         <servlet-name>versions</servlet-name>
@@ -188,6 +195,11 @@
         <url-pattern>/v2/sla/*</url-pattern>
     </servlet-mapping>
 
+    <servlet-mapping>
+        <servlet-name>validate</servlet-name>
+        <url-pattern>/v2/validate</url-pattern>
+    </servlet-mapping>
+
     <!-- welcome-file -->
     <welcome-file-list>
         <welcome-file>index.jsp</welcome-file>

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/docs/src/site/twiki/DG_CommandLineTool.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/DG_CommandLineTool.twiki b/docs/src/site/twiki/DG_CommandLineTool.twiki
index 9494b22..1823247 100644
--- a/docs/src/site/twiki/DG_CommandLineTool.twiki
+++ b/docs/src/site/twiki/DG_CommandLineTool.twiki
@@ -136,7 +136,9 @@ usage:
                                       system mode [NORMAL|NOWEBSERVICE|SAFEMODE]
                   -version            show Oozie server build version
 .
-      oozie validate <ARGS> : validate a workflow XML file
+      oozie validate <OPTIONS> <ARGS> : validate a workflow, coordinator, bundle XML file
+                           -auth <arg>    select authentication type [SIMPLE|KERBEROS]
+                           -oozie <arg>   Oozie URL
 .
       oozie sla <OPTIONS> : sla operations (Deprecated as of Oozie 4.0)
                 -auth <arg>           select authentication type [SIMPLE|KERBEROS]
@@ -1430,13 +1432,28 @@ Example:
 <verbatim>
 $ oozie validate myApp/workflow.xml
 .
-Error: Invalid workflow-app, org.xml.sax.SAXParseException: cvc-complex-type.2.4.a:
+Error: E0701: XML schema error, workflow.xml, org.xml.sax.SAXParseException: cvc-complex-type.2.4.a:
        Invalid content was found starting with element 'xend'. One of '{"uri:oozie:workflow:0.1":decision,
        "uri:oozie:workflow:0.1":fork, "uri:oozie:workflow:0.1":join, "uri:oozie:workflow:0.1":kill,
        "uri:oozie:workflow:0.1":action, "uri:oozie:workflow:0.1":end}' is expected.
 </verbatim>
 
-It performs an XML Schema validation on the specified workflow XML file.
+<verbatim>
+$ oozie validate /home/test/myApp/coordinator.xml
+.
+Error: E0701: XML schema error, coordinator.xml, org.xml.sax.SAXParseException; lineNumber: 4; columnNumber: 52; cvc-elt.1.a:
+       Cannot find the declaration of element 'coordinator-app-invalid'.
+</verbatim>
+
+<verbatim>
+$ oozie validate hdfs://localhost:8020/user/test/myApp/bundle.xml
+.
+Error: E0701: XML schema error, bundle.xml, org.xml.sax.SAXParseException: cvc-complex-type.2.4.b:
+       The content of element 'bundle-app' is not complete. One of '{"uri:oozie:bundle:0.1":coordinator}' is expected.
+</verbatim>
+
+It performs an XML Schema validation on the specified workflow, coordinator, bundle XML file.
+The XML file can be a local file or in HDFS.
 
 ---+++ Getting list of available sharelib
 This command is used to get list of available sharelib.

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/docs/src/site/twiki/WebServicesAPI.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/WebServicesAPI.twiki b/docs/src/site/twiki/WebServicesAPI.twiki
index 3d97447..a34f2d3 100644
--- a/docs/src/site/twiki/WebServicesAPI.twiki
+++ b/docs/src/site/twiki/WebServicesAPI.twiki
@@ -2025,6 +2025,75 @@ Ignore coordinator actions
 <verbatim>
 PUT /oozie/v2/job/job-3?action=ignore&type=action&scope=3-4
 </verbatim>
+
+---+++ Validate End-Point
+
+This endpoint is to validate a workflow, coordinator, bundle XML file.
+
+---++++ Validate a local file
+
+*Request:*
+
+<verbatim>
+POST /oozie/v2/validate?file=/home/test/myApp/workflow.xml
+Content-Type: application/xml;charset=UTF-8
+.
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow-app xmlns="uri:oozie:workflow:0.3" name="test">
+    <start to="shell"/>
+    <action name="shell">
+        <shell xmlns="uri:oozie:shell-action:0.3">
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <exec>script.sh</exec>
+            <argument></argument>
+            <file>script.sh</file>
+            <capture-output/>
+        </shell>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+    <kill name="fail">
+        <message>failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
+    </kill>
+    <end name="end"/>
+</workflow-app>
+</verbatim>
+
+*Response:*
+
+<verbatim>
+HTTP/1.1 200 OK
+Content-Type: application/json;charset=UTF-8
+.
+{
+  validate: "Valid workflow-app"
+}
+</verbatim>
+
+---++++ Validate a file in HDFS
+
+You can validate a workflow, coordinator, bundle XML file in HDFS. The XML file must already exist in HDFS.
+
+*Request:*
+
+<verbatim>
+POST /oozie/v2/validate?file=hdfs://localhost:8020/user/test/myApp/workflow.xml
+Content-Type: application/xml;charset=UTF-8
+.
+</verbatim>
+
+*Response:*
+
+<verbatim>
+HTTP/1.1 200 OK
+Content-Type: application/json;charset=UTF-8
+.
+{
+  validate: "Valid workflow-app"
+}
+</verbatim>
+
 </noautolink>
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 8df39ce..fe8d1fc 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2159 'oozie validate' command should be moved server-side (seoeun25 via rkanter)
 OOZIE-2271 Upgrade Tomcat to 6.0.44 (rkanter)
 OOZIE-2266 Fix 'total' actions returned in coordinator job info (sai-krish via rkanter)
 OOZIE-2264 Fix coord:offset(n,"DAY") to resolve correct data set(kailongs via puru)

http://git-wip-us.apache.org/repos/asf/oozie/blob/cc94ad8e/webapp/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/webapp/src/main/webapp/WEB-INF/web.xml b/webapp/src/main/webapp/WEB-INF/web.xml
index d865d23..89f15f3 100644
--- a/webapp/src/main/webapp/WEB-INF/web.xml
+++ b/webapp/src/main/webapp/WEB-INF/web.xml
@@ -122,6 +122,13 @@
         <load-on-startup>1</load-on-startup>
     </servlet>
 
+    <servlet>
+        <servlet-name>validate</servlet-name>
+        <display-name>WS API for Workflow Applications</display-name>
+        <servlet-class>org.apache.oozie.servlet.V2ValidateServlet</servlet-class>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+
     <!-- servlet-mapping -->
     <servlet-mapping>
         <servlet-name>versions</servlet-name>
@@ -188,6 +195,11 @@
         <url-pattern>/v2/sla/*</url-pattern>
     </servlet-mapping>
 
+    <servlet-mapping>
+        <servlet-name>validate</servlet-name>
+        <url-pattern>/v2/validate</url-pattern>
+    </servlet-mapping>
+
     <!-- welcome-file -->
     <welcome-file-list>
         <welcome-file>index.jsp</welcome-file>