You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by ro...@apache.org on 2013/06/09 06:46:46 UTC

svn commit: r1491136 - in /oozie/trunk: ./ client/src/main/java/org/apache/oozie/cli/ client/src/main/java/org/apache/oozie/client/ client/src/main/java/org/apache/oozie/client/rest/ core/src/main/java/org/apache/oozie/executor/jpa/sla/ core/src/main/j...

Author: rohini
Date: Sun Jun  9 04:46:45 2013
New Revision: 1491136

URL: http://svn.apache.org/r1491136
Log:
OOZIE-1400 REST API to fetch SLA (rohini)

Added:
    oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/sla/SLASummaryGetForFilterJPAExecutor.java
    oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2SLAServlet.java
    oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV2SLAServlet.java
Modified:
    oozie/trunk/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
    oozie/trunk/client/src/main/java/org/apache/oozie/client/OozieClient.java
    oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java
    oozie/trunk/core/src/main/java/org/apache/oozie/servlet/SLAServlet.java
    oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java
    oozie/trunk/core/src/test/java/org/apache/oozie/client/TestWorkflowClient.java
    oozie/trunk/release-log.txt
    oozie/trunk/webapp/src/main/webapp/WEB-INF/web.xml

Modified: oozie/trunk/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
URL: http://svn.apache.org/viewvc/oozie/trunk/client/src/main/java/org/apache/oozie/cli/OozieCLI.java?rev=1491136&r1=1491135&r2=1491136&view=diff
==============================================================================
--- oozie/trunk/client/src/main/java/org/apache/oozie/cli/OozieCLI.java (original)
+++ oozie/trunk/client/src/main/java/org/apache/oozie/cli/OozieCLI.java Sun Jun  9 04:46:45 2013
@@ -461,7 +461,7 @@ public class OozieCLI {
         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(SLA_CMD, "", "sla operations (Supported in Oozie-2.0 or later)", createSlaOptions(), false);
+        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);
         parser.addCommand(HIVE_CMD, "-X ", "submit a hive job, everything after '-X' are pass-through parameters to hive, any '-D' "

Modified: oozie/trunk/client/src/main/java/org/apache/oozie/client/OozieClient.java
URL: http://svn.apache.org/viewvc/oozie/trunk/client/src/main/java/org/apache/oozie/client/OozieClient.java?rev=1491136&r1=1491135&r2=1491136&view=diff
==============================================================================
--- oozie/trunk/client/src/main/java/org/apache/oozie/client/OozieClient.java (original)
+++ oozie/trunk/client/src/main/java/org/apache/oozie/client/OozieClient.java Sun Jun  9 04:46:45 2013
@@ -132,6 +132,16 @@ public class OozieClient {
 
     public static final String FILTER_APPNAME = "appname";
 
+    public static final String FILTER_SLA_APPNAME = "app_name";
+
+    public static final String FILTER_SLA_ID = "id";
+
+    public static final String FILTER_SLA_PARENT_ID = "parent_id";
+
+    public static final String FILTER_SLA_NOMINAL_START = "nominal_start";
+
+    public static final String FILTER_SLA_NOMINAL_END = "nominal_end";
+
     public static final String CHANGE_VALUE_ENDTIME = "endtime";
 
     public static final String CHANGE_VALUE_PAUSETIME = "pausetime";
@@ -156,6 +166,7 @@ public class OozieClient {
     private String baseUrl;
     private String protocolUrl;
     private boolean validatedVersion = false;
+    private JSONArray supportedVersions;
     private final Map<String, String> headers = new HashMap<String, String>();
 
     private static ThreadLocal<String> USER_NAME_TL = new ThreadLocal<String>();
@@ -239,6 +250,27 @@ public class OozieClient {
         this.debugMode = debugMode;
     }
 
+    private String getBaseURLForVersion(long protocolVersion) throws OozieClientException {
+        try {
+            if (supportedVersions == null) {
+                supportedVersions = getSupportedProtocolVersions();
+            }
+            if (supportedVersions == null) {
+                throw new OozieClientException("HTTP error", "no response message");
+            }
+            if (supportedVersions.contains(protocolVersion)) {
+                return baseUrl + "v" + protocolVersion + "/";
+            }
+            else {
+                throw new OozieClientException(OozieClientException.UNSUPPORTED_VERSION, "Protocol version "
+                        + protocolVersion + " is not supported");
+            }
+        }
+        catch (IOException e) {
+            throw new OozieClientException(OozieClientException.IO_ERROR, e);
+        }
+    }
+
     /**
      * Validate that the Oozie client and server instances are protocol compatible.
      *
@@ -247,39 +279,33 @@ public class OozieClient {
     public synchronized void validateWSVersion() throws OozieClientException {
         if (!validatedVersion) {
             try {
-                URL url = new URL(baseUrl + RestConstants.VERSIONS);
-                HttpURLConnection conn = createConnection(url, "GET");
-                if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
-                    JSONArray array = (JSONArray) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
-                    if (array == null) {
-                        throw new OozieClientException("HTTP error", "no response message");
-                    }
-                    if (!array.contains(WS_PROTOCOL_VERSION) && !array.contains(WS_PROTOCOL_VERSION_1)
-                            && !array.contains(WS_PROTOCOL_VERSION_0)) {
-                        StringBuilder msg = new StringBuilder();
-                        msg.append("Supported version [").append(WS_PROTOCOL_VERSION).append(
-                                "] or less, Unsupported versions[");
-                        String separator = "";
-                        for (Object version : array) {
-                            msg.append(separator).append(version);
-                        }
-                        msg.append("]");
-                        throw new OozieClientException(OozieClientException.UNSUPPORTED_VERSION, msg.toString());
-                    }
-                    if (array.contains(WS_PROTOCOL_VERSION)) {
-                        protocolUrl = baseUrl + "v" + WS_PROTOCOL_VERSION + "/";
-                    }
-                    else if (array.contains(WS_PROTOCOL_VERSION_1)) {
-                        protocolUrl = baseUrl + "v" + WS_PROTOCOL_VERSION_1 + "/";
-                    }
-                    else {
-                        if (array.contains(WS_PROTOCOL_VERSION_0)) {
-                            protocolUrl = baseUrl + "v" + WS_PROTOCOL_VERSION_0 + "/";
-                        }
+                supportedVersions = getSupportedProtocolVersions();
+                if (supportedVersions == null) {
+                    throw new OozieClientException("HTTP error", "no response message");
+                }
+                if (!supportedVersions.contains(WS_PROTOCOL_VERSION)
+                        && !supportedVersions.contains(WS_PROTOCOL_VERSION_1)
+                        && !supportedVersions.contains(WS_PROTOCOL_VERSION_0)) {
+                    StringBuilder msg = new StringBuilder();
+                    msg.append("Supported version [").append(WS_PROTOCOL_VERSION)
+                            .append("] or less, Unsupported versions[");
+                    String separator = "";
+                    for (Object version : supportedVersions) {
+                        msg.append(separator).append(version);
                     }
+                    msg.append("]");
+                    throw new OozieClientException(OozieClientException.UNSUPPORTED_VERSION, msg.toString());
+                }
+                if (supportedVersions.contains(WS_PROTOCOL_VERSION)) {
+                    protocolUrl = baseUrl + "v" + WS_PROTOCOL_VERSION + "/";
+                }
+                else if (supportedVersions.contains(WS_PROTOCOL_VERSION_1)) {
+                    protocolUrl = baseUrl + "v" + WS_PROTOCOL_VERSION_1 + "/";
                 }
                 else {
-                    handleError(conn);
+                    if (supportedVersions.contains(WS_PROTOCOL_VERSION_0)) {
+                        protocolUrl = baseUrl + "v" + WS_PROTOCOL_VERSION_0 + "/";
+                    }
                 }
             }
             catch (IOException ex) {
@@ -289,6 +315,19 @@ public class OozieClient {
         }
     }
 
+    private JSONArray getSupportedProtocolVersions() throws IOException, OozieClientException {
+        JSONArray versions = null;
+        URL url = new URL(baseUrl + RestConstants.VERSIONS);
+        HttpURLConnection conn = createConnection(url, "GET");
+        if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
+            versions = (JSONArray) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
+        }
+        else {
+            handleError(conn);
+        }
+        return versions;
+    }
+
     /**
      * Create an empty configuration with just the {@link #USER_NAME} set to the JVM user name.
      *
@@ -351,11 +390,17 @@ public class OozieClient {
         return Collections.unmodifiableMap(headers).keySet().iterator();
     }
 
-    private URL createURL(String collection, String resource, Map<String, String> parameters) throws IOException,
-            OozieClientException {
+    private URL createURL(Long protocolVersion, String collection, String resource, Map<String, String> parameters)
+            throws IOException, OozieClientException {
         validateWSVersion();
         StringBuilder sb = new StringBuilder();
-        sb.append(protocolUrl).append(collection);
+        if (protocolVersion == null) {
+            sb.append(protocolUrl);
+        }
+        else {
+            sb.append(getBaseURLForVersion(protocolVersion));
+        }
+        sb.append(collection);
         if (resource != null && resource.length() > 0) {
             sb.append("/").append(resource);
         }
@@ -409,9 +454,15 @@ public class OozieClient {
         private final String collection;
         private final String resource;
         private final Map<String, String> params;
+        private final Long protocolVersion;
 
         public ClientCallable(String method, String collection, String resource, Map<String, String> params) {
+            this(method, null, collection, resource, params);
+        }
+
+        public ClientCallable(String method, Long protocolVersion, String collection, String resource, Map<String, String> params) {
             this.method = method;
+            this.protocolVersion = protocolVersion;
             this.collection = collection;
             this.resource = resource;
             this.params = params;
@@ -419,7 +470,7 @@ public class OozieClient {
 
         public T call() throws OozieClientException {
             try {
-                URL url = createURL(collection, resource, params);
+                URL url = createURL(protocolVersion, collection, resource, params);
                 if (validateCommand(url.toString())) {
                     if (getDebugMode() > 0) {
                         System.out.println(method + " " + url);
@@ -1283,7 +1334,7 @@ public class OozieClient {
     private class SlaInfo extends ClientCallable<Void> {
 
         SlaInfo(int start, int len, String filter) {
-            super("GET", RestConstants.SLA, "", prepareParams(RestConstants.SLA_GT_SEQUENCE_ID,
+            super("GET", WS_PROTOCOL_VERSION_1, RestConstants.SLA, "", prepareParams(RestConstants.SLA_GT_SEQUENCE_ID,
                     Integer.toString(start), RestConstants.MAX_EVENTS, Integer.toString(len),
                     RestConstants.JOBS_FILTER_PARAM, filter));
         }

Modified: oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java
URL: http://svn.apache.org/viewvc/oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java?rev=1491136&r1=1491135&r2=1491136&view=diff
==============================================================================
--- oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java (original)
+++ oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/JsonTags.java Sun Jun  9 04:46:45 2013
@@ -147,6 +147,23 @@ public interface JsonTags {
     public static final String BUNDLE_JOB_EXTERNAL_ID = "bundleExternalId";
     public static final String BUNDLE_COORDINATOR_JOBS = "bundleCoordJobs";
 
+    public static final String SLA_SUMMARY_LIST = "slaSummaryList";
+    public static final String SLA_SUMMARY_ID = "id";
+    public static final String SLA_SUMMARY_PARENT_ID = "parentId";
+    public static final String SLA_SUMMARY_APP_NAME = "appName";
+    public static final String SLA_SUMMARY_APP_TYPE = "appType";
+    public static final String SLA_SUMMARY_USER = "user";
+    public static final String SLA_SUMMARY_NOMINAL_TIME = "nominalTime";
+    public static final String SLA_SUMMARY_EXPECTED_START = "expectedStart";
+    public static final String SLA_SUMMARY_ACTUAL_START = "actualStart";
+    public static final String SLA_SUMMARY_EXPECTED_END = "expectedEnd";
+    public static final String SLA_SUMMARY_ACTUAL_END = "actualEnd";
+    public static final String SLA_SUMMARY_EXPECTED_DURATION = "expectedDuration";
+    public static final String SLA_SUMMARY_ACTUAL_DURATION = "actualDuration";
+    public static final String SLA_SUMMARY_JOB_STATUS = "jobStatus";
+    public static final String SLA_SUMMARY_SLA_STATUS = "slaStatus";
+    public static final String SLA_SUMMARY_LAST_MODIFIED = "lastModified";
+
     public static final String TO_STRING = "toString";
 
 

Added: oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/sla/SLASummaryGetForFilterJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/sla/SLASummaryGetForFilterJPAExecutor.java?rev=1491136&view=auto
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/sla/SLASummaryGetForFilterJPAExecutor.java (added)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/sla/SLASummaryGetForFilterJPAExecutor.java Sun Jun  9 04:46:45 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.executor.jpa.sla;
+
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import javax.persistence.EntityManager;
+import javax.persistence.Query;
+
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.executor.jpa.JPAExecutor;
+import org.apache.oozie.executor.jpa.JPAExecutorException;
+import org.apache.oozie.sla.SLASummaryBean;
+
+/**
+ * Load the list of SLASummaryBean (for dashboard) and return the list.
+ */
+public class SLASummaryGetForFilterJPAExecutor implements JPAExecutor<List<SLASummaryBean>> {
+
+    private static final String selectStr = "SELECT OBJECT(s) FROM SLASummaryBean s WHERE ";
+
+    private SLASummaryFilter filter;
+    private int numMaxResults;
+
+
+    public SLASummaryGetForFilterJPAExecutor(SLASummaryFilter filter, int numMaxResults) {
+        this.filter = filter;
+        this.numMaxResults = numMaxResults;
+    }
+
+    @Override
+    public String getName() {
+        return "SLASummaryGetForFilterJPAExecutor";
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public List<SLASummaryBean> execute(EntityManager em) throws JPAExecutorException {
+        List<SLASummaryBean> ssBean;
+        StringBuilder sb = new StringBuilder(selectStr);
+        Map<String, Object> queryParams = new LinkedHashMap<String, Object>();
+        boolean firstCondition = true;
+        if (filter.getJobId() != null) {
+            firstCondition = false;
+            if (filter.getParentId() != null) {
+                sb.append("(s.jobId = :jobId OR s.parentId = :parentId)");
+                queryParams.put("jobId", filter.getJobId());
+                queryParams.put("parentId", filter.getParentId());
+            }
+            else {
+                sb.append("s.jobId = :jobId");
+                queryParams.put("jobId", filter.getJobId());
+            }
+        }
+        if (filter.getParentId() != null && filter.getJobId() == null) {
+            firstCondition = false;
+            sb.append("s.parentId = :parentId");
+            queryParams.put("parentId", filter.getParentId());
+        }
+        if (filter.getAppName() != null && filter.getJobId() == null && filter.getParentId() == null) {
+            firstCondition = false;
+            sb.append("s.appName = :appName");
+            queryParams.put("appName", filter.getAppName());
+        }
+        if (filter.getNominalStart() != null) {
+            if (firstCondition) {
+                firstCondition = false;
+            }
+            else {
+                sb.append(" AND ");
+            }
+            sb.append("s.nominalTimeTS >= :nominalTimeStart");
+            queryParams.put("nominalTimeStart", new Timestamp(filter.getNominalStart().getTime()));
+        }
+
+        if (filter.getNominalEnd() != null) {
+            if (firstCondition) {
+                firstCondition = false;
+            }
+            else {
+                sb.append(" AND ");
+            }
+            sb.append("s.nominalTimeTS <= :nominalTimeEnd");
+            queryParams.put("nominalTimeEnd", new Timestamp(filter.getNominalEnd().getTime()));
+        }
+
+        sb.append(" ORDER BY s.nominalTimeTS");
+        try {
+            Query q = em.createQuery(sb.toString());
+            for (Map.Entry<String, Object> entry : queryParams.entrySet()) {
+                q.setParameter(entry.getKey(), entry.getValue());
+            }
+            q.setMaxResults(numMaxResults);
+            ssBean = (List<SLASummaryBean>) q.getResultList();
+        }
+        catch (Exception e) {
+            throw new JPAExecutorException(ErrorCode.E0603, e.getMessage(), e);
+        }
+        return ssBean;
+    }
+
+    public static class SLASummaryFilter {
+
+        private String appName;
+        private String jobId;
+        private String parentId;
+        private Date nominalStart;
+        private Date nominalEnd;
+
+        public SLASummaryFilter() {
+        }
+
+        public String getAppName() {
+            return appName;
+        }
+
+        public void setAppName(String appName) {
+            this.appName = appName;
+        }
+
+        public String getJobId() {
+            return jobId;
+        }
+
+        public void setJobId(String jobId) {
+            this.jobId = jobId;
+        }
+
+        public String getParentId() {
+            return parentId;
+        }
+
+        public void setParentId(String parentId) {
+            this.parentId = parentId;
+        }
+
+        public Date getNominalStart() {
+            return nominalStart;
+        }
+
+        public void setNominalStart(Date nominalStart) {
+            this.nominalStart = nominalStart;
+        }
+
+        public Date getNominalEnd() {
+            return nominalEnd;
+        }
+
+        public void setNominalEnd(Date nominalEnd) {
+            this.nominalEnd = nominalEnd;
+        }
+
+    }
+
+}

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/servlet/SLAServlet.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/SLAServlet.java?rev=1491136&r1=1491135&r2=1491136&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/servlet/SLAServlet.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/servlet/SLAServlet.java Sun Jun  9 04:46:45 2013
@@ -66,6 +66,10 @@ public class SLAServlet extends JsonRest
         super(INSTRUMENTATION_NAME, RESOURCES_INFO);
     }
 
+    public SLAServlet(String instrumentationName, ResourceInfo... resourcesInfo) {
+        super(INSTRUMENTATION_NAME, resourcesInfo);
+    }
+
     /**
      * Return information about SLA Events.
      */
@@ -79,7 +83,7 @@ public class SLAServlet extends JsonRest
             String gtSequenceNum = request.getParameter(RestConstants.SLA_GT_SEQUENCE_ID);
             String strMaxEvents = request.getParameter(RestConstants.MAX_EVENTS);
             String filter = request.getParameter(RestConstants.JOBS_FILTER_PARAM);
-            Map<String, List<String>> filterList = parseFilter(filter);
+            Map<String, List<String>> filterList = parseFilter(filter, SLA_FILTER_NAMES);
 
             int maxNoEvents = 100; // Default
             XLog.getLog(getClass()).debug(
@@ -127,10 +131,10 @@ public class SLAServlet extends JsonRest
         }
     }
 
-    protected Map<String, List<String>> parseFilter(String filter) throws ServletException {
+    protected Map<String, List<String>> parseFilter(String filterString, Set<String> allowedFilters) throws ServletException {
         Map<String, List<String>> map = new HashMap<String, List<String>>();
-        if (filter != null) {
-            StringTokenizer st = new StringTokenizer(filter, ";");
+        if (filterString != null) {
+            StringTokenizer st = new StringTokenizer(filterString, ";");
             while (st.hasMoreTokens()) {
                 String token = st.nextToken();
                 if (token.contains("=")) {
@@ -139,7 +143,7 @@ public class SLAServlet extends JsonRest
                         throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0401,
                                 "elements must be name=value pairs");
                     }
-                    if (!SLA_FILTER_NAMES.contains(pair[0])) {
+                    if (!allowedFilters.contains(pair[0])) {
                         throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0401,
                                 "invalid/unsupported names in filter");
                     }

Added: oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2SLAServlet.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2SLAServlet.java?rev=1491136&view=auto
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2SLAServlet.java (added)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2SLAServlet.java Sun Jun  9 04:46:45 2013
@@ -0,0 +1,155 @@
+/**
+ * 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 java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.XException;
+import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.command.CommandException;
+import org.apache.oozie.executor.jpa.sla.SLASummaryGetForFilterJPAExecutor;
+import org.apache.oozie.executor.jpa.sla.SLASummaryGetForFilterJPAExecutor.SLASummaryFilter;
+import org.apache.oozie.service.JPAService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.sla.SLASummaryBean;
+import org.apache.oozie.util.XLog;
+import org.json.simple.JSONObject;
+
+@SuppressWarnings("serial")
+public class V2SLAServlet extends SLAServlet {
+
+    private static final String INSTRUMENTATION_NAME = "v2sla";
+    private static final JsonRestServlet.ResourceInfo RESOURCES_INFO[] = new JsonRestServlet.ResourceInfo[1];
+    private static final Set<String> SLA_FILTER_NAMES = new HashSet<String>();
+
+    static {
+        SLA_FILTER_NAMES.add(OozieClient.FILTER_SLA_ID);
+        SLA_FILTER_NAMES.add(OozieClient.FILTER_SLA_PARENT_ID);
+        SLA_FILTER_NAMES.add(OozieClient.FILTER_SLA_APPNAME);
+        SLA_FILTER_NAMES.add(OozieClient.FILTER_SLA_NOMINAL_START);
+        SLA_FILTER_NAMES.add(OozieClient.FILTER_SLA_NOMINAL_END);
+    }
+
+    static {
+        RESOURCES_INFO[0] = new JsonRestServlet.ResourceInfo("", Arrays.asList("GET"),
+                Arrays.asList(new JsonRestServlet.ParameterInfo(RestConstants.JOBS_FILTER_PARAM, String.class, false,
+                        Arrays.asList("GET"))));
+    }
+
+    public V2SLAServlet() {
+        super(INSTRUMENTATION_NAME, RESOURCES_INFO);
+    }
+
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
+
+        XLog.getLog(getClass()).debug("Got SLA GET request:" + request.getQueryString());
+        try {
+            stopCron();
+            JSONObject json = getSLASummaryList(request, response);
+            startCron();
+            if (json == null) {
+                response.setStatus(HttpServletResponse.SC_OK);
+            }
+            else {
+                sendJsonResponse(response, HttpServletResponse.SC_OK, json);
+            }
+        }
+        catch (CommandException ce) {
+            ce.printStackTrace();
+            XLog.getLog(getClass()).error("Command exception ", ce);
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ce);
+        }
+        catch (RuntimeException re) {
+            re.printStackTrace();
+            XLog.getLog(getClass()).error("Runtime error ", re);
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0307, re.getMessage());
+        }
+    }
+
+    private JSONObject getSLASummaryList(HttpServletRequest request, HttpServletResponse response)
+            throws ServletException, CommandException {
+        String timeZoneId = request.getParameter(RestConstants.TIME_ZONE_PARAM) == null ? null : request
+                .getParameter(RestConstants.TIME_ZONE_PARAM);
+        String filterString = request.getParameter(RestConstants.JOBS_FILTER_PARAM);
+        String maxResults = request.getParameter(RestConstants.LEN_PARAM);
+        int numMaxResults = 1000; // Default
+
+        if (maxResults != null) {
+            numMaxResults = Integer.parseInt(maxResults);
+        }
+
+        try {
+            Map<String, List<String>> filterList = parseFilter(URLDecoder.decode(filterString, "UTF-8"), SLA_FILTER_NAMES);
+            SLASummaryFilter filter = new SLASummaryFilter();
+            if (filterList.containsKey(OozieClient.FILTER_SLA_ID)) {
+                filter.setJobId(filterList.get(OozieClient.FILTER_SLA_ID).get(0));
+            }
+            if (filterList.containsKey(OozieClient.FILTER_SLA_PARENT_ID)) {
+                filter.setParentId(filterList.get(OozieClient.FILTER_SLA_PARENT_ID).get(0));
+            }
+            if (filterList.containsKey(OozieClient.FILTER_SLA_APPNAME)) {
+                filter.setAppName(filterList.get(OozieClient.FILTER_SLA_APPNAME).get(0));
+            }
+            if (filterList.containsKey(OozieClient.FILTER_SLA_NOMINAL_START)) {
+                filter.setNominalStart(new Date(Long.parseLong(filterList.get(OozieClient.FILTER_SLA_NOMINAL_START).get(0))));
+            }
+            if (filterList.containsKey(OozieClient.FILTER_SLA_NOMINAL_END)) {
+                filter.setNominalEnd(new Date(Long.parseLong(filterList.get(OozieClient.FILTER_SLA_NOMINAL_END).get(0))));
+            }
+
+            if (filter.getAppName() == null && filter.getJobId() == null && filter.getParentId() == null) {
+                throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0305,
+                        "At least one of the filter parameters - " + OozieClient.FILTER_SLA_ID + ","
+                                + OozieClient.FILTER_SLA_PARENT_ID + " or " + OozieClient.FILTER_SLA_APPNAME
+                                + " should be specified in the filter query parameter");
+            }
+
+            JPAService jpaService = Services.get().get(JPAService.class);
+            List<SLASummaryBean> slaSummaryList = null;
+            if (jpaService != null) {
+                slaSummaryList = jpaService.execute(new SLASummaryGetForFilterJPAExecutor(filter, numMaxResults));
+            }
+            else {
+                XLog.getLog(getClass()).error(ErrorCode.E0610);
+            }
+            return SLASummaryBean.toJSONObject(slaSummaryList, timeZoneId);
+        }
+        catch (XException ex) {
+            throw new CommandException(ex);
+        }
+        catch (UnsupportedEncodingException e) {
+            throw new XServletException(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, ErrorCode.E0307,
+                    "Unsupported Encoding", e);
+        }
+    }
+
+}

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java?rev=1491136&r1=1491135&r2=1491136&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java Sun Jun  9 04:46:45 2013
@@ -19,6 +19,7 @@ package org.apache.oozie.sla;
 
 import java.sql.Timestamp;
 import java.util.Date;
+import java.util.List;
 
 import javax.persistence.Basic;
 import javax.persistence.Column;
@@ -28,10 +29,14 @@ import javax.persistence.NamedQueries;
 import javax.persistence.NamedQuery;
 import javax.persistence.Table;
 
+import org.apache.oozie.AppType;
 import org.apache.oozie.client.event.SLAEvent;
 import org.apache.oozie.client.rest.JsonBean;
+import org.apache.oozie.client.rest.JsonTags;
+import org.apache.oozie.client.rest.JsonUtils;
 import org.apache.oozie.util.DateUtils;
 import org.apache.openjpa.persistence.jdbc.Index;
+import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 
 @Entity
@@ -49,8 +54,9 @@ public class SLASummaryBean implements J
     private String jobId;
 
     @Basic
-    @Column(name = "user")
-    private String user;
+    @Index
+    @Column(name = "parent_id")
+    private String parentId;
 
     @Basic
     @Index
@@ -58,9 +64,12 @@ public class SLASummaryBean implements J
     private String appName;
 
     @Basic
-    @Index
-    @Column(name = "parent_id")
-    private String parentId;
+    @Column(name = "app_type")
+    private String appType;
+
+    @Basic
+    @Column(name = "user")
+    private String user;
 
     @Basic
     @Index
@@ -245,6 +254,14 @@ public class SLASummaryBean implements J
         this.appName = appName;
     }
 
+    public AppType getAppType() {
+        return AppType.valueOf(appType);
+    }
+
+    public void setAppType(AppType appType) {
+        this.appType = appType.toString();
+    }
+
     public byte getSlaProcessed() {
         return slaProcessed;
     }
@@ -261,16 +278,89 @@ public class SLASummaryBean implements J
         this.lastModifiedTS = DateUtils.convertDateToTimestamp(lastModified);
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public JSONObject toJSONObject() {
-        // TODO Auto-generated method stub
-        return null;
+        JSONObject json = new JSONObject();
+        json.put(JsonTags.SLA_SUMMARY_ID, jobId);
+        if (parentId != null) {
+            json.put(JsonTags.SLA_SUMMARY_PARENT_ID, parentId);
+        }
+        json.put(JsonTags.SLA_SUMMARY_APP_NAME, appName);
+        json.put(JsonTags.SLA_SUMMARY_APP_TYPE, appType);
+        json.put(JsonTags.SLA_SUMMARY_USER, user);
+        json.put(JsonTags.SLA_SUMMARY_NOMINAL_TIME, nominalTimeTS.getTime());
+        if (expectedStartTS != null) {
+            json.put(JsonTags.SLA_SUMMARY_EXPECTED_START, expectedStartTS.getTime());
+        }
+        if (actualStartTS != null) {
+            json.put(JsonTags.SLA_SUMMARY_ACTUAL_START, actualStartTS.getTime());
+        }
+        json.put(JsonTags.SLA_SUMMARY_EXPECTED_END, expectedEndTS.getTime());
+        if (actualEndTS != null) {
+            json.put(JsonTags.SLA_SUMMARY_ACTUAL_END, actualEndTS.getTime());
+        }
+        json.put(JsonTags.SLA_SUMMARY_EXPECTED_DURATION, expectedDuration);
+        json.put(JsonTags.SLA_SUMMARY_ACTUAL_DURATION, actualDuration);
+        json.put(JsonTags.SLA_SUMMARY_JOB_STATUS, jobStatus);
+        json.put(JsonTags.SLA_SUMMARY_SLA_STATUS, slaStatus);
+        json.put(JsonTags.SLA_SUMMARY_LAST_MODIFIED, lastModifiedTS.getTime());
+        return json;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public JSONObject toJSONObject(String timeZoneId) {
-        // TODO Auto-generated method stub
-        return null;
+        if (timeZoneId == null) {
+            return toJSONObject();
+        }
+        else {
+            JSONObject json = new JSONObject();
+            json.put(JsonTags.SLA_SUMMARY_ID, jobId);
+            if (parentId != null) {
+                json.put(JsonTags.SLA_SUMMARY_PARENT_ID, parentId);
+            }
+            json.put(JsonTags.SLA_SUMMARY_APP_NAME, appName);
+            json.put(JsonTags.SLA_SUMMARY_APP_TYPE, appType);
+            json.put(JsonTags.SLA_SUMMARY_USER, user);
+            json.put(JsonTags.SLA_SUMMARY_NOMINAL_TIME, JsonUtils.formatDateRfc822(nominalTimeTS, timeZoneId));
+            if (expectedStartTS != null) {
+                json.put(JsonTags.SLA_SUMMARY_EXPECTED_START, JsonUtils.formatDateRfc822(expectedStartTS, timeZoneId));
+            }
+            if (actualStartTS != null) {
+                json.put(JsonTags.SLA_SUMMARY_ACTUAL_START, JsonUtils.formatDateRfc822(actualStartTS, timeZoneId));
+            }
+            json.put(JsonTags.SLA_SUMMARY_EXPECTED_END, JsonUtils.formatDateRfc822(expectedEndTS, timeZoneId));
+            if (actualEndTS != null) {
+                json.put(JsonTags.SLA_SUMMARY_ACTUAL_END, JsonUtils.formatDateRfc822(actualEndTS, timeZoneId));
+            }
+            json.put(JsonTags.SLA_SUMMARY_EXPECTED_DURATION, expectedDuration);
+            json.put(JsonTags.SLA_SUMMARY_ACTUAL_DURATION, actualDuration);
+            json.put(JsonTags.SLA_SUMMARY_JOB_STATUS, jobStatus);
+            json.put(JsonTags.SLA_SUMMARY_SLA_STATUS, slaStatus);
+            json.put(JsonTags.SLA_SUMMARY_LAST_MODIFIED, JsonUtils.formatDateRfc822(lastModifiedTS, timeZoneId));
+            return json;
+        }
+    }
+
+    /**
+     * Convert a sla summary list into a json object.
+     *
+     * @param slaSummaryList sla summary list.
+     * @param timeZoneId time zone to use for dates in the JSON array.
+     * @return the corresponding JSON object.
+     */
+    @SuppressWarnings("unchecked")
+    public static JSONObject toJSONObject(List<? extends SLASummaryBean> slaSummaryList, String timeZoneId) {
+        JSONObject json = new JSONObject();
+        JSONArray array = new JSONArray();
+        if (slaSummaryList != null) {
+            for (SLASummaryBean summary : slaSummaryList) {
+                array.add(summary.toJSONObject(timeZoneId));
+            }
+        }
+        json.put(JsonTags.SLA_SUMMARY_LIST, array);
+        return json;
     }
 
 }

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/client/TestWorkflowClient.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/client/TestWorkflowClient.java?rev=1491136&r1=1491135&r2=1491136&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/client/TestWorkflowClient.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/client/TestWorkflowClient.java Sun Jun  9 04:46:45 2013
@@ -35,6 +35,7 @@ import org.apache.oozie.servlet.V1JobSer
 import org.apache.oozie.servlet.V1JobsServlet;
 import org.apache.oozie.servlet.V2AdminServlet;
 import org.apache.oozie.servlet.V2JobServlet;
+import org.apache.oozie.servlet.V2SLAServlet;
 import org.json.simple.JSONArray;
 
 import java.io.*;
@@ -48,24 +49,32 @@ public class TestWorkflowClient extends 
 
     static {
         new HeaderTestingVersionServlet();
-        new V0JobServlet();
         new V0JobsServlet();
         new V1JobsServlet();
-        new V1AdminServlet();
+        new V0JobServlet();
         new V1JobServlet();
         new V2JobServlet();
+        new V1AdminServlet();
         new V2AdminServlet();
         new SLAServlet();
+        new V2SLAServlet();
     }
 
     private 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/*",
-            VERSION + "/sla/*" };
+    static final String VERSION_0 = "/v" + OozieClient.WS_PROTOCOL_VERSION_0;
+    static final String VERSION_1 = "/v" + OozieClient.WS_PROTOCOL_VERSION_1;
+    static final String VERSION_2 = "/v" + OozieClient.WS_PROTOCOL_VERSION;
+    static final String[] END_POINTS = { "/versions",
+            VERSION_0 + "/jobs", VERSION_1 + "/jobs", VERSION_2 + "/jobs",
+            VERSION_0 + "/job/*", VERSION_1 + "/job/*", VERSION_2 + "/job/*",
+            VERSION_1 + "/admin/*", VERSION_2 + "/admin/*",
+            VERSION_1 + "/sla/*", VERSION_2 + "/sla/*" };
     @SuppressWarnings("rawtypes")
-    static final Class[] SERVLET_CLASSES = {HeaderTestingVersionServlet.class, V0JobsServlet.class,
-            V0JobServlet.class, V1AdminServlet.class, SLAServlet.class, V2AdminServlet.class,  V1JobServlet.class,
-            V2JobServlet.class, V1JobsServlet.class};
+    static final Class[] SERVLET_CLASSES = {HeaderTestingVersionServlet.class,
+            V0JobsServlet.class, V1JobsServlet.class, V1JobsServlet.class,
+            V0JobServlet.class, V1JobServlet.class, V2JobServlet.class,
+            V1AdminServlet.class, V2AdminServlet.class,
+            SLAServlet.class, V2SLAServlet.class};
 
     protected void setUp() throws Exception {
         super.setUp();

Added: oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV2SLAServlet.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV2SLAServlet.java?rev=1491136&view=auto
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV2SLAServlet.java (added)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV2SLAServlet.java Sun Jun  9 04:46:45 2013
@@ -0,0 +1,197 @@
+/**
+ * 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 java.io.InputStreamReader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.oozie.AppType;
+import org.apache.oozie.client.event.SLAEvent.EventStatus;
+import org.apache.oozie.client.event.SLAEvent.SLAStatus;
+import org.apache.oozie.client.rest.JsonBean;
+import org.apache.oozie.client.rest.JsonTags;
+import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.executor.jpa.JPAExecutorException;
+import org.apache.oozie.executor.jpa.sla.SLACalculationInsertUpdateJPAExecutor;
+import org.apache.oozie.service.JPAService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.sla.SLASummaryBean;
+import org.apache.oozie.util.DateUtils;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+import org.json.simple.JSONValue;
+
+public class TestV2SLAServlet extends DagServletTestCase {
+
+    private static final boolean IS_SECURITY_ENABLED = false;
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+    }
+
+    public void testSLA() throws Exception {
+        runTest("/v2/sla", V2SLAServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            public Void call() throws Exception {
+
+                final Date currentTime = new Date(System.currentTimeMillis());
+                final Date nominalTime1 = DateUtils.parseDateUTC("2012-06-01T010:00Z");
+                final Date nominalTime2 = DateUtils.parseDateUTC("2012-06-02T010:20Z");
+                final Date nominalTime3 = DateUtils.parseDateUTC("2012-06-03T014:00Z");
+                insertEntriesIntoSLASummaryTable(2, "1-", "-W", "1-C", nominalTime1, "testapp-1", AppType.WORKFLOW_JOB,
+                        currentTime);
+                insertEntriesIntoSLASummaryTable(3, "2-", "-W", null, nominalTime2, "testapp-2", AppType.WORKFLOW_JOB,
+                        currentTime);
+                insertEntriesIntoSLASummaryTable(6, "3-", "-W", "2-C", nominalTime3, "testapp-3", AppType.WORKFLOW_JOB,
+                        currentTime);
+
+                Map<String, String> queryParams = new HashMap<String, String>();
+                JSONArray array = null;
+
+                URL url = createURL("", queryParams);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("GET");
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+
+                queryParams.put(RestConstants.JOBS_FILTER_PARAM, "app_name=testapp-1");
+                array = getSLAJSONResponse(queryParams);
+                // Matches first two - 1-1-W and 1-2-W
+                assertSLAJSONResponse(array, 1, 2, "1-", "-W", "1-C", nominalTime1, "testapp-1", AppType.WORKFLOW_JOB,
+                        currentTime);
+
+                queryParams.put(RestConstants.JOBS_FILTER_PARAM, "app_name=testapp-2;id=2-2-W");
+                array = getSLAJSONResponse(queryParams);
+                // Matches second element - 2-2-W
+                assertSLAJSONResponse(array, 2, 2, "2-", "-W", null, nominalTime2, "testapp-2", AppType.WORKFLOW_JOB,
+                        currentTime);
+
+                queryParams.put(RestConstants.JOBS_FILTER_PARAM, "app_name=testapp-3;nominal_start="
+                        + DateUtils.parseDateUTC("2012-06-03T016:00Z").getTime());
+                array = getSLAJSONResponse(queryParams);
+                // Matches 3-6 elements - 3-3-W 3-4-W 3-5-W 3-6-W
+                assertSLAJSONResponse(array, 3, 6, "3-", "-W", "2-C", nominalTime3, "testapp-3", AppType.WORKFLOW_JOB,
+                        currentTime);
+
+                queryParams.put(RestConstants.JOBS_FILTER_PARAM, "parent_id=2-C;nominal_start="
+                        + DateUtils.parseDateUTC("2012-06-03T016:00Z").getTime() + ";nominal_end="
+                        + DateUtils.parseDateUTC("2012-06-03T017:00Z").getTime());
+                array = getSLAJSONResponse(queryParams);
+                // Matches 3rd and 4th element - 3-3-W 3-4-W
+                assertSLAJSONResponse(array, 3, 4, "3-", "-W", "2-C", nominalTime3, "testapp-3", AppType.WORKFLOW_JOB,
+                        currentTime);
+
+                return null;
+            }
+        });
+    }
+
+    private JSONArray getSLAJSONResponse(Map<String, String> queryParams) throws Exception {
+        URL url = createURL("", queryParams);
+        HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+        conn.setRequestMethod("GET");
+        assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+        assertTrue(conn.getHeaderField("content-type").startsWith(RestConstants.JSON_CONTENT_TYPE));
+        JSONObject json = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
+        JSONArray array = (JSONArray) json.get(JsonTags.SLA_SUMMARY_LIST);
+        return array;
+    }
+
+    private void assertSLAJSONResponse(JSONArray array, int startRange, int endRange, String jobIDPrefix,
+            String jobIDSuffix, String parentId, Date startNominalTime, String appName, AppType appType,
+            Date currentTime) throws Exception {
+        Calendar nominalTime = Calendar.getInstance();
+        nominalTime.setTime(startNominalTime);
+        nominalTime.add(Calendar.HOUR, (startRange - 1));
+        int index = 0;
+        assertEquals(endRange - (startRange - 1), array.size());
+        for (int i = startRange; i <= endRange; i++) {
+            Calendar actualStart = (Calendar) nominalTime.clone();
+            actualStart.add(Calendar.MINUTE, i);
+            Calendar expectedEnd = (Calendar) nominalTime.clone();
+            expectedEnd.add(Calendar.MINUTE, 60);
+            Calendar actualEnd = (Calendar) expectedEnd.clone();
+            actualEnd.add(Calendar.MINUTE, i);
+            JSONObject json = (JSONObject) array.get(index++);
+            assertEquals(jobIDPrefix + i + jobIDSuffix, json.get(JsonTags.SLA_SUMMARY_ID));
+            assertEquals(parentId, json.get(JsonTags.SLA_SUMMARY_PARENT_ID));
+            assertEquals(appName, json.get(JsonTags.SLA_SUMMARY_APP_NAME));
+            assertEquals(appType.name(), json.get(JsonTags.SLA_SUMMARY_APP_TYPE));
+            assertEquals("RUNNING", json.get(JsonTags.SLA_SUMMARY_JOB_STATUS));
+            assertEquals(SLAStatus.IN_PROCESS.name(), json.get(JsonTags.SLA_SUMMARY_SLA_STATUS));
+            assertEquals(nominalTime.getTimeInMillis(), json.get(JsonTags.SLA_SUMMARY_NOMINAL_TIME));
+            assertEquals(nominalTime.getTimeInMillis(), json.get(JsonTags.SLA_SUMMARY_EXPECTED_START));
+            assertEquals(actualStart.getTimeInMillis(), json.get(JsonTags.SLA_SUMMARY_ACTUAL_START));
+            assertEquals(expectedEnd.getTimeInMillis(), json.get(JsonTags.SLA_SUMMARY_EXPECTED_END));
+            assertEquals(actualEnd.getTimeInMillis(), json.get(JsonTags.SLA_SUMMARY_ACTUAL_END));
+            assertEquals(10L, json.get(JsonTags.SLA_SUMMARY_EXPECTED_DURATION));
+            assertEquals(15L, json.get(JsonTags.SLA_SUMMARY_ACTUAL_DURATION));
+            assertEquals(currentTime.getTime(), json.get(JsonTags.SLA_SUMMARY_LAST_MODIFIED));
+            nominalTime.add(Calendar.HOUR, 1);
+        }
+    }
+
+    private void insertEntriesIntoSLASummaryTable(int numEntries, String jobIDPrefix, String jobIDSuffix,
+            String parentId, Date startNominalTime, String appName, AppType appType, Date currentTime)
+            throws JPAExecutorException {
+        List<JsonBean> list = new ArrayList<JsonBean>();
+        Calendar nominalTime = Calendar.getInstance();
+        nominalTime.setTime(startNominalTime);
+        for (int i = 1; i <= numEntries; i++) {
+            Calendar actualStart = (Calendar) nominalTime.clone();
+            actualStart.add(Calendar.MINUTE, i);
+            Calendar expectedEnd = (Calendar) nominalTime.clone();
+            expectedEnd.add(Calendar.MINUTE, 60);
+            Calendar actualEnd = (Calendar) expectedEnd.clone();
+            actualEnd.add(Calendar.MINUTE, i);
+            SLASummaryBean bean = new SLASummaryBean();
+            bean.setJobId(jobIDPrefix + i + jobIDSuffix);
+            bean.setParentId(parentId);
+            bean.setAppName(appName);
+            bean.setAppType(appType);
+            bean.setJobStatus("RUNNING");
+            bean.setEventStatus(EventStatus.END_MISS);
+            bean.setSLAStatus(SLAStatus.IN_PROCESS);
+            bean.setNominalTime(nominalTime.getTime());
+            bean.setExpectedStart(nominalTime.getTime());
+            bean.setActualStart(actualStart.getTime());
+            bean.setExpectedEnd(expectedEnd.getTime());
+            bean.setActualEnd(actualEnd.getTime());
+            bean.setExpectedDuration(10);
+            bean.setActualDuration(15);
+            bean.setSlaProcessed((byte) 1);
+            bean.setUser("testuser");
+            bean.setLastModifiedTime(currentTime);
+            list.add(bean);
+            nominalTime.add(Calendar.HOUR, 1);
+        }
+
+        SLACalculationInsertUpdateJPAExecutor writeCmd = new SLACalculationInsertUpdateJPAExecutor(list, null);
+        JPAService jpaService = Services.get().get(JPAService.class);
+        jpaService.execute(writeCmd);
+    }
+}

Modified: oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/trunk/release-log.txt?rev=1491136&r1=1491135&r2=1491136&view=diff
==============================================================================
--- oozie/trunk/release-log.txt (original)
+++ oozie/trunk/release-log.txt Sun Jun  9 04:46:45 2013
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1400 REST API to fetch SLA (rohini)
 OOZIE-1375 Generate Job notification events for Workflow Actions (mona)
 OOZIE-1357 Can't view more than 1000 actions of a coordinator and paging does not work (ryota)
 OOZIE-1381 Oozie does not support access to the distributed cache file under different name node (ryota)

Modified: oozie/trunk/webapp/src/main/webapp/WEB-INF/web.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/webapp/src/main/webapp/WEB-INF/web.xml?rev=1491136&r1=1491135&r2=1491136&view=diff
==============================================================================
--- oozie/trunk/webapp/src/main/webapp/WEB-INF/web.xml (original)
+++ oozie/trunk/webapp/src/main/webapp/WEB-INF/web.xml Sun Jun  9 04:46:45 2013
@@ -104,6 +104,13 @@
         <load-on-startup>1</load-on-startup>
     </servlet>
 
+    <servlet>
+        <servlet-name>v2sla</servlet-name>
+        <display-name>WS API for specific SLA Events</display-name>
+        <servlet-class>org.apache.oozie.servlet.V2SLAServlet</servlet-class>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+
 	<!-- servlet-mapping -->
     <servlet-mapping>
         <servlet-name>versions</servlet-name>
@@ -166,7 +173,7 @@
     </servlet-mapping>
 
     <servlet-mapping>
-        <servlet-name>sla-event</servlet-name>
+        <servlet-name>v2sla</servlet-name>
         <url-pattern>/v2/sla/*</url-pattern>
     </servlet-mapping>