You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/11/30 01:12:26 UTC

[89/91] [abbrv] hive git commit: HIVE-12338: Add webui to HiveServer2 (Jimmy, reviewed by Mohit, Szehon, Lefty)

HIVE-12338: Add webui to HiveServer2 (Jimmy, reviewed by Mohit, Szehon, Lefty)


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

Branch: refs/heads/spark
Commit: 2c0c191cdd6b2d1aebe4502e24cc2b3d041bf3ca
Parents: a51e5d4
Author: Jimmy Xiang <jx...@apache.org>
Authored: Thu Nov 19 08:10:29 2015 -0800
Committer: Jimmy Xiang <jx...@apache.org>
Committed: Sun Nov 29 09:55:09 2015 -0800

----------------------------------------------------------------------
 common/pom.xml                                  |   5 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   5 +
 .../hive/http/AdminAuthorizedServlet.java       |  45 ++
 .../java/org/apache/hive/http/ConfServlet.java  | 101 +++++
 .../java/org/apache/hive/http/HttpServer.java   | 316 ++++++++++++++
 .../org/apache/hive/http/JMXJsonServlet.java    | 412 +++++++++++++++++++
 pom.xml                                         |   1 +
 ql/pom.xml                                      |   6 +
 service/pom.xml                                 |  56 +++
 .../hive/service/cli/operation/Operation.java   |   2 +-
 .../service/cli/operation/OperationManager.java |  26 +-
 .../service/cli/operation/SQLOperation.java     |   8 +-
 .../service/cli/session/HiveSessionBase.java    |   4 +
 .../service/cli/session/HiveSessionImpl.java    |  12 +
 .../service/cli/session/SessionManager.java     |  16 +-
 .../apache/hive/service/server/HiveServer2.java |  47 +++
 .../hive-webapps/hiveserver2/hiveserver2.jsp    | 186 +++++++++
 .../hive-webapps/hiveserver2/index.html         |  20 +
 .../static/css/bootstrap-theme.min.css          |  10 +
 .../hive-webapps/static/css/bootstrap.min.css   |   9 +
 .../resources/hive-webapps/static/css/hive.css  |  24 ++
 .../fonts/glyphicons-halflings-regular.eot      | Bin 0 -> 14079 bytes
 .../fonts/glyphicons-halflings-regular.svg      | 228 ++++++++++
 .../fonts/glyphicons-halflings-regular.ttf      | Bin 0 -> 29512 bytes
 .../fonts/glyphicons-halflings-regular.woff     | Bin 0 -> 16448 bytes
 .../hive-webapps/static/hive_logo.jpeg          | Bin 0 -> 5616 bytes
 spark-client/pom.xml                            |   6 +
 27 files changed, 1529 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index ee74282..72bb550 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -56,6 +56,11 @@
       <version>${commons-lang.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.eclipse.jetty.aggregate</groupId>
+      <artifactId>jetty-all</artifactId>
+      <version>${jetty.version}</version>
+    </dependency>
+    <dependency>
       <groupId>joda-time</groupId>
       <artifactId>joda-time</artifactId>
       <version>${joda.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index db942b0..9e805bd 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1847,6 +1847,11 @@ public class HiveConf extends Configuration {
     HIVE_SERVER2_PARALLEL_COMPILATION("hive.driver.parallel.compilation", false, "Whether to\n" +
         "enable parallel compilation between sessions on HiveServer2. The default is false."),
 
+    // HiveServer2 WebUI
+    HIVE_SERVER2_WEBUI_BIND_HOST("hive.server2.webui.host", "0.0.0.0", "The host address the HiveServer2 WebUI will listen on"),
+    HIVE_SERVER2_WEBUI_PORT("hive.server2.webui.port", 10002, "The port the HiveServer2 WebUI will listen on"),
+    HIVE_SERVER2_WEBUI_MAX_THREADS("hive.server2.webui.max.threads", 50, "The max HiveServer2 WebUI threads"),
+
     // Tez session settings
     HIVE_SERVER2_TEZ_DEFAULT_QUEUES("hive.server2.tez.default.queues", "",
         "A list of comma separated values corresponding to YARN queues of the same name.\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/common/src/java/org/apache/hive/http/AdminAuthorizedServlet.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/AdminAuthorizedServlet.java b/common/src/java/org/apache/hive/http/AdminAuthorizedServlet.java
new file mode 100644
index 0000000..5d957c2
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/AdminAuthorizedServlet.java
@@ -0,0 +1,45 @@
+/**
+ * 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.http;
+
+import java.io.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.eclipse.jetty.servlet.DefaultServlet;
+
+/**
+ * General servlet which is admin-authorized.
+ */
+public class AdminAuthorizedServlet extends DefaultServlet {
+
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+    throws ServletException, IOException {
+    // Do the authorization
+    if (HttpServer.hasAdministratorAccess(getServletContext(), request,
+        response)) {
+      // Authorization is done. Just call super.
+      super.doGet(request, response);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/common/src/java/org/apache/hive/http/ConfServlet.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/ConfServlet.java b/common/src/java/org/apache/hive/http/ConfServlet.java
new file mode 100644
index 0000000..253df4f
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/ConfServlet.java
@@ -0,0 +1,101 @@
+/**
+ * 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.http;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A servlet to print out the running configuration data.
+ */
+public class ConfServlet extends HttpServlet {
+  private static final long serialVersionUID = 1L;
+
+  private static final String FORMAT_JSON = "json";
+  private static final String FORMAT_XML = "xml";
+  private static final String FORMAT_PARAM = "format";
+
+  /**
+   * Return the Configuration of the daemon hosting this servlet.
+   * This is populated when the HttpServer starts.
+   */
+  private Configuration getConfFromContext() {
+    Configuration conf = (Configuration)getServletContext().getAttribute(
+        HttpServer.CONF_CONTEXT_ATTRIBUTE);
+    assert conf != null;
+    return conf;
+  }
+
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws ServletException, IOException {
+
+    if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
+                                                   request, response)) {
+      return;
+    }
+
+    String format = request.getParameter(FORMAT_PARAM);
+    if (null == format) {
+      format = FORMAT_XML;
+    }
+
+    if (FORMAT_XML.equals(format)) {
+      response.setContentType("text/xml; charset=utf-8");
+    } else if (FORMAT_JSON.equals(format)) {
+      response.setContentType("application/json; charset=utf-8");
+    }
+
+    Writer out = response.getWriter();
+    try {
+      writeResponse(getConfFromContext(), out, format);
+    } catch (BadFormatException bfe) {
+      response.sendError(HttpServletResponse.SC_BAD_REQUEST, bfe.getMessage());
+    }
+    out.close();
+  }
+
+  /**
+   * Guts of the servlet - extracted for easy testing.
+   */
+  static void writeResponse(Configuration conf, Writer out, String format)
+    throws IOException, BadFormatException {
+    if (FORMAT_JSON.equals(format)) {
+      Configuration.dumpConfiguration(conf, out);
+    } else if (FORMAT_XML.equals(format)) {
+      conf.writeXml(out);
+    } else {
+      throw new BadFormatException("Bad format: " + format);
+    }
+  }
+
+  public static class BadFormatException extends Exception {
+    private static final long serialVersionUID = 1L;
+
+    public BadFormatException(String msg) {
+      super(msg);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/common/src/java/org/apache/hive/http/HttpServer.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/HttpServer.java b/common/src/java/org/apache/hive/http/HttpServer.java
new file mode 100644
index 0000000..1ff8d7c
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/HttpServer.java
@@ -0,0 +1,316 @@
+/**
+ * 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.http;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URL;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.Shell;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Logger;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.FileManager;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.handler.ContextHandler.Context;
+import org.eclipse.jetty.server.handler.ContextHandlerCollection;
+import org.eclipse.jetty.server.nio.SelectChannelConnector;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
+import org.eclipse.jetty.webapp.WebAppContext;
+
+/**
+ * A simple embedded Jetty server to serve as HS2/HMS web UI.
+ */
+public class HttpServer {
+  public static final String CONF_CONTEXT_ATTRIBUTE = "hive.conf";
+  public static final String ADMINS_ACL = "admins.acl";
+
+  private final AccessControlList adminsAcl;
+  private final String appDir;
+  private final String name;
+  private final String host;
+  private final int port;
+  private final int maxThreads;
+  private final Configuration conf;
+  private final WebAppContext webAppContext;
+  private final Server webServer;
+
+  /**
+   * Create a status server on the given port.
+   */
+  public HttpServer(String name, String host, int port, int maxThreads,
+      Configuration conf, AccessControlList adminsAcl) throws IOException {
+    this.name = name;
+    this.host = host;
+    this.port = port;
+    this.maxThreads = maxThreads;
+    this.conf = conf;
+    this.adminsAcl = adminsAcl;
+
+    webServer = new Server();
+    appDir = getWebAppsPath(name);
+    webAppContext = createWebAppContext();
+    initializeWebServer();
+  }
+
+  public void start() throws Exception {
+    webServer.start();
+  }
+
+  public void stop() throws Exception {
+    webServer.stop();
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  /**
+   * Set servlet context attribute that can be used in jsp.
+   */
+  public void setContextAttribute(String name, Object value) {
+    webAppContext.getServletContext().setAttribute(name, value);
+  }
+
+  /**
+   * Checks the user has privileges to access to instrumentation servlets.
+   * <p/>
+   * If <code>hadoop.security.instrumentation.requires.admin</code> is set to FALSE
+   * (default value) it always returns TRUE.
+   * <p/>
+   * If <code>hadoop.security.instrumentation.requires.admin</code> is set to TRUE
+   * it will check if the current user is in the admin ACLS. If the user is
+   * in the admin ACLs it returns TRUE, otherwise it returns FALSE.
+   *
+   * @param servletContext the servlet context.
+   * @param request the servlet request.
+   * @param response the servlet response.
+   * @return TRUE/FALSE based on the logic described above.
+   */
+  static boolean isInstrumentationAccessAllowed(
+    ServletContext servletContext, HttpServletRequest request,
+    HttpServletResponse response) throws IOException {
+    Configuration conf =
+      (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE);
+
+    boolean access = true;
+    boolean adminAccess = conf.getBoolean(
+      CommonConfigurationKeys.HADOOP_SECURITY_INSTRUMENTATION_REQUIRES_ADMIN,
+      false);
+    if (adminAccess) {
+      access = hasAdministratorAccess(servletContext, request, response);
+    }
+    return access;
+  }
+
+  /**
+   * Does the user sending the HttpServletRequest have the administrator ACLs? If
+   * it isn't the case, response will be modified to send an error to the user.
+   *
+   * @param servletContext
+   * @param request
+   * @param response used to send the error response if user does not have admin access.
+   * @return true if admin-authorized, false otherwise
+   * @throws IOException
+   */
+  static boolean hasAdministratorAccess(
+      ServletContext servletContext, HttpServletRequest request,
+      HttpServletResponse response) throws IOException {
+    Configuration conf =
+        (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE);
+    // If there is no authorization, anybody has administrator access.
+    if (!conf.getBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
+      return true;
+    }
+
+    String remoteUser = request.getRemoteUser();
+    if (remoteUser == null) {
+      response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
+                         "Unauthenticated users are not " +
+                         "authorized to access this page.");
+      return false;
+    }
+
+    if (servletContext.getAttribute(ADMINS_ACL) != null &&
+        !userHasAdministratorAccess(servletContext, remoteUser)) {
+      response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User "
+          + remoteUser + " is unauthorized to access this page.");
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Get the admin ACLs from the given ServletContext and check if the given
+   * user is in the ACL.
+   *
+   * @param servletContext the context containing the admin ACL.
+   * @param remoteUser the remote user to check for.
+   * @return true if the user is present in the ACL, false if no ACL is set or
+   *         the user is not present
+   */
+  static boolean userHasAdministratorAccess(ServletContext servletContext,
+      String remoteUser) {
+    AccessControlList adminsAcl = (AccessControlList) servletContext
+        .getAttribute(ADMINS_ACL);
+    UserGroupInformation remoteUserUGI =
+        UserGroupInformation.createRemoteUser(remoteUser);
+    return adminsAcl != null && adminsAcl.isUserAllowed(remoteUserUGI);
+  }
+
+  /**
+   * Create the web context for the application of specified name
+   */
+  WebAppContext createWebAppContext() {
+    WebAppContext ctx = new WebAppContext();
+    setContextAttributes(ctx.getServletContext());
+    ctx.setDisplayName(name);
+    ctx.setContextPath("/");
+    ctx.setWar(appDir + "/" + name);
+    return ctx;
+  }
+
+  /**
+   * Create a default regular channel connector for "http" requests
+   */
+  Connector createDefaultChannelConnector() {
+    SelectChannelConnector connector = new SelectChannelConnector();
+    connector.setLowResourcesMaxIdleTime(10000);
+    connector.setAcceptQueueSize(maxThreads);
+    connector.setResolveNames(false);
+    connector.setUseDirectBuffers(false);
+    connector.setReuseAddress(!Shell.WINDOWS);
+    return connector;
+  }
+
+  void setContextAttributes(Context ctx) {
+    ctx.setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
+    ctx.setAttribute(ADMINS_ACL, adminsAcl);
+  }
+
+  void initializeWebServer() {
+    // Create the thread pool for the web server to handle HTTP requests
+    QueuedThreadPool threadPool = maxThreads <= 0 ? new QueuedThreadPool()
+      : new QueuedThreadPool(maxThreads);
+    threadPool.setDaemon(true);
+    threadPool.setName(name + "-web");
+    webServer.setThreadPool(threadPool);
+
+    // Create the channel connector for the web server
+    Connector connector = createDefaultChannelConnector();
+    connector.setHost(host);
+    connector.setPort(port);
+    webServer.addConnector(connector);
+
+    // Configure web application contexts for the web server
+    ContextHandlerCollection contexts = new ContextHandlerCollection();
+    contexts.addHandler(webAppContext);
+    webServer.setHandler(contexts);
+
+    addServlet("jmx", "/jmx", JMXJsonServlet.class);
+    addServlet("conf", "/conf", ConfServlet.class);
+
+    ServletContextHandler staticCtx =
+      new ServletContextHandler(contexts, "/static");
+    staticCtx.setResourceBase(appDir + "/static");
+    staticCtx.addServlet(DefaultServlet.class, "/*");
+    staticCtx.setDisplayName("static");
+
+    String logDir = getLogDir();
+    if (logDir != null) {
+      ServletContextHandler logCtx =
+        new ServletContextHandler(contexts, "/logs");
+      setContextAttributes(logCtx.getServletContext());
+      logCtx.addServlet(AdminAuthorizedServlet.class, "/*");
+      logCtx.setResourceBase(logDir);
+      logCtx.setDisplayName("logs");
+    }
+  }
+
+  String getLogDir() {
+    String logDir = conf.get("hive.log.dir");
+    if (logDir == null) {
+      logDir = System.getProperty("hive.log.dir");
+    }
+    if (logDir != null) {
+      return logDir;
+    }
+
+    LoggerContext context = (LoggerContext)LogManager.getContext(false);
+    for (Logger logger: context.getLoggers()) {
+      for (Appender appender: logger.getAppenders().values()) {
+        if (appender instanceof AbstractOutputStreamAppender) {
+          OutputStreamManager manager =
+            ((AbstractOutputStreamAppender<?>)appender).getManager();
+          if (manager instanceof FileManager) {
+            String fileName = ((FileManager)manager).getFileName();
+            if (fileName != null) {
+              return fileName.substring(0, fileName.lastIndexOf('/'));
+            }
+          }
+        }
+      }
+    }
+    return null;
+  }
+
+  String getWebAppsPath(String appName) throws FileNotFoundException {
+    String relativePath = "hive-webapps/" + appName;
+    URL url = getClass().getClassLoader().getResource(relativePath);
+    if (url == null) {
+      throw new FileNotFoundException(relativePath
+          + " not found in CLASSPATH");
+    }
+    String urlString = url.toString();
+    return urlString.substring(0, urlString.lastIndexOf('/'));
+  }
+
+  /**
+   * Add a servlet in the server.
+   * @param name The name of the servlet (can be passed as null)
+   * @param pathSpec The path spec for the servlet
+   * @param clazz The servlet class
+   */
+  void addServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz) {
+    ServletHolder holder = new ServletHolder(clazz);
+    if (name != null) {
+      holder.setName(name);
+    }
+    webAppContext.addServlet(holder, pathSpec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/common/src/java/org/apache/hive/http/JMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/JMXJsonServlet.java b/common/src/java/org/apache/hive/http/JMXJsonServlet.java
new file mode 100644
index 0000000..7535b26
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/JMXJsonServlet.java
@@ -0,0 +1,412 @@
+/**
+ * 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.http;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Array;
+import java.util.Iterator;
+import java.util.Set;
+
+import javax.management.AttributeNotFoundException;
+import javax.management.InstanceNotFoundException;
+import javax.management.IntrospectionException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import javax.management.RuntimeErrorException;
+import javax.management.RuntimeMBeanException;
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.CompositeType;
+import javax.management.openmbean.TabularData;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonGenerator;
+
+/*
+ * This servlet is based off of the JMXProxyServlet from Tomcat 7.0.14. It has
+ * been rewritten to be read only and to output in a JSON format so it is not
+ * really that close to the original.
+ */
+/**
+ * Provides Read only web access to JMX.
+ * <p>
+ * This servlet generally will be placed under the /jmx URL for each
+ * HttpServer.  It provides read only
+ * access to JMX metrics.  The optional <code>qry</code> parameter
+ * may be used to query only a subset of the JMX Beans.  This query
+ * functionality is provided through the
+ * {@link MBeanServer#queryNames(ObjectName, javax.management.QueryExp)}
+ * method.
+ * <p>
+ * For example <code>http://.../jmx?qry=Hadoop:*</code> will return
+ * all Hadoop metrics exposed through JMX.
+ * <p>
+ * The optional <code>get</code> parameter is used to query a specific
+ * attribute of a JMX bean.  The format of the URL is
+ * <code>http://.../jmx?get=MXBeanName::AttributeName<code>
+ * <p>
+ * For example 
+ * <code>
+ * http://../jmx?get=Hadoop:service=NameNode,name=NameNodeInfo::ClusterId
+ * </code> will return the cluster id of the namenode mxbean.
+ * <p>
+ * If the <code>qry</code> or the <code>get</code> parameter is not formatted
+ * correctly then a 400 BAD REQUEST http response code will be returned.
+ * <p>
+ * If a resource such as an mbean or attribute can not be found,
+ * a 404 SC_NOT_FOUND http response code will be returned.
+ * <p>
+ * The return format is JSON and in the form
+ * <p>
+ *  <code><pre>
+ *  {
+ *    "beans" : [
+ *      {
+ *        "name":"bean-name"
+ *        ...
+ *      }
+ *    ]
+ *  }
+ *  </pre></code>
+ *  <p>
+ *  The servlet attempts to convert the JMXBeans into JSON. Each
+ *  bean's attributes will be converted to a JSON object member.
+ *  
+ *  If the attribute is a boolean, a number, a string, or an array
+ *  it will be converted to the JSON equivalent. 
+ *  
+ *  If the value is a {@link CompositeData} then it will be converted
+ *  to a JSON object with the keys as the name of the JSON member and
+ *  the value is converted following these same rules.
+ *  
+ *  If the value is a {@link TabularData} then it will be converted
+ *  to an array of the {@link CompositeData} elements that it contains.
+ *  
+ *  All other objects will be converted to a string and output as such.
+ *  
+ *  The bean's name and modelerType will be returned for all beans.
+ *
+ *  Optional paramater "callback" should be used to deliver JSONP response.
+ *  
+ */
+public class JMXJsonServlet extends HttpServlet {
+  private static final Log LOG = LogFactory.getLog(JMXJsonServlet.class);
+  static final String ACCESS_CONTROL_ALLOW_METHODS =
+      "Access-Control-Allow-Methods";
+  static final String ACCESS_CONTROL_ALLOW_ORIGIN =
+      "Access-Control-Allow-Origin";
+
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * MBean server.
+   */
+  protected transient MBeanServer mBeanServer;
+
+  protected transient JsonFactory jsonFactory;
+
+  /**
+   * Initialize this servlet.
+   */
+  @Override
+  public void init() throws ServletException {
+    // Retrieve the MBean server
+    mBeanServer = ManagementFactory.getPlatformMBeanServer();
+    jsonFactory = new JsonFactory();
+  }
+
+  /**
+   * Process a GET request for the specified resource.
+   * 
+   * @param request
+   *          The servlet request we are processing
+   * @param response
+   *          The servlet response we are creating
+   */
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response) {
+    try {
+      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) {
+        return;
+      }
+      JsonGenerator jg = null;
+      PrintWriter writer = null;
+      try {
+        writer = response.getWriter();
+
+        response.setContentType("application/json; charset=utf8");
+        response.setHeader(ACCESS_CONTROL_ALLOW_METHODS, "GET");
+        response.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, "*");
+
+        jg = jsonFactory.createJsonGenerator(writer);
+        jg.disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET);
+        jg.useDefaultPrettyPrinter();
+        jg.writeStartObject();
+
+        // query per mbean attribute
+        String getmethod = request.getParameter("get");
+        if (getmethod != null) {
+          String[] splitStrings = getmethod.split("\\:\\:");
+          if (splitStrings.length != 2) {
+            jg.writeStringField("result", "ERROR");
+            jg.writeStringField("message", "query format is not as expected.");
+            jg.flush();
+            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+            return;
+          }
+          listBeans(jg, new ObjectName(splitStrings[0]), splitStrings[1],
+              response);
+          return;
+        }
+
+        // query per mbean
+        String qry = request.getParameter("qry");
+        if (qry == null) {
+          qry = "*:*";
+        }
+        listBeans(jg, new ObjectName(qry), null, response);
+      } finally {
+        if (jg != null) {
+          jg.close();
+        }
+        if (writer != null) {
+          writer.close();
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Caught an exception while processing JMX request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } catch (MalformedObjectNameException e) {
+      LOG.error("Caught an exception while processing JMX request", e);
+      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+    }
+  }
+
+  // --------------------------------------------------------- Private Methods
+  private void listBeans(JsonGenerator jg, ObjectName qry, String attribute, 
+      HttpServletResponse response) 
+  throws IOException {
+    LOG.debug("Listing beans for "+qry);
+    Set<ObjectName> names = null;
+    names = mBeanServer.queryNames(qry, null);
+
+    jg.writeArrayFieldStart("beans");
+    Iterator<ObjectName> it = names.iterator();
+    while (it.hasNext()) {
+      ObjectName oname = it.next();
+      MBeanInfo minfo;
+      String code = "";
+      Object attributeinfo = null;
+      try {
+        minfo = mBeanServer.getMBeanInfo(oname);
+        code = minfo.getClassName();
+        String prs = "";
+        try {
+          if ("org.apache.commons.modeler.BaseModelMBean".equals(code)) {
+            prs = "modelerType";
+            code = (String) mBeanServer.getAttribute(oname, prs);
+          }
+          if (attribute!=null) {
+            prs = attribute;
+            attributeinfo = mBeanServer.getAttribute(oname, prs);
+          }
+        } catch (AttributeNotFoundException e) {
+          // If the modelerType attribute was not found, the class name is used
+          // instead.
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch (MBeanException e) {
+          // The code inside the attribute getter threw an exception so log it,
+          // and fall back on the class name
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch (RuntimeException e) {
+          // For some reason even with an MBeanException available to them
+          // Runtime exceptions can still find their way through, so treat them
+          // the same as MBeanException
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch ( ReflectionException e ) {
+          // This happens when the code inside the JMX bean (setter?? from the
+          // java docs) threw an exception, so log it and fall back on the 
+          // class name
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        }
+      } catch (InstanceNotFoundException e) {
+        //Ignored for some reason the bean was not found so don't output it
+        continue;
+      } catch ( IntrospectionException e ) {
+        // This is an internal error, something odd happened with reflection so
+        // log it and don't output the bean.
+        LOG.error("Problem while trying to process JMX query: " + qry
+            + " with MBean " + oname, e);
+        continue;
+      } catch ( ReflectionException e ) {
+        // This happens when the code inside the JMX bean threw an exception, so
+        // log it and don't output the bean.
+        LOG.error("Problem while trying to process JMX query: " + qry
+            + " with MBean " + oname, e);
+        continue;
+      }
+
+      jg.writeStartObject();
+      jg.writeStringField("name", oname.toString());
+
+      jg.writeStringField("modelerType", code);
+      if ((attribute != null) && (attributeinfo == null)) {
+        jg.writeStringField("result", "ERROR");
+        jg.writeStringField("message", "No attribute with name " + attribute
+            + " was found.");
+        jg.writeEndObject();
+        jg.writeEndArray();
+        jg.close();
+        response.setStatus(HttpServletResponse.SC_NOT_FOUND);
+        return;
+      }
+
+      if (attribute != null) {
+        writeAttribute(jg, attribute, attributeinfo);
+      } else {
+        MBeanAttributeInfo attrs[] = minfo.getAttributes();
+        for (int i = 0; i < attrs.length; i++) {
+          writeAttribute(jg, oname, attrs[i]);
+        }
+      }
+      jg.writeEndObject();
+    }
+    jg.writeEndArray();
+  }
+
+  private void writeAttribute(JsonGenerator jg, ObjectName oname, MBeanAttributeInfo attr) throws IOException {
+    if (!attr.isReadable()) {
+      return;
+    }
+    String attName = attr.getName();
+    if ("modelerType".equals(attName)) {
+      return;
+    }
+    if (attName.indexOf("=") >= 0 || attName.indexOf(":") >= 0
+        || attName.indexOf(" ") >= 0) {
+      return;
+    }
+    Object value = null;
+    try {
+      value = mBeanServer.getAttribute(oname, attName);
+    } catch (RuntimeMBeanException e) {
+      // UnsupportedOperationExceptions happen in the normal course of business,
+      // so no need to log them as errors all the time.
+      if (e.getCause() instanceof UnsupportedOperationException) {
+        LOG.debug("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      } else {
+        LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      }
+      return;
+    } catch (RuntimeErrorException e) {
+      // RuntimeErrorException happens when an unexpected failure occurs in getAttribute
+      // for example https://issues.apache.org/jira/browse/DAEMON-120
+      LOG.debug("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (AttributeNotFoundException e) {
+      //Ignored the attribute was not found, which should never happen because the bean
+      //just told us that it has this attribute, but if this happens just don't output
+      //the attribute.
+      return;
+    } catch (MBeanException e) {
+      //The code inside the attribute getter threw an exception so log it, and
+      // skip outputting the attribute
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (RuntimeException e) {
+      //For some reason even with an MBeanException available to them Runtime exceptions
+      //can still find their way through, so treat them the same as MBeanException
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (ReflectionException e) {
+      //This happens when the code inside the JMX bean (setter?? from the java docs)
+      //threw an exception, so log it and skip outputting the attribute
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (InstanceNotFoundException e) {
+      //Ignored the mbean itself was not found, which should never happen because we
+      //just accessed it (perhaps something unregistered in-between) but if this
+      //happens just don't output the attribute.
+      return;
+    }
+
+    writeAttribute(jg, attName, value);
+  }
+
+  private void writeAttribute(JsonGenerator jg, String attName, Object value) throws IOException {
+    jg.writeFieldName(attName);
+    writeObject(jg, value);
+  }
+
+  private void writeObject(JsonGenerator jg, Object value) throws IOException {
+    if(value == null) {
+      jg.writeNull();
+    } else {
+      Class<?> c = value.getClass();
+      if (c.isArray()) {
+        jg.writeStartArray();
+        int len = Array.getLength(value);
+        for (int j = 0; j < len; j++) {
+          Object item = Array.get(value, j);
+          writeObject(jg, item);
+        }
+        jg.writeEndArray();
+      } else if(value instanceof Number) {
+        Number n = (Number)value;
+        jg.writeNumber(n.toString());
+      } else if(value instanceof Boolean) {
+        Boolean b = (Boolean)value;
+        jg.writeBoolean(b);
+      } else if(value instanceof CompositeData) {
+        CompositeData cds = (CompositeData)value;
+        CompositeType comp = cds.getCompositeType();
+        Set<String> keys = comp.keySet();
+        jg.writeStartObject();
+        for(String key: keys) {
+          writeAttribute(jg, key, cds.get(key));
+        }
+        jg.writeEndObject();
+      } else if(value instanceof TabularData) {
+        TabularData tds = (TabularData)value;
+        jg.writeStartArray();
+        for(Object entry : tds.values()) {
+          writeObject(jg, entry);
+        }
+        jg.writeEndArray();
+      } else {
+        jg.writeString(value.toString());
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 34bdbf6..22b4ca8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -134,6 +134,7 @@
     <jackson.version>1.9.2</jackson.version>
     <!-- jackson 1 and 2 lines can coexist without issue, as they have different artifactIds -->
     <jackson.new.version>2.4.2</jackson.new.version>
+    <jasper.version>5.5.23</jasper.version>
     <javaewah.version>0.3.2</javaewah.version>
     <javolution.version>5.5.1</javolution.version>
     <jdo-api.version>3.0.1</jdo-api.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index d893099..145ba9b 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -44,6 +44,12 @@
       <groupId>org.apache.hive</groupId>
       <artifactId>hive-common</artifactId>
       <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.eclipse.jetty.aggregate</groupId>
+          <artifactId>jetty-all</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/pom.xml
----------------------------------------------------------------------
diff --git a/service/pom.xml b/service/pom.xml
index afa52cf..ebc5966 100644
--- a/service/pom.xml
+++ b/service/pom.xml
@@ -82,6 +82,16 @@
         <version>${jetty.version}</version>
     </dependency>
     <dependency>
+      <groupId>tomcat</groupId>
+      <artifactId>jasper-compiler</artifactId>
+      <version>${jasper.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>tomcat</groupId>
+      <artifactId>jasper-runtime</artifactId>
+      <version>${jasper.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.thrift</groupId>
       <artifactId>libfb303</artifactId>
       <version>${libfb303.version}</version>
@@ -140,6 +150,14 @@
   <build>
     <sourceDirectory>${basedir}/src/java</sourceDirectory>
     <testSourceDirectory>${basedir}/src/test</testSourceDirectory>
+    <resources>
+      <resource>
+        <directory>${project.build.directory}</directory>
+        <includes>
+          <include>hive-webapps/**</include>
+        </includes>
+      </resource>
+    </resources>
     <plugins>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
@@ -155,12 +173,50 @@
               <sources>
                 <source>src/model</source>
                 <source>src/gen/thrift/gen-javabean</source>
+                <source>${project.build.directory}/generated-sources/java</source>
               </sources>
             </configuration>
           </execution>
         </executions>
       </plugin>
       <plugin>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <!-- Generate web app sources -->
+          <execution>
+            <id>generate</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <target>
+                <property name="build.webapps"
+                  location="${project.build.directory}/hive-webapps"/>
+                <property name="src.webapps"
+                  location="${basedir}/src/resources/hive-webapps"/>
+                <property name="generated.sources"
+                  location="${project.build.directory}/generated-sources"/>
+                <mkdir dir="${build.webapps}"/>
+                <copy todir="${build.webapps}">
+                  <fileset dir="${src.webapps}">
+                    <exclude name="**/*.jsp"/>
+                    <exclude name="**/.*"/>
+                  </fileset>
+                </copy>
+                <taskdef classname="org.apache.jasper.JspC"
+                  name="jspcompiler" classpathref="maven.compile.classpath"/>
+                <mkdir dir="${build.webapps}/hiveserver2/WEB-INF"/>
+                <jspcompiler uriroot="${src.webapps}/hiveserver2"
+                  outputdir="${generated.sources}/java"
+                  package="org.apache.hive.generated.hiveserver2"
+                  webxml="${build.webapps}/hiveserver2/WEB-INF/web.xml"/>
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <executions>

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/java/org/apache/hive/service/cli/operation/Operation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
index 25cefc2..d2b3f9c 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
@@ -73,7 +73,7 @@ public abstract class Operation {
   protected Map<String, String> confOverlay = new HashMap<String, String>();
 
   private long operationTimeout;
-  private long lastAccessTime;
+  private volatile long lastAccessTime;
 
   protected static final EnumSet<FetchOrientation> DEFAULT_FETCH_ORIENTATION_SET =
       EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST);

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java b/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
index b0bd351..92135cd 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
@@ -20,12 +20,12 @@ package org.apache.hive.service.cli.operation;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.metrics.common.Metrics;
 import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
@@ -49,6 +49,7 @@ import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.config.Configuration;
 import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
@@ -57,8 +58,8 @@ import org.slf4j.LoggerFactory;
  */
 public class OperationManager extends AbstractService {
   private final Logger LOG = LoggerFactory.getLogger(OperationManager.class.getName());
-  private final Map<OperationHandle, Operation> handleToOperation =
-      new HashMap<OperationHandle, Operation>();
+  private final ConcurrentHashMap<OperationHandle, Operation> handleToOperation =
+      new ConcurrentHashMap<OperationHandle, Operation>();
 
   public OperationManager() {
     super(OperationManager.class.getSimpleName());
@@ -165,24 +166,24 @@ public class OperationManager extends AbstractService {
     return operation;
   }
 
-  private synchronized Operation getOperationInternal(OperationHandle operationHandle) {
+  private Operation getOperationInternal(OperationHandle operationHandle) {
     return handleToOperation.get(operationHandle);
   }
 
-  private synchronized Operation removeTimedOutOperation(OperationHandle operationHandle) {
+  private Operation removeTimedOutOperation(OperationHandle operationHandle) {
     Operation operation = handleToOperation.get(operationHandle);
     if (operation != null && operation.isTimedOut(System.currentTimeMillis())) {
-      handleToOperation.remove(operationHandle);
+      handleToOperation.remove(operationHandle, operation);
       return operation;
     }
     return null;
   }
 
-  private synchronized void addOperation(Operation operation) {
+  private void addOperation(Operation operation) {
     handleToOperation.put(operation.getHandle(), operation);
   }
 
-  private synchronized Operation removeOperation(OperationHandle opHandle) {
+  private Operation removeOperation(OperationHandle opHandle) {
     return handleToOperation.remove(opHandle);
   }
 
@@ -292,6 +293,11 @@ public class OperationManager extends AbstractService {
     return schema;
   }
 
+  public Collection<Operation> getOperations() {
+    return Collections.unmodifiableCollection(handleToOperation.values());
+  }
+
+
   public OperationLog getOperationLogByThread() {
     return OperationLog.getCurrentOperationLog();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 1331a99..d90dd0d 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -162,12 +162,16 @@ public class SQLOperation extends ExecuteStatementOperation {
     } catch (HiveSQLException e) {
       setState(OperationState.ERROR);
       throw e;
-    } catch (Exception e) {
+    } catch (Throwable e) {
       setState(OperationState.ERROR);
       throw new HiveSQLException("Error running query: " + e.toString(), e);
     }
   }
 
+  public String getQueryStr() {
+    return driver == null || driver.getPlan() == null ? "Unknown" : driver.getPlan().getQueryStr();
+  }
+
   private void runQuery(HiveConf sqlOperationConf) throws HiveSQLException {
     try {
       // In Hive server mode, we are not able to retry in the FetchTask
@@ -190,7 +194,7 @@ public class SQLOperation extends ExecuteStatementOperation {
         setState(OperationState.ERROR);
         throw e;
       }
-    } catch (Exception e) {
+    } catch (Throwable e) {
       setState(OperationState.ERROR);
       throw new HiveSQLException("Error running query: " + e.toString(), e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
index 1ab5652..5d12a85 100644
--- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
+++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
@@ -85,4 +85,8 @@ public interface HiveSessionBase {
   void setIpAddress(String ipAddress);
 
   long getLastAccessTime();
+
+  long getCreationTime();
+
+  int getOpenOperationCount();
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
index a14908b..50c912e 100644
--- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
+++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
@@ -85,6 +85,7 @@ public class HiveSessionImpl implements HiveSession {
   private String username;
   private final String password;
   private final HiveConf hiveConf;
+  private final long creationTime;
   // TODO: some SessionState internals are not thread safe. The compile-time internals are synced
   //       via session-scope or global compile lock. The run-time internals work by magic!
   //       They probably work because races are relatively unlikely and few tools run parallel
@@ -114,6 +115,7 @@ public class HiveSessionImpl implements HiveSession {
       HiveConf serverhiveConf, String ipAddress) {
     this.username = username;
     this.password = password;
+    creationTime = System.currentTimeMillis();
     this.sessionHandle = new SessionHandle(protocol);
     this.hiveConf = new HiveConf(serverhiveConf);
     this.ipAddress = ipAddress;
@@ -697,6 +699,11 @@ public class HiveSessionImpl implements HiveSession {
   }
 
   @Override
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  @Override
   public void closeExpiredOperations() {
     OperationHandle[] handles;
     synchronized (opHandleSet) {
@@ -785,6 +792,11 @@ public class HiveSessionImpl implements HiveSession {
   }
 
   @Override
+  public int getOpenOperationCount() {
+    return opHandleSet.size();
+  }
+
+  @Override
   public String getIpAddress() {
     return ipAddress;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
index d11cf3d..637cdca 100644
--- a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -22,6 +22,8 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
@@ -29,7 +31,6 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
@@ -38,18 +39,19 @@ import org.apache.hadoop.hive.common.metrics.common.Metrics;
 import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.common.metrics.common.MetricsVariable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.hooks.HookUtils;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
+import org.apache.hive.service.cli.operation.Operation;
 import org.apache.hive.service.cli.operation.OperationManager;
 import org.apache.hive.service.cli.thrift.TProtocolVersion;
 import org.apache.hive.service.server.HiveServer2;
 import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * SessionManager.
@@ -452,6 +454,14 @@ public class SessionManager extends CompositeService {
     return backgroundOperationPool.submit(r);
   }
 
+  public Collection<Operation> getOperations() {
+    return operationManager.getOperations();
+  }
+
+  public Collection<HiveSession> getSessions() {
+    return Collections.unmodifiableCollection(handleToSession.values());
+  }
+
   public int getOpenSessionCount() {
     return handleToSession.size();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index b30b6a2..204eb5a 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -56,9 +56,12 @@ import org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager;
 import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hive.common.util.HiveStringUtils;
 import org.apache.hive.common.util.HiveVersionInfo;
+import org.apache.hive.http.HttpServer;
 import org.apache.hive.service.CompositeService;
+import org.apache.hive.service.ServiceException;
 import org.apache.hive.service.cli.CLIService;
 import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
@@ -86,6 +89,7 @@ public class HiveServer2 extends CompositeService {
   private String znodePath;
   private CuratorFramework zooKeeperClient;
   private boolean registeredWithZooKeeper = false;
+  private HttpServer webServer; // Web UI
 
   public HiveServer2() {
     super(HiveServer2.class.getSimpleName());
@@ -116,6 +120,32 @@ public class HiveServer2 extends CompositeService {
     } catch (Throwable t) {
       throw new Error("Unable to intitialize HiveServer2", t);
     }
+    // Setup web UI
+    try {
+      if (hiveConf.getBoolVar(ConfVars.HIVE_IN_TEST)) {
+        LOG.info("Web UI is disabled since in test mode");
+      } else {
+        int webUIPort =
+          hiveConf.getIntVar(ConfVars.HIVE_SERVER2_WEBUI_PORT);
+        if (webUIPort <= 0) {
+          LOG.info("Web UI is disabled since port is set to " + webUIPort);
+        } else {
+          AccessControlList adminsAcl =
+            new AccessControlList(hiveConf.getVar(ConfVars.USERS_IN_ADMIN_ROLE));
+          hiveConf.set("startcode", String.valueOf(System.currentTimeMillis()));
+          webServer = new HttpServer("hiveserver2",
+            hiveConf.getVar(ConfVars.HIVE_SERVER2_WEBUI_BIND_HOST),
+            webUIPort,
+            hiveConf.getIntVar(ConfVars.HIVE_SERVER2_WEBUI_MAX_THREADS),
+            hiveConf, adminsAcl);
+          // SessionManager is initialized
+          webServer.setContextAttribute("hive.sm",
+            cliService.getSessionManager());
+        }
+      }
+    } catch (IOException ie) {
+      throw new ServiceException(ie);
+    }
     // Add a shutdown hook for catching SIGTERM & SIGINT
     Runtime.getRuntime().addShutdownHook(new Thread() {
       @Override
@@ -371,6 +401,15 @@ public class HiveServer2 extends CompositeService {
   @Override
   public synchronized void start() {
     super.start();
+    if (webServer != null) {
+      try {
+        webServer.start();
+        LOG.info("Web UI has started on port " + webServer.getPort());
+      } catch (Exception e) {
+        LOG.error("Error starting Web UI: ", e);
+        throw new ServiceException(e);
+      }
+    }
   }
 
   @Override
@@ -378,6 +417,14 @@ public class HiveServer2 extends CompositeService {
     LOG.info("Shutting down HiveServer2");
     HiveConf hiveConf = this.getHiveConf();
     super.stop();
+    if (webServer != null) {
+      try {
+        webServer.stop();
+        LOG.info("Web UI has stopped");
+      } catch (Exception e) {
+        LOG.error("Error stopping Web UI: ", e);
+      }
+    }
     // Shutdown Metrics
     if (MetricsFactory.getInstance() != null) {
       try {

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
----------------------------------------------------------------------
diff --git a/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp b/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
new file mode 100644
index 0000000..4fad63c
--- /dev/null
+++ b/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
@@ -0,0 +1,186 @@
+<%--
+/**
+ * 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.
+ */
+--%>
+<%@ page contentType="text/html;charset=UTF-8"
+  import="org.apache.hadoop.conf.Configuration"
+  import="org.apache.hive.common.util.HiveVersionInfo"
+  import="org.apache.hive.service.cli.operation.Operation"
+  import="org.apache.hive.service.cli.operation.SQLOperation"
+  import="org.apache.hive.service.cli.session.SessionManager"
+  import="org.apache.hive.service.cli.session.HiveSession"
+  import="javax.servlet.ServletContext"
+  import="java.util.Collection"
+  import="java.util.Date"
+%>
+
+<%
+ServletContext ctx = getServletContext();
+Configuration conf = (Configuration)ctx.getAttribute("hive.conf");
+long startcode = conf.getLong("startcode", System.currentTimeMillis());
+SessionManager sessionManager =
+  (SessionManager)ctx.getAttribute("hive.sm");
+%>
+
+<!--[if IE]>
+<!DOCTYPE html>
+<![endif]-->
+<?xml version="1.0" encoding="UTF-8" ?>
+<html lang="en">
+  <head>
+    <meta charset="utf-8">
+    <title>HiveServer2</title>
+    <meta name="viewport" content="width=device-width, initial-scale=1.0">
+    <meta name="description" content="">
+
+    <link href="/static/css/bootstrap.min.css" rel="stylesheet">
+    <link href="/static/css/bootstrap-theme.min.css" rel="stylesheet">
+    <link href="/static/css/hive.css" rel="stylesheet">
+  </head>
+
+  <body>
+  <div class="navbar  navbar-fixed-top navbar-default">
+      <div class="container">
+          <div class="navbar-header">
+              <button type="button" class="navbar-toggle" data-toggle="collapse" data-target=".navbar-collapse">
+                  <span class="icon-bar"></span>
+                  <span class="icon-bar"></span>
+                  <span class="icon-bar"></span>
+              </button>
+              <a class="navbar-brand" href="/hiveserver2.jsp"><img src="/static/hive_logo.jpeg" alt="Hive Logo"/></a>
+          </div>
+          <div class="collapse navbar-collapse">
+              <ul class="nav navbar-nav">
+                <li class="active"><a href="/">Home</a></li>
+                <li><a href="/logs/">Local logs</a></li>
+                <li><a href="/jmx">Metrics Dump</a></li>
+                <li><a href="/conf">Hive Configuration</a></li>
+            </ul>
+          </div><!--/.nav-collapse -->
+        </div>
+      </div>
+    </div>
+
+<div class="container">
+    <div class="row inner_header">
+        <div class="page-header">
+            <h1>HiveServer2</h1>
+        </div>
+    </div>
+    <div class="row">
+
+<%
+if (sessionManager != null) { 
+  long currentTime = System.currentTimeMillis();
+%> 
+
+<section>
+<h2>Active Sessions</h2>
+<table id="attributes_table" class="table table-striped">
+    <tr>
+        <th>User Name</th>
+        <th>IP Address</th>
+        <th>Operation Count</th>
+        <th>Active Time (s)</th>
+        <th>Idle Time (s)</th>
+    </tr>
+<%
+Collection<HiveSession> hiveSessions = sessionManager.getSessions();
+for (HiveSession hiveSession: hiveSessions) {
+%>
+    <tr>
+        <td><%= hiveSession.getUserName() %></td>
+        <td><%= hiveSession.getIpAddress() %></td>
+        <td><%= hiveSession.getOpenOperationCount() %></td>
+        <td><%= (currentTime - hiveSession.getCreationTime())/1000 %></td>
+        <td><%= (currentTime - hiveSession.getLastAccessTime())/1000 %></td>
+    </tr>
+<%
+}
+%>
+<tr>
+  <td colspan="5">Total number of sessions: <%= hiveSessions.size() %></td>
+</tr>
+</table>
+</section>
+
+<section>
+<h2>Queries</h2>
+<table id="attributes_table" class="table table-striped">
+    <tr>
+        <th>User Name</th>
+        <th>Query</th>
+        <th>State</th>
+        <th>Elapsed Time (s)</th>
+    </tr>
+<%
+int queries = 0;
+Collection<Operation> operations = sessionManager.getOperations();
+for (Operation operation: operations) {
+  if (operation instanceof SQLOperation) {
+    SQLOperation query = (SQLOperation) operation;
+    queries++;
+%>
+    <tr>
+        <td><%= query.getParentSession().getUserName() %></td>
+        <td><%= query.getQueryStr() %></td>
+        <td><%= query.getStatus().getState() %></td>
+        <td><%= (currentTime - query.getLastAccessTime())/1000 %></td>
+    </tr>
+<%
+  }
+}
+%>
+<tr>
+  <td colspan="4">Total number of queries: <%= queries %></td>
+</tr>
+</table>
+</section>
+<% 
+ }
+%>
+
+    <section>
+    <h2>Software Attributes</h2>
+    <table id="attributes_table" class="table table-striped">
+        <tr>
+            <th>Attribute Name</th>
+            <th>Value</th>
+            <th>Description</th>
+        </tr>
+        <tr>
+            <td>Hive Version</td>
+            <td><%= HiveVersionInfo.getVersion() %>, r<%= HiveVersionInfo.getRevision() %></td>
+            <td>Hive version and revision</td>
+        </tr>
+        <tr>
+            <td>Hive Compiled</td>
+            <td><%= HiveVersionInfo.getDate() %>, <%= HiveVersionInfo.getUser() %></td>
+            <td>When Hive was compiled and by whom</td>
+        </tr>
+        <tr>
+            <td>HiveServer2 Start Time</td>
+            <td><%= new Date(startcode) %></td>
+            <td>Date stamp of when this HiveServer2 was started</td>
+        </tr>
+    </table>
+    </section>
+    </div>
+</div>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/resources/hive-webapps/hiveserver2/index.html
----------------------------------------------------------------------
diff --git a/service/src/resources/hive-webapps/hiveserver2/index.html b/service/src/resources/hive-webapps/hiveserver2/index.html
new file mode 100644
index 0000000..f18ba53
--- /dev/null
+++ b/service/src/resources/hive-webapps/hiveserver2/index.html
@@ -0,0 +1,20 @@
+<!--
+/**
+ * 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.
+ */
+-->
+<meta HTTP-EQUIV="REFRESH" content="0;url=/hiveserver2.jsp"/>

http://git-wip-us.apache.org/repos/asf/hive/blob/2c0c191c/service/src/resources/hive-webapps/static/css/bootstrap-theme.min.css
----------------------------------------------------------------------
diff --git a/service/src/resources/hive-webapps/static/css/bootstrap-theme.min.css b/service/src/resources/hive-webapps/static/css/bootstrap-theme.min.css
new file mode 100755
index 0000000..c31428b
--- /dev/null
+++ b/service/src/resources/hive-webapps/static/css/bootstrap-theme.min.css
@@ -0,0 +1,10 @@
+/*!
+ * Bootstrap v3.0.0
+ *
+ * Copyright 2013 Twitter, Inc
+ * Licensed under the Apache License v2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Designed and built with all the love in the world by @mdo and @fat.
+ */
+.btn-default,.btn-primary,.btn-success,.btn-info,.btn-warning,.btn-danger{text-shadow:0 -1px 0 rgba(0,0,0,0.2);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.15),0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.15),0 1px 1px rgba(0,0,0,0.075)}.btn-default:active,.btn-primary:active,.btn-success:active,.btn-info:active,.btn-warning:active,.btn-danger:active,.btn-default.active,.btn-primary.active,.btn-success.active,.btn-info.active,.btn-warning.active,.btn-danger.active{-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,0.125);box-shadow:inset 0 3px 5px rgba(0,0,0,0.125)}.btn:active,.btn.active{background-image:none}.btn-default{text-shadow:0 1px 0 #fff;background-image:-webkit-gradient(linear,left 0,left 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,0%,#e6e6e6,100%);background-image:-moz-linear-gradient(top,#fff 0,#e6e6e6 100%);background-image:linear-gradient(to bottom,#fff 0,#e6e6e6 100%);background-repeat:repeat-x;border-co
 lor:#e0e0e0;border-color:#ccc;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0)}.btn-default:active,.btn-default.active{background-color:#e6e6e6;border-color:#e0e0e0}.btn-primary{background-image:-webkit-gradient(linear,left 0,left 100%,from(#428bca),to(#3071a9));background-image:-webkit-linear-gradient(top,#428bca,0%,#3071a9,100%);background-image:-moz-linear-gradient(top,#428bca 0,#3071a9 100%);background-image:linear-gradient(to bottom,#428bca 0,#3071a9 100%);background-repeat:repeat-x;border-color:#2d6ca2;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca',endColorstr='#ff3071a9',GradientType=0)}.btn-primary:active,.btn-primary.active{background-color:#3071a9;border-color:#2d6ca2}.btn-success{background-image:-webkit-gradient(linear,left 0,left 100%,from(#5cb85c),to(#449d44));background-image:-webkit-linear-gradient(top,#5cb85c,0%,#449d44,100%);background-image:-moz-linear-gradient(top,#5cb
 85c 0,#449d44 100%);background-image:linear-gradient(to bottom,#5cb85c 0,#449d44 100%);background-repeat:repeat-x;border-color:#419641;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c',endColorstr='#ff449d44',GradientType=0)}.btn-success:active,.btn-success.active{background-color:#449d44;border-color:#419641}.btn-warning{background-image:-webkit-gradient(linear,left 0,left 100%,from(#f0ad4e),to(#ec971f));background-image:-webkit-linear-gradient(top,#f0ad4e,0%,#ec971f,100%);background-image:-moz-linear-gradient(top,#f0ad4e 0,#ec971f 100%);background-image:linear-gradient(to bottom,#f0ad4e 0,#ec971f 100%);background-repeat:repeat-x;border-color:#eb9316;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e',endColorstr='#ffec971f',GradientType=0)}.btn-warning:active,.btn-warning.active{background-color:#ec971f;border-color:#eb9316}.btn-danger{background-image:-webkit-gradient(linear,left 0,left 100%,from(#d9534f),to(#c9302c));background-i
 mage:-webkit-linear-gradient(top,#d9534f,0%,#c9302c,100%);background-image:-moz-linear-gradient(top,#d9534f 0,#c9302c 100%);background-image:linear-gradient(to bottom,#d9534f 0,#c9302c 100%);background-repeat:repeat-x;border-color:#c12e2a;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f',endColorstr='#ffc9302c',GradientType=0)}.btn-danger:active,.btn-danger.active{background-color:#c9302c;border-color:#c12e2a}.btn-info{background-image:-webkit-gradient(linear,left 0,left 100%,from(#5bc0de),to(#31b0d5));background-image:-webkit-linear-gradient(top,#5bc0de,0%,#31b0d5,100%);background-image:-moz-linear-gradient(top,#5bc0de 0,#31b0d5 100%);background-image:linear-gradient(to bottom,#5bc0de 0,#31b0d5 100%);background-repeat:repeat-x;border-color:#2aabd2;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff31b0d5',GradientType=0)}.btn-info:active,.btn-info.active{background-color:#31b0d5;border-color:#2aabd2}.thumbnail,.img-
 thumbnail{-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.075);box-shadow:0 1px 2px rgba(0,0,0,0.075)}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus,.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{background-color:#357ebd;background-image:-webkit-gradient(linear,left 0,left 100%,from(#428bca),to(#357ebd));background-image:-webkit-linear-gradient(top,#428bca,0%,#357ebd,100%);background-image:-moz-linear-gradient(top,#428bca 0,#357ebd 100%);background-image:linear-gradient(to bottom,#428bca 0,#357ebd 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca',endColorstr='#ff357ebd',GradientType=0)}.navbar{background-image:-webkit-gradient(linear,left 0,left 100%,from(#fff),to(#f8f8f8));background-image:-webkit-linear-gradient(top,#fff,0%,#f8f8f8,100%);background-image:-moz-linear-gradient(top,#fff 0,#f8f8f8 100%);background-image:linear-gradient(to bottom,#fff 0,#f8f8f8 100%);background-repeat:repe
 at-x;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fff8f8f8',GradientType=0);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.15),0 1px 5px rgba(0,0,0,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.15),0 1px 5px rgba(0,0,0,0.075)}.navbar .navbar-nav>.active>a{background-color:#f8f8f8}.navbar-brand,.navbar-nav>li>a{text-shadow:0 1px 0 rgba(255,255,255,0.25)}.navbar-inverse{background-image:-webkit-gradient(linear,left 0,left 100%,from(#3c3c3c),to(#222));background-image:-webkit-linear-gradient(top,#3c3c3c,0%,#222,100%);background-image:-moz-linear-gradient(top,#3c3c3c 0,#222 100%);background-image:linear-gradient(to bottom,#3c3c3c 0,#222 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff3c3c3c',endColorstr='#ff222222',GradientType=0)}.navbar-inverse .navbar-nav>.active>a{background-color:#222}.navbar-inverse .navbar-brand,.navbar-inverse .navbar-nav>li>a{text-shadow
 :0 -1px 0 rgba(0,0,0,0.25)}.navbar-static-top,.navbar-fixed-top,.navbar-fixed-bottom{border-radius:0}.alert{text-shadow:0 1px 0 rgba(255,255,255,0.2);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.25),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.25),0 1px 2px rgba(0,0,0,0.05)}.alert-success{background-image:-webkit-gradient(linear,left 0,left 100%,from(#dff0d8),to(#c8e5bc));background-image:-webkit-linear-gradient(top,#dff0d8,0%,#c8e5bc,100%);background-image:-moz-linear-gradient(top,#dff0d8 0,#c8e5bc 100%);background-image:linear-gradient(to bottom,#dff0d8 0,#c8e5bc 100%);background-repeat:repeat-x;border-color:#b2dba1;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8',endColorstr='#ffc8e5bc',GradientType=0)}.alert-info{background-image:-webkit-gradient(linear,left 0,left 100%,from(#d9edf7),to(#b9def0));background-image:-webkit-linear-gradient(top,#d9edf7,0%,#b9def0,100%);background-image:-moz-linear-gradient(top,#d9edf7 0,#b9
 def0 100%);background-image:linear-gradient(to bottom,#d9edf7 0,#b9def0 100%);background-repeat:repeat-x;border-color:#9acfea;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7',endColorstr='#ffb9def0',GradientType=0)}.alert-warning{background-image:-webkit-gradient(linear,left 0,left 100%,from(#fcf8e3),to(#f8efc0));background-image:-webkit-linear-gradient(top,#fcf8e3,0%,#f8efc0,100%);background-image:-moz-linear-gradient(top,#fcf8e3 0,#f8efc0 100%);background-image:linear-gradient(to bottom,#fcf8e3 0,#f8efc0 100%);background-repeat:repeat-x;border-color:#f5e79e;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3',endColorstr='#fff8efc0',GradientType=0)}.alert-danger{background-image:-webkit-gradient(linear,left 0,left 100%,from(#f2dede),to(#e7c3c3));background-image:-webkit-linear-gradient(top,#f2dede,0%,#e7c3c3,100%);background-image:-moz-linear-gradient(top,#f2dede 0,#e7c3c3 100%);background-image:linear-gradient(to bottom,#f2dede 0,
 #e7c3c3 100%);background-repeat:repeat-x;border-color:#dca7a7;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede',endColorstr='#ffe7c3c3',GradientType=0)}.progress{background-image:-webkit-gradient(linear,left 0,left 100%,from(#ebebeb),to(#f5f5f5));background-image:-webkit-linear-gradient(top,#ebebeb,0%,#f5f5f5,100%);background-image:-moz-linear-gradient(top,#ebebeb 0,#f5f5f5 100%);background-image:linear-gradient(to bottom,#ebebeb 0,#f5f5f5 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffebebeb',endColorstr='#fff5f5f5',GradientType=0)}.progress-bar{background-image:-webkit-gradient(linear,left 0,left 100%,from(#428bca),to(#3071a9));background-image:-webkit-linear-gradient(top,#428bca,0%,#3071a9,100%);background-image:-moz-linear-gradient(top,#428bca 0,#3071a9 100%);background-image:linear-gradient(to bottom,#428bca 0,#3071a9 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient
 (startColorstr='#ff428bca',endColorstr='#ff3071a9',GradientType=0)}.progress-bar-success{background-image:-webkit-gradient(linear,left 0,left 100%,from(#5cb85c),to(#449d44));background-image:-webkit-linear-gradient(top,#5cb85c,0%,#449d44,100%);background-image:-moz-linear-gradient(top,#5cb85c 0,#449d44 100%);background-image:linear-gradient(to bottom,#5cb85c 0,#449d44 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c',endColorstr='#ff449d44',GradientType=0)}.progress-bar-info{background-image:-webkit-gradient(linear,left 0,left 100%,from(#5bc0de),to(#31b0d5));background-image:-webkit-linear-gradient(top,#5bc0de,0%,#31b0d5,100%);background-image:-moz-linear-gradient(top,#5bc0de 0,#31b0d5 100%);background-image:linear-gradient(to bottom,#5bc0de 0,#31b0d5 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff31b0d5',GradientType=0)}.progress-bar-warning{backg
 round-image:-webkit-gradient(linear,left 0,left 100%,from(#f0ad4e),to(#ec971f));background-image:-webkit-linear-gradient(top,#f0ad4e,0%,#ec971f,100%);background-image:-moz-linear-gradient(top,#f0ad4e 0,#ec971f 100%);background-image:linear-gradient(to bottom,#f0ad4e 0,#ec971f 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e',endColorstr='#ffec971f',GradientType=0)}.progress-bar-danger{background-image:-webkit-gradient(linear,left 0,left 100%,from(#d9534f),to(#c9302c));background-image:-webkit-linear-gradient(top,#d9534f,0%,#c9302c,100%);background-image:-moz-linear-gradient(top,#d9534f 0,#c9302c 100%);background-image:linear-gradient(to bottom,#d9534f 0,#c9302c 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f',endColorstr='#ffc9302c',GradientType=0)}.list-group{border-radius:4px;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.075);box-shadow:0 1px 2px rgba(0,0,0,0.075)}.li
 st-group-item.active,.list-group-item.active:hover,.list-group-item.active:focus{text-shadow:0 -1px 0 #3071a9;background-image:-webkit-gradient(linear,left 0,left 100%,from(#428bca),to(#3278b3));background-image:-webkit-linear-gradient(top,#428bca,0%,#3278b3,100%);background-image:-moz-linear-gradient(top,#428bca 0,#3278b3 100%);background-image:linear-gradient(to bottom,#428bca 0,#3278b3 100%);background-repeat:repeat-x;border-color:#3278b3;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca',endColorstr='#ff3278b3',GradientType=0)}.panel{-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.panel-default>.panel-heading{background-image:-webkit-gradient(linear,left 0,left 100%,from(#f5f5f5),to(#e8e8e8));background-image:-webkit-linear-gradient(top,#f5f5f5,0%,#e8e8e8,100%);background-image:-moz-linear-gradient(top,#f5f5f5 0,#e8e8e8 100%);background-image:linear-gradient(to bottom,#f5f5f5 0,#e8e8e8 100%);background-repeat:repeat-x
 ;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#ffe8e8e8',GradientType=0)}.panel-primary>.panel-heading{background-image:-webkit-gradient(linear,left 0,left 100%,from(#428bca),to(#357ebd));background-image:-webkit-linear-gradient(top,#428bca,0%,#357ebd,100%);background-image:-moz-linear-gradient(top,#428bca 0,#357ebd 100%);background-image:linear-gradient(to bottom,#428bca 0,#357ebd 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca',endColorstr='#ff357ebd',GradientType=0)}.panel-success>.panel-heading{background-image:-webkit-gradient(linear,left 0,left 100%,from(#dff0d8),to(#d0e9c6));background-image:-webkit-linear-gradient(top,#dff0d8,0%,#d0e9c6,100%);background-image:-moz-linear-gradient(top,#dff0d8 0,#d0e9c6 100%);background-image:linear-gradient(to bottom,#dff0d8 0,#d0e9c6 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8
 ',endColorstr='#ffd0e9c6',GradientType=0)}.panel-info>.panel-heading{background-image:-webkit-gradient(linear,left 0,left 100%,from(#d9edf7),to(#c4e3f3));background-image:-webkit-linear-gradient(top,#d9edf7,0%,#c4e3f3,100%);background-image:-moz-linear-gradient(top,#d9edf7 0,#c4e3f3 100%);background-image:linear-gradient(to bottom,#d9edf7 0,#c4e3f3 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7',endColorstr='#ffc4e3f3',GradientType=0)}.panel-warning>.panel-heading{background-image:-webkit-gradient(linear,left 0,left 100%,from(#fcf8e3),to(#faf2cc));background-image:-webkit-linear-gradient(top,#fcf8e3,0%,#faf2cc,100%);background-image:-moz-linear-gradient(top,#fcf8e3 0,#faf2cc 100%);background-image:linear-gradient(to bottom,#fcf8e3 0,#faf2cc 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3',endColorstr='#fffaf2cc',GradientType=0)}.panel-danger>.panel-heading{backgro
 und-image:-webkit-gradient(linear,left 0,left 100%,from(#f2dede),to(#ebcccc));background-image:-webkit-linear-gradient(top,#f2dede,0%,#ebcccc,100%);background-image:-moz-linear-gradient(top,#f2dede 0,#ebcccc 100%);background-image:linear-gradient(to bottom,#f2dede 0,#ebcccc 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede',endColorstr='#ffebcccc',GradientType=0)}.well{background-image:-webkit-gradient(linear,left 0,left 100%,from(#e8e8e8),to(#f5f5f5));background-image:-webkit-linear-gradient(top,#e8e8e8,0%,#f5f5f5,100%);background-image:-moz-linear-gradient(top,#e8e8e8 0,#f5f5f5 100%);background-image:linear-gradient(to bottom,#e8e8e8 0,#f5f5f5 100%);background-repeat:repeat-x;border-color:#dcdcdc;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffe8e8e8',endColorstr='#fff5f5f5',GradientType=0);-webkit-box-shadow:inset 0 1px 3px rgba(0,0,0,0.05),0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 3px rgba(0
 ,0,0,0.05),0 1px 0 rgba(255,255,255,0.1)}
\ No newline at end of file