You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2013/09/17 23:56:44 UTC

svn commit: r1524233 - in /hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton: JobItemBean.java Main.java Server.java

Author: thejas
Date: Tue Sep 17 21:56:44 2013
New Revision: 1524233

URL: http://svn.apache.org/r1524233
Log:
HIVE-4443: [HCatalog] Have an option for GET queue to return all job information in single call (Daniel Dai via Thejas Nair)

Added:
    hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JobItemBean.java
Modified:
    hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java
    hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java

Added: hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JobItemBean.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JobItemBean.java?rev=1524233&view=auto
==============================================================================
--- hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JobItemBean.java (added)
+++ hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JobItemBean.java Tue Sep 17 21:56:44 2013
@@ -0,0 +1,38 @@
+/**
+ * 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.hive.hcatalog.templeton;
+
+public class JobItemBean {
+  public String id;
+  public QueueStatusBean detail;
+
+  public JobItemBean() {
+  }
+
+  /**
+   * Create a new JobItemBean
+   *
+   * @param id         job id
+   * @param detail     job detail
+   */
+  public JobItemBean(String id, QueueStatusBean detail) {
+    this.id = id;
+    this.detail = detail;
+  }
+}

Modified: hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java?rev=1524233&r1=1524232&r2=1524233&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java (original)
+++ hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java Tue Sep 17 21:56:44 2013
@@ -176,6 +176,8 @@ public class Main {
              FilterMapping.REQUEST);
     root.addFilter(fHolder, "/" + SERVLET_PATH + "/v1/queue/*", 
              FilterMapping.REQUEST);
+    root.addFilter(fHolder, "/" + SERVLET_PATH + "/v1/jobs/*",
+             FilterMapping.REQUEST);
     root.addFilter(fHolder, "/" + SERVLET_PATH + "/v1/mapreduce/*", 
              FilterMapping.REQUEST);
     root.addFilter(fHolder, "/" + SERVLET_PATH + "/v1/status/*", 

Modified: hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java?rev=1524233&r1=1524232&r2=1524233&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java (original)
+++ hive/branches/branch-0.12/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java Tue Sep 17 21:56:44 2013
@@ -681,12 +681,55 @@ public class Server {
 
   /**
    * Return the status of the jobid.
+   * @deprecated use GET jobs/{jobid} instead.
    */
+  @Deprecated
   @GET
   @Path("queue/{jobid}")
   @Produces({MediaType.APPLICATION_JSON})
   public QueueStatusBean showQueueId(@PathParam("jobid") String jobid)
     throws NotAuthorizedException, BadParam, IOException, InterruptedException {
+    return showJobId(jobid);
+  }
+
+  /**
+   * Kill a job in the queue.
+   * @deprecated use DELETE jobs/{jobid} instead.
+   */
+  @Deprecated
+  @DELETE
+  @Path("queue/{jobid}")
+  @Produces({MediaType.APPLICATION_JSON})
+  public QueueStatusBean deleteQueueId(@PathParam("jobid") String jobid)
+    throws NotAuthorizedException, BadParam, IOException, InterruptedException {
+    return deleteJobId(jobid);
+  }
+
+  /**
+   * Return all the known job ids for this user.
+   * @deprecated use GET jobs instead.
+   */
+  @Deprecated
+  @GET
+  @Path("queue")
+  @Produces({MediaType.APPLICATION_JSON})
+  public List<String> showQueueList(@QueryParam("showall") boolean showall)
+    throws NotAuthorizedException, BadParam, IOException, InterruptedException {
+
+    verifyUser();
+
+    ListDelegator d = new ListDelegator(appConf);
+    return d.run(getDoAsUser(), showall);
+  }
+
+  /**
+   * Return the status of the jobid.
+   */
+  @GET
+  @Path("jobs/{jobid}")
+  @Produces({MediaType.APPLICATION_JSON})
+  public QueueStatusBean showJobId(@PathParam("jobid") String jobid)
+    throws NotAuthorizedException, BadParam, IOException, InterruptedException {
 
     verifyUser();
     verifyParam(jobid, ":jobid");
@@ -699,9 +742,9 @@ public class Server {
    * Kill a job in the queue.
    */
   @DELETE
-  @Path("queue/{jobid}")
+  @Path("jobs/{jobid}")
   @Produces({MediaType.APPLICATION_JSON})
-  public QueueStatusBean deleteQueueId(@PathParam("jobid") String jobid)
+  public QueueStatusBean deleteJobId(@PathParam("jobid") String jobid)
     throws NotAuthorizedException, BadParam, IOException, InterruptedException {
 
     verifyUser();
@@ -715,15 +758,36 @@ public class Server {
    * Return all the known job ids for this user.
    */
   @GET
-  @Path("queue")
+  @Path("jobs")
   @Produces({MediaType.APPLICATION_JSON})
-  public List<String> showQueueList(@QueryParam("showall") boolean showall)
+  public List<JobItemBean> showJobList(@QueryParam("fields") String fields,
+                                       @QueryParam("showall") boolean showall)
     throws NotAuthorizedException, BadParam, IOException, InterruptedException {
 
     verifyUser();
 
-    ListDelegator d = new ListDelegator(appConf);
-    return d.run(getDoAsUser(), showall);
+    boolean showDetails = false;
+    if (fields!=null && !fields.equals("*")) {
+      throw new BadParam("fields value other than * is not supported");
+    }
+    if (fields!=null && fields.equals("*")) {
+      showDetails = true;
+    }
+
+    ListDelegator ld = new ListDelegator(appConf);
+    List<String> list = ld.run(getDoAsUser(), showall);
+    List<JobItemBean> detailList = new ArrayList<JobItemBean>();
+    for (String job : list) {
+      JobItemBean jobItem = new JobItemBean();
+      jobItem.id = job;
+      if (showDetails) {
+        StatusDelegator sd = new StatusDelegator(appConf);
+        QueueStatusBean statusBean = sd.run(getDoAsUser(), job);
+        jobItem.detail = statusBean;
+      }
+      detailList.add(jobItem);
+    }
+    return detailList;
   }
 
   /**