You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by pr...@apache.org on 2015/08/11 14:20:36 UTC

[13/50] [abbrv] incubator-lens git commit: LENS-477 : Add Lens server health check

LENS-477 : Add Lens server health check


Project: http://git-wip-us.apache.org/repos/asf/incubator-lens/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-lens/commit/651a9b6a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-lens/tree/651a9b6a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-lens/diff/651a9b6a

Branch: refs/heads/current-release-line
Commit: 651a9b6a41401fb637bc8130489f18654c0bca1f
Parents: ad8f4a8
Author: Yash Sharma <ya...@gmail.com>
Authored: Mon Jul 27 10:21:35 2015 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Mon Jul 27 10:21:35 2015 +0530

----------------------------------------------------------------------
 .../org/apache/lens/server/api/LensService.java |  25 ++
 .../server/api/events/LensEventService.java     |   4 +-
 .../lens/server/api/health/HealthCheckable.java |  29 ++
 .../lens/server/api/health/HealthStatus.java    |  32 ++
 .../api/metastore/CubeMetastoreService.java     |   3 +-
 .../lens/server/api/metrics/MetricsService.java |   5 +-
 .../org/apache/lens/server/BaseLensService.java | 433 +++++++++++++++++++
 .../apache/lens/server/EventServiceImpl.java    |  10 +
 .../org/apache/lens/server/LensService.java     | 424 ------------------
 .../org/apache/lens/server/LensServices.java    |  19 +-
 .../healthcheck/LensServiceHealthCheck.java     |  45 ++
 .../metastore/CubeMetastoreServiceImpl.java     |  36 +-
 .../lens/server/metrics/MetricsServiceImpl.java |  44 +-
 .../server/query/QueryExecutionServiceImpl.java |  57 ++-
 .../lens/server/quota/QuotaServiceImpl.java     |  21 +-
 .../scheduler/QuerySchedulerServiceImpl.java    |  22 +-
 .../lens/server/session/HiveSessionService.java |  20 +-
 .../lens/server/session/SessionResource.java    |   4 +-
 .../apache/lens/server/TestServerRestart.java   |  10 +
 .../server/healthcheck/TestHealthChecks.java    | 127 ++++++
 src/site/apt/user/cli.apt                       |   8 +-
 21 files changed, 920 insertions(+), 458 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server-api/src/main/java/org/apache/lens/server/api/LensService.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/LensService.java b/lens-server-api/src/main/java/org/apache/lens/server/api/LensService.java
new file mode 100644
index 0000000..2098c3e
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/LensService.java
@@ -0,0 +1,25 @@
+/**
+ * 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.lens.server.api;
+
+import org.apache.lens.server.api.health.HealthCheckable;
+
+public interface LensService extends HealthCheckable {
+  //TODO: Pull out method definitions from individual services
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server-api/src/main/java/org/apache/lens/server/api/events/LensEventService.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/events/LensEventService.java b/lens-server-api/src/main/java/org/apache/lens/server/api/events/LensEventService.java
index 74cc00c..4536a18 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/events/LensEventService.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/events/LensEventService.java
@@ -20,12 +20,13 @@ package org.apache.lens.server.api.events;
 
 import java.util.Collection;
 
+import org.apache.lens.server.api.LensService;
 import org.apache.lens.server.api.error.LensException;
 
 /**
  * Singleton which is responsible for managing the event-listener mapping as well as processing events.
  */
-public interface LensEventService {
+public interface LensEventService extends LensService {
 
   /**
    * The Constant NAME.
@@ -71,4 +72,5 @@ public interface LensEventService {
    * @return all the listeners
    */
   <T extends LensEvent> Collection<LensEventListener> getListeners(Class<T> changeType);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server-api/src/main/java/org/apache/lens/server/api/health/HealthCheckable.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/health/HealthCheckable.java b/lens-server-api/src/main/java/org/apache/lens/server/api/health/HealthCheckable.java
new file mode 100644
index 0000000..667165b
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/health/HealthCheckable.java
@@ -0,0 +1,29 @@
+/**
+ * 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.lens.server.api.health;
+
+public interface HealthCheckable {
+
+  /**
+   * Returns the health status of the service.
+   *
+   * @return
+   */
+  HealthStatus getHealthStatus();
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server-api/src/main/java/org/apache/lens/server/api/health/HealthStatus.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/health/HealthStatus.java b/lens-server-api/src/main/java/org/apache/lens/server/api/health/HealthStatus.java
new file mode 100644
index 0000000..21ce9da
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/health/HealthStatus.java
@@ -0,0 +1,32 @@
+/**
+ * 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.lens.server.api.health;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+@RequiredArgsConstructor
+public class HealthStatus {
+
+  @Getter
+  private final boolean isHealthy;
+  @Getter
+  private final String details;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java b/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
index bd31f82..d7a033e 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.metastore.*;
+import org.apache.lens.server.api.LensService;
 import org.apache.lens.server.api.error.LensException;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -30,7 +31,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 /**
  * Server api for OLAP Cube Metastore.
  */
-public interface CubeMetastoreService {
+public interface CubeMetastoreService extends LensService {
 
   /** The constant NAME */
   String NAME = "metastore";

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MetricsService.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MetricsService.java b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MetricsService.java
index 72fe455..7fd2d81 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MetricsService.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MetricsService.java
@@ -18,13 +18,15 @@
  */
 package org.apache.lens.server.api.metrics;
 
+import org.apache.lens.server.api.LensService;
+
 import org.glassfish.jersey.server.ContainerRequest;
 import org.glassfish.jersey.server.model.ResourceMethod;
 
 /**
  * The Interface MetricsService.
  */
-public interface MetricsService {
+public interface MetricsService extends LensService {
 
   /**
    * The Constant NAME.
@@ -193,4 +195,5 @@ public interface MetricsService {
    * Specifies the count of all opened sessions which are not closed after restarted and  the restored sessions
    */
   int getActiveSessions();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java b/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java
new file mode 100644
index 0000000..da11069
--- /dev/null
+++ b/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java
@@ -0,0 +1,433 @@
+/**
+ * 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.lens.server;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import javax.ws.rs.BadRequestException;
+import javax.ws.rs.ClientErrorException;
+import javax.ws.rs.NotAuthorizedException;
+import javax.ws.rs.NotFoundException;
+
+import org.apache.lens.api.LensConf;
+import org.apache.lens.api.LensSessionHandle;
+import org.apache.lens.server.api.LensConfConstants;
+import org.apache.lens.server.api.LensService;
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.events.LensEvent;
+import org.apache.lens.server.api.events.LensEventService;
+import org.apache.lens.server.api.health.HealthStatus;
+import org.apache.lens.server.session.LensSessionImpl;
+import org.apache.lens.server.user.UserConfigLoaderFactory;
+import org.apache.lens.server.util.UtilityMethods;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+
+import org.apache.hive.service.CompositeService;
+import org.apache.hive.service.auth.AuthenticationProviderFactory;
+import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.auth.PasswdAuthenticationProvider;
+import org.apache.hive.service.cli.CLIService;
+import org.apache.hive.service.cli.HandleIdentifier;
+import org.apache.hive.service.cli.HiveSQLException;
+import org.apache.hive.service.cli.SessionHandle;
+import org.apache.hive.service.cli.session.SessionManager;
+import org.apache.hive.service.cli.thrift.TSessionHandle;
+
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * The Class LensService.
+ */
+@Slf4j
+public abstract class BaseLensService extends CompositeService implements Externalizable, LensService {
+
+  /** The cli service. */
+  private final CLIService cliService;
+
+  /** The stopped. */
+  protected boolean stopped = false;
+
+  // Static session map which is used by query submission thread to get the
+  // lens session before submitting a query to hive server
+  /** The session map. */
+  protected static final ConcurrentHashMap<String, LensSessionHandle> SESSION_MAP
+    = new ConcurrentHashMap<String, LensSessionHandle>();
+
+  /**
+   * Instantiates a new lens service.
+   *
+   * @param name       the name
+   * @param cliService the cli service
+   */
+  protected BaseLensService(String name, CLIService cliService) {
+    super(name);
+    this.cliService = cliService;
+  }
+
+  /**
+   * @return the cliService
+   */
+  public CLIService getCliService() {
+    return cliService;
+  }
+
+  public String getServerDomain() {
+    return cliService.getHiveConf().get(LensConfConstants.SERVER_DOMAIN);
+  }
+
+  public static int getNumberOfSessions() {
+    return BaseLensService.SESSION_MAP.size();
+  }
+
+  /**
+   * Open session.
+   *
+   * @param username      the username
+   * @param password      the password
+   * @param configuration the configuration
+   * @return the lens session handle
+   * @throws LensException the lens exception
+   */
+  public LensSessionHandle openSession(String username, String password, Map<String, String> configuration)
+    throws LensException {
+    if (StringUtils.isBlank(username)) {
+      throw new BadRequestException("User name cannot be null or empty");
+    }
+    SessionHandle sessionHandle;
+    username = UtilityMethods.removeDomain(username);
+    doPasswdAuth(username, password);
+    try {
+      Map<String, String> sessionConf = new HashMap<String, String>();
+      sessionConf.putAll(LensSessionImpl.DEFAULT_HIVE_SESSION_CONF);
+      if (configuration != null) {
+        sessionConf.putAll(configuration);
+      }
+      Map<String, String> userConfig = UserConfigLoaderFactory.getUserConfig(username);
+      log.info("Got user config: {}", userConfig);
+      UtilityMethods.mergeMaps(sessionConf, userConfig, false);
+      sessionConf.put(LensConfConstants.SESSION_LOGGEDIN_USER, username);
+      if (sessionConf.get(LensConfConstants.SESSION_CLUSTER_USER) == null) {
+        log.info("Didn't get cluster user from user config loader. Setting same as logged in user: {}", username);
+        sessionConf.put(LensConfConstants.SESSION_CLUSTER_USER, username);
+      }
+      String clusterUser = sessionConf.get(LensConfConstants.SESSION_CLUSTER_USER);
+      password = "useless";
+      if (cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION)
+        .equals(HiveAuthFactory.AuthTypes.KERBEROS.toString())
+        && cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
+        String delegationTokenStr = null;
+        try {
+          delegationTokenStr = cliService.getDelegationTokenFromMetaStore(username);
+        } catch (UnsupportedOperationException e) {
+          // The delegation token is not applicable in the given deployment mode
+        }
+        sessionHandle = cliService.openSessionWithImpersonation(clusterUser, password, sessionConf, delegationTokenStr);
+      } else {
+        sessionHandle = cliService.openSession(clusterUser, password, sessionConf);
+      }
+    } catch (Exception e) {
+      throw new LensException(e);
+    }
+    LensSessionHandle lensSession = new LensSessionHandle(sessionHandle.getHandleIdentifier().getPublicId(),
+      sessionHandle.getHandleIdentifier().getSecretId());
+    SESSION_MAP.put(lensSession.getPublicId().toString(), lensSession);
+    return lensSession;
+  }
+
+  protected LensEventService getEventService() {
+    LensEventService  eventService = (LensEventService) LensServices.get().getService(LensEventService.NAME);
+    if (eventService == null) {
+      throw new NullPointerException("Could not get event service");
+    }
+    return eventService;
+  }
+
+  protected void notifyEvent(LensEvent event) throws LensException {
+    getEventService().notifyEvent(event);
+  }
+
+  /**
+   * Restore session from previous instance of lens server.
+   *
+   * @param sessionHandle the session handle
+   * @param userName      the user name
+   * @param password      the password
+   * @throws LensException the lens exception
+   */
+  public void restoreSession(LensSessionHandle sessionHandle, String userName, String password) throws LensException {
+    HandleIdentifier handleIdentifier = new HandleIdentifier(sessionHandle.getPublicId(), sessionHandle.getSecretId());
+    SessionHandle hiveSessionHandle = new SessionHandle(new TSessionHandle(handleIdentifier.toTHandleIdentifier()));
+    try {
+      SessionHandle restoredHandle = cliService.restoreSession(hiveSessionHandle, userName, password,
+        new HashMap<String, String>());
+      LensSessionHandle restoredSession = new LensSessionHandle(restoredHandle.getHandleIdentifier().getPublicId(),
+        restoredHandle.getHandleIdentifier().getSecretId());
+      SESSION_MAP.put(restoredSession.getPublicId().toString(), restoredSession);
+    } catch (HiveSQLException e) {
+      throw new LensException("Error restoring session " + sessionHandle, e);
+    }
+  }
+
+  /**
+   * Do passwd auth.
+   *
+   * @param userName the user name
+   * @param password the password
+   */
+  private void doPasswdAuth(String userName, String password) {
+    // Lens confs to Hive Confs.
+    for (ConfVars var : new ConfVars[]{ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN}) {
+      if (cliService.getHiveConf().getVar(var) == null) {
+        cliService.getHiveConf().setVar(var, cliService.getHiveConf().get(LensConfConstants.SERVER_DOMAIN));
+      }
+    }
+    String authType = cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION);
+    // No-op when authType is NOSASL
+    if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) {
+      try {
+        AuthenticationProviderFactory.AuthMethods authMethod = AuthenticationProviderFactory.AuthMethods
+          .getValidAuthMethod(authType);
+        PasswdAuthenticationProvider provider = AuthenticationProviderFactory.getAuthenticationProvider(authMethod,
+          cliService.getHiveConf());
+        provider.Authenticate(userName, password);
+      } catch (Exception e) {
+        log.error("Auth error: ", e);
+        throw new NotAuthorizedException(e);
+      }
+    }
+  }
+
+  /**
+   * Close session.
+   *
+   * @param sessionHandle the session handle
+   * @throws LensException the lens exception
+   */
+  public void closeSession(LensSessionHandle sessionHandle) throws LensException {
+    try {
+      cliService.closeSession(getHiveSessionHandle(sessionHandle));
+      SESSION_MAP.remove(sessionHandle.getPublicId().toString());
+    } catch (Exception e) {
+      throw new LensException(e);
+    }
+  }
+
+  public SessionManager getSessionManager() {
+    return cliService.getSessionManager();
+  }
+
+  /**
+   * Gets the session.
+   *
+   * @param sessionHandle the session handle
+   * @return the session
+   */
+  public LensSessionImpl getSession(LensSessionHandle sessionHandle) {
+    if (sessionHandle == null) {
+      throw new ClientErrorException("Session is null " + sessionHandle, 400);
+    }
+
+    try {
+      return ((LensSessionImpl) getSessionManager().getSession(getHiveSessionHandle(sessionHandle)));
+    } catch (HiveSQLException exc) {
+      log.warn("Session {} not found", sessionHandle.getPublicId(), exc);
+      // throw resource gone exception (410)
+      throw new ClientErrorException("Session " + sessionHandle.getPublicId() + " is invalid " + sessionHandle, 410);
+    }
+  }
+
+  /**
+   * Acquire.
+   *
+   * @param sessionHandle the session handle
+   */
+  public void acquire(LensSessionHandle sessionHandle) {
+    if (sessionHandle != null) {
+      log.debug("Acquiring lens session:{}", sessionHandle.getPublicId());
+      getSession(sessionHandle).acquire();
+    }
+  }
+
+  /**
+   * Acquire a lens session specified by the public UUID.
+   *
+   * @param sessionHandle public UUID of the session
+   */
+  public void acquire(String sessionHandle) {
+    LensSessionHandle handle = SESSION_MAP.get(sessionHandle);
+
+    if (handle == null) {
+      throw new NotFoundException("Session handle not found " + sessionHandle);
+    }
+
+    acquire(handle);
+  }
+
+  /**
+   * Release.
+   *
+   * @param sessionHandle the session handle
+   */
+  public void release(LensSessionHandle sessionHandle) {
+    if (sessionHandle != null) {
+      getSession(sessionHandle).release();
+      log.debug("Released lens session:{}", sessionHandle.getPublicId());
+    }
+  }
+
+  /**
+   * Releases a lens session specified by the public UUID.
+   *
+   * @param sessionHandle the session handle
+   * @throws LensException if session cannot be released
+   */
+  public void release(String sessionHandle) throws LensException {
+    LensSessionHandle handle = SESSION_MAP.get(sessionHandle);
+    if (handle != null) {
+      getSession(handle).release();
+    }
+  }
+
+  /**
+   * Gets the session handle.
+   *
+   * @param sessionid the sessionid
+   * @return the session handle
+   */
+  protected LensSessionHandle getSessionHandle(String sessionid) {
+    return SESSION_MAP.get(sessionid);
+  }
+
+  /**
+   * Adds the resource.
+   *
+   * @param sessionHandle the session handle
+   * @param type          the type
+   * @param path          the path
+   * @throws LensException the lens exception
+   */
+  public void addResource(LensSessionHandle sessionHandle, String type, String path) throws LensException {
+  }
+
+  /**
+   * Delete resource.
+   *
+   * @param sessionHandle the session handle
+   * @param type          the type
+   * @param path          the path
+   * @throws LensException the lens exception
+   */
+  public void deleteResource(LensSessionHandle sessionHandle, String type, String path) throws LensException {
+  }
+
+  /**
+   * Gets the hive session handle.
+   *
+   * @param lensHandle the lens handle
+   * @return the hive session handle
+   */
+  public static SessionHandle getHiveSessionHandle(LensSessionHandle lensHandle) {
+    return new SessionHandle(new HandleIdentifier(lensHandle.getPublicId(), lensHandle.getSecretId()),
+      CLIService.SERVER_VERSION);
+  }
+
+  /**
+   * Gets the lens conf.
+   *
+   * @param sessionHandle the session handle
+   * @param conf          the conf
+   * @return the lens conf
+   * @throws LensException the lens exception
+   */
+  public Configuration getLensConf(LensSessionHandle sessionHandle, LensConf conf) throws LensException {
+    Configuration qconf = new Configuration(false);
+    for (Map.Entry<String, String> entry : getSession(sessionHandle).getSessionConf()) {
+      qconf.set(entry.getKey(), entry.getValue());
+    }
+
+    if (conf != null && !conf.getProperties().isEmpty()) {
+      for (Map.Entry<String, String> entry : conf.getProperties().entrySet()) {
+        qconf.set(entry.getKey(), entry.getValue());
+      }
+    }
+    qconf.setClassLoader(getSession(sessionHandle).getClassLoader());
+    return qconf;
+  }
+
+  /**
+   * Gets the lens conf.
+   *
+   * @param conf the conf
+   * @return the lens conf
+   * @throws LensException the lens exception
+   */
+  public Configuration getLensConf(LensConf conf) throws LensException {
+    Configuration qconf = LensSessionImpl.createDefaultConf();
+
+    if (conf != null && !conf.getProperties().isEmpty()) {
+      for (Map.Entry<String, String> entry : conf.getProperties().entrySet()) {
+        qconf.set(entry.getKey(), entry.getValue());
+      }
+    }
+    return qconf;
+  }
+
+  /**
+   * Prepare stopping.
+   */
+  public void prepareStopping() {
+    this.stopped = true;
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see java.io.Externalizable#readExternal(java.io.ObjectInput)
+   */
+  @Override
+  public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see java.io.Externalizable#writeExternal(java.io.ObjectOutput)
+   */
+  @Override
+  public void writeExternal(ObjectOutput out) throws IOException {
+  }
+
+  /**
+   * Returns the health status of the service.
+   *
+   * @return
+   */
+  public abstract HealthStatus getHealthStatus();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/EventServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/EventServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/EventServiceImpl.java
index 2c6f912..a276828 100644
--- a/lens-server/src/main/java/org/apache/lens/server/EventServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/EventServiceImpl.java
@@ -27,6 +27,7 @@ import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.events.LensEvent;
 import org.apache.lens.server.api.events.LensEventListener;
 import org.apache.lens.server.api.events.LensEventService;
+import org.apache.lens.server.api.health.HealthStatus;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hive.service.AbstractService;
@@ -171,6 +172,15 @@ public class EventServiceImpl extends AbstractService implements LensEventServic
     return Collections.unmodifiableList(eventListeners.get(eventType));
   }
 
+  @Override
+  public HealthStatus getHealthStatus() {
+    return (this.getServiceState().equals(STATE.STARTED)
+        && !eventHandlerPool.isShutdown()
+        && !eventHandlerPool.isTerminated())
+        ? new HealthStatus(true, "Event service is healthy.")
+        : new HealthStatus(false, "Event service is unhealthy.");
+  }
+
   /*
    * (non-Javadoc)
    *

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/LensService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/LensService.java b/lens-server/src/main/java/org/apache/lens/server/LensService.java
deleted file mode 100644
index 0e564a9..0000000
--- a/lens-server/src/main/java/org/apache/lens/server/LensService.java
+++ /dev/null
@@ -1,424 +0,0 @@
-/**
- * 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.lens.server;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.ws.rs.BadRequestException;
-import javax.ws.rs.ClientErrorException;
-import javax.ws.rs.NotAuthorizedException;
-import javax.ws.rs.NotFoundException;
-
-import org.apache.lens.api.LensConf;
-import org.apache.lens.api.LensSessionHandle;
-import org.apache.lens.server.api.LensConfConstants;
-import org.apache.lens.server.api.error.LensException;
-import org.apache.lens.server.api.events.LensEvent;
-import org.apache.lens.server.api.events.LensEventService;
-import org.apache.lens.server.session.LensSessionImpl;
-import org.apache.lens.server.user.UserConfigLoaderFactory;
-import org.apache.lens.server.util.UtilityMethods;
-
-import org.apache.commons.lang3.StringUtils;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-
-import org.apache.hive.service.CompositeService;
-import org.apache.hive.service.auth.AuthenticationProviderFactory;
-import org.apache.hive.service.auth.HiveAuthFactory;
-import org.apache.hive.service.auth.PasswdAuthenticationProvider;
-import org.apache.hive.service.cli.CLIService;
-import org.apache.hive.service.cli.HandleIdentifier;
-import org.apache.hive.service.cli.HiveSQLException;
-import org.apache.hive.service.cli.SessionHandle;
-import org.apache.hive.service.cli.session.SessionManager;
-import org.apache.hive.service.cli.thrift.TSessionHandle;
-
-import lombok.extern.slf4j.Slf4j;
-
-/**
- * The Class LensService.
- */
-@Slf4j
-public abstract class LensService extends CompositeService implements Externalizable {
-
-  /** The cli service. */
-  private final CLIService cliService;
-
-  /** The stopped. */
-  protected boolean stopped = false;
-
-  // Static session map which is used by query submission thread to get the
-  // lens session before submitting a query to hive server
-  /** The session map. */
-  protected static final ConcurrentHashMap<String, LensSessionHandle> SESSION_MAP
-    = new ConcurrentHashMap<String, LensSessionHandle>();
-
-  /**
-   * Instantiates a new lens service.
-   *
-   * @param name       the name
-   * @param cliService the cli service
-   */
-  protected LensService(String name, CLIService cliService) {
-    super(name);
-    this.cliService = cliService;
-  }
-
-  /**
-   * @return the cliService
-   */
-  public CLIService getCliService() {
-    return cliService;
-  }
-
-  public String getServerDomain() {
-    return cliService.getHiveConf().get(LensConfConstants.SERVER_DOMAIN);
-  }
-
-  public static int getNumberOfSessions() {
-    return LensService.SESSION_MAP.size();
-  }
-
-  /**
-   * Open session.
-   *
-   * @param username      the username
-   * @param password      the password
-   * @param configuration the configuration
-   * @return the lens session handle
-   * @throws LensException the lens exception
-   */
-  public LensSessionHandle openSession(String username, String password, Map<String, String> configuration)
-    throws LensException {
-    if (StringUtils.isBlank(username)) {
-      throw new BadRequestException("User name cannot be null or empty");
-    }
-    SessionHandle sessionHandle;
-    username = UtilityMethods.removeDomain(username);
-    doPasswdAuth(username, password);
-    try {
-      Map<String, String> sessionConf = new HashMap<String, String>();
-      sessionConf.putAll(LensSessionImpl.DEFAULT_HIVE_SESSION_CONF);
-      if (configuration != null) {
-        sessionConf.putAll(configuration);
-      }
-      Map<String, String> userConfig = UserConfigLoaderFactory.getUserConfig(username);
-      log.info("Got user config: {}", userConfig);
-      UtilityMethods.mergeMaps(sessionConf, userConfig, false);
-      sessionConf.put(LensConfConstants.SESSION_LOGGEDIN_USER, username);
-      if (sessionConf.get(LensConfConstants.SESSION_CLUSTER_USER) == null) {
-        log.info("Didn't get cluster user from user config loader. Setting same as logged in user: {}", username);
-        sessionConf.put(LensConfConstants.SESSION_CLUSTER_USER, username);
-      }
-      String clusterUser = sessionConf.get(LensConfConstants.SESSION_CLUSTER_USER);
-      password = "useless";
-      if (cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION)
-        .equals(HiveAuthFactory.AuthTypes.KERBEROS.toString())
-        && cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
-        String delegationTokenStr = null;
-        try {
-          delegationTokenStr = cliService.getDelegationTokenFromMetaStore(username);
-        } catch (UnsupportedOperationException e) {
-          // The delegation token is not applicable in the given deployment mode
-        }
-        sessionHandle = cliService.openSessionWithImpersonation(clusterUser, password, sessionConf, delegationTokenStr);
-      } else {
-        sessionHandle = cliService.openSession(clusterUser, password, sessionConf);
-      }
-    } catch (Exception e) {
-      throw new LensException(e);
-    }
-    LensSessionHandle lensSession = new LensSessionHandle(sessionHandle.getHandleIdentifier().getPublicId(),
-      sessionHandle.getHandleIdentifier().getSecretId());
-    SESSION_MAP.put(lensSession.getPublicId().toString(), lensSession);
-    return lensSession;
-  }
-
-  protected LensEventService getEventService() {
-    LensEventService  eventService = (LensEventService) LensServices.get().getService(LensEventService.NAME);
-    if (eventService == null) {
-      throw new NullPointerException("Could not get event service");
-    }
-    return eventService;
-  }
-
-  protected void notifyEvent(LensEvent event) throws LensException {
-    getEventService().notifyEvent(event);
-  }
-
-  /**
-   * Restore session from previous instance of lens server.
-   *
-   * @param sessionHandle the session handle
-   * @param userName      the user name
-   * @param password      the password
-   * @throws LensException the lens exception
-   */
-  public void restoreSession(LensSessionHandle sessionHandle, String userName, String password) throws LensException {
-    HandleIdentifier handleIdentifier = new HandleIdentifier(sessionHandle.getPublicId(), sessionHandle.getSecretId());
-    SessionHandle hiveSessionHandle = new SessionHandle(new TSessionHandle(handleIdentifier.toTHandleIdentifier()));
-    try {
-      SessionHandle restoredHandle = cliService.restoreSession(hiveSessionHandle, userName, password,
-        new HashMap<String, String>());
-      LensSessionHandle restoredSession = new LensSessionHandle(restoredHandle.getHandleIdentifier().getPublicId(),
-        restoredHandle.getHandleIdentifier().getSecretId());
-      SESSION_MAP.put(restoredSession.getPublicId().toString(), restoredSession);
-    } catch (HiveSQLException e) {
-      throw new LensException("Error restoring session " + sessionHandle, e);
-    }
-  }
-
-  /**
-   * Do passwd auth.
-   *
-   * @param userName the user name
-   * @param password the password
-   */
-  private void doPasswdAuth(String userName, String password) {
-    // Lens confs to Hive Confs.
-    for (ConfVars var : new ConfVars[]{ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN}) {
-      if (cliService.getHiveConf().getVar(var) == null) {
-        cliService.getHiveConf().setVar(var, cliService.getHiveConf().get(LensConfConstants.SERVER_DOMAIN));
-      }
-    }
-    String authType = cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION);
-    // No-op when authType is NOSASL
-    if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) {
-      try {
-        AuthenticationProviderFactory.AuthMethods authMethod = AuthenticationProviderFactory.AuthMethods
-          .getValidAuthMethod(authType);
-        PasswdAuthenticationProvider provider = AuthenticationProviderFactory.getAuthenticationProvider(authMethod,
-          cliService.getHiveConf());
-        provider.Authenticate(userName, password);
-      } catch (Exception e) {
-        log.error("Auth error: ", e);
-        throw new NotAuthorizedException(e);
-      }
-    }
-  }
-
-  /**
-   * Close session.
-   *
-   * @param sessionHandle the session handle
-   * @throws LensException the lens exception
-   */
-  public void closeSession(LensSessionHandle sessionHandle) throws LensException {
-    try {
-      cliService.closeSession(getHiveSessionHandle(sessionHandle));
-      SESSION_MAP.remove(sessionHandle.getPublicId().toString());
-    } catch (Exception e) {
-      throw new LensException(e);
-    }
-  }
-
-  public SessionManager getSessionManager() {
-    return cliService.getSessionManager();
-  }
-
-  /**
-   * Gets the session.
-   *
-   * @param sessionHandle the session handle
-   * @return the session
-   */
-  public LensSessionImpl getSession(LensSessionHandle sessionHandle) {
-    if (sessionHandle == null) {
-      throw new ClientErrorException("Session is null " + sessionHandle, 400);
-    }
-
-    try {
-      return ((LensSessionImpl) getSessionManager().getSession(getHiveSessionHandle(sessionHandle)));
-    } catch (HiveSQLException exc) {
-      log.warn("Session {} not found", sessionHandle.getPublicId(), exc);
-      // throw resource gone exception (410)
-      throw new ClientErrorException("Session " + sessionHandle.getPublicId() + " is invalid " + sessionHandle, 410);
-    }
-  }
-
-  /**
-   * Acquire.
-   *
-   * @param sessionHandle the session handle
-   */
-  public void acquire(LensSessionHandle sessionHandle) {
-    if (sessionHandle != null) {
-      log.debug("Acquiring lens session:{}", sessionHandle.getPublicId());
-      getSession(sessionHandle).acquire();
-    }
-  }
-
-  /**
-   * Acquire a lens session specified by the public UUID.
-   *
-   * @param sessionHandle public UUID of the session
-   */
-  public void acquire(String sessionHandle) {
-    LensSessionHandle handle = SESSION_MAP.get(sessionHandle);
-
-    if (handle == null) {
-      throw new NotFoundException("Session handle not found " + sessionHandle);
-    }
-
-    acquire(handle);
-  }
-
-  /**
-   * Release.
-   *
-   * @param sessionHandle the session handle
-   */
-  public void release(LensSessionHandle sessionHandle) {
-    if (sessionHandle != null) {
-      getSession(sessionHandle).release();
-      log.debug("Released lens session:{}", sessionHandle.getPublicId());
-    }
-  }
-
-  /**
-   * Releases a lens session specified by the public UUID.
-   *
-   * @param sessionHandle the session handle
-   * @throws LensException if session cannot be released
-   */
-  public void release(String sessionHandle) throws LensException {
-    LensSessionHandle handle = SESSION_MAP.get(sessionHandle);
-    if (handle != null) {
-      getSession(handle).release();
-    }
-  }
-
-  /**
-   * Gets the session handle.
-   *
-   * @param sessionid the sessionid
-   * @return the session handle
-   */
-  protected LensSessionHandle getSessionHandle(String sessionid) {
-    return SESSION_MAP.get(sessionid);
-  }
-
-  /**
-   * Adds the resource.
-   *
-   * @param sessionHandle the session handle
-   * @param type          the type
-   * @param path          the path
-   * @throws LensException the lens exception
-   */
-  public void addResource(LensSessionHandle sessionHandle, String type, String path) throws LensException {
-  }
-
-  /**
-   * Delete resource.
-   *
-   * @param sessionHandle the session handle
-   * @param type          the type
-   * @param path          the path
-   * @throws LensException the lens exception
-   */
-  public void deleteResource(LensSessionHandle sessionHandle, String type, String path) throws LensException {
-  }
-
-  /**
-   * Gets the hive session handle.
-   *
-   * @param lensHandle the lens handle
-   * @return the hive session handle
-   */
-  public static SessionHandle getHiveSessionHandle(LensSessionHandle lensHandle) {
-    return new SessionHandle(new HandleIdentifier(lensHandle.getPublicId(), lensHandle.getSecretId()),
-      CLIService.SERVER_VERSION);
-  }
-
-  /**
-   * Gets the lens conf.
-   *
-   * @param sessionHandle the session handle
-   * @param conf          the conf
-   * @return the lens conf
-   * @throws LensException the lens exception
-   */
-  public Configuration getLensConf(LensSessionHandle sessionHandle, LensConf conf) throws LensException {
-    Configuration qconf = new Configuration(false);
-    for (Map.Entry<String, String> entry : getSession(sessionHandle).getSessionConf()) {
-      qconf.set(entry.getKey(), entry.getValue());
-    }
-
-    if (conf != null && !conf.getProperties().isEmpty()) {
-      for (Map.Entry<String, String> entry : conf.getProperties().entrySet()) {
-        qconf.set(entry.getKey(), entry.getValue());
-      }
-    }
-    qconf.setClassLoader(getSession(sessionHandle).getClassLoader());
-    return qconf;
-  }
-
-  /**
-   * Gets the lens conf.
-   *
-   * @param conf the conf
-   * @return the lens conf
-   * @throws LensException the lens exception
-   */
-  public Configuration getLensConf(LensConf conf) throws LensException {
-    Configuration qconf = LensSessionImpl.createDefaultConf();
-
-    if (conf != null && !conf.getProperties().isEmpty()) {
-      for (Map.Entry<String, String> entry : conf.getProperties().entrySet()) {
-        qconf.set(entry.getKey(), entry.getValue());
-      }
-    }
-    return qconf;
-  }
-
-  /**
-   * Prepare stopping.
-   */
-  public void prepareStopping() {
-    this.stopped = true;
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see java.io.Externalizable#readExternal(java.io.ObjectInput)
-   */
-  @Override
-  public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see java.io.Externalizable#writeExternal(java.io.ObjectOutput)
-   */
-  @Override
-  public void writeExternal(ObjectOutput out) throws IOException {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/LensServices.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/LensServices.java b/lens-server/src/main/java/org/apache/lens/server/LensServices.java
index 1bcc5944..c4a61c8 100644
--- a/lens-server/src/main/java/org/apache/lens/server/LensServices.java
+++ b/lens-server/src/main/java/org/apache/lens/server/LensServices.java
@@ -80,7 +80,7 @@ public class LensServices extends CompositeService implements ServiceProvider {
   private final Map<String, Service> services = new LinkedHashMap<String, Service>();
 
   /** The lens services. */
-  private final List<LensService> lensServices = new ArrayList<LensService>();
+  private final List<BaseLensService> lensServices = new ArrayList<BaseLensService>();
 
   /** The persist dir. */
   private Path persistDir;
@@ -209,11 +209,11 @@ public class LensServices extends CompositeService implements ServiceProvider {
 
           Class<?> cls = Class.forName(serviceClassName);
 
-          if (LensService.class.isAssignableFrom(cls)) {
-            Class<? extends LensService> serviceClass = (Class<? extends LensService>) cls;
+          if (BaseLensService.class.isAssignableFrom(cls)) {
+            Class<? extends BaseLensService> serviceClass = (Class<? extends BaseLensService>) cls;
             log.info("Adding {}  service with {}", sName, serviceClass);
             Constructor<?> constructor = serviceClass.getConstructor(CLIService.class);
-            LensService service = (LensService) constructor.newInstance(new Object[]{cliService});
+            BaseLensService service = (BaseLensService) constructor.newInstance(new Object[]{cliService});
             addService(service);
             lensServices.add(service);
           } else if (Service.class.isAssignableFrom(cls)) {
@@ -297,7 +297,7 @@ public class LensServices extends CompositeService implements ServiceProvider {
     if (conf.getBoolean(SERVER_RECOVER_ON_RESTART,
       DEFAULT_SERVER_RECOVER_ON_RESTART)) {
 
-      for (LensService service : lensServices) {
+      for (BaseLensService service : lensServices) {
         ObjectInputStream in = null;
         try {
           try {
@@ -330,7 +330,8 @@ public class LensServices extends CompositeService implements ServiceProvider {
           log.info("Persisting server state in {}", persistDir);
 
           long now = System.currentTimeMillis();
-          for (LensService service : lensServices) {
+
+          for (BaseLensService service : lensServices) {
             log.info("Persisting state of service: {}", service.getName());
             Path serviceWritePath = new Path(persistDir, service.getName() + ".out" + "." + now);
             ObjectOutputStream out = null;
@@ -369,7 +370,7 @@ public class LensServices extends CompositeService implements ServiceProvider {
    * @param service the service
    * @return the service persist path
    */
-  private Path getServicePersistPath(LensService service) {
+  private Path getServicePersistPath(BaseLensService service) {
     return new Path(persistDir, service.getName() + ".final");
   }
 
@@ -382,7 +383,7 @@ public class LensServices extends CompositeService implements ServiceProvider {
     if (getServiceState() != STATE.STOPPED) {
       log.info("Stopping lens server");
       stopping = true;
-      for (LensService service : lensServices) {
+      for (BaseLensService service : lensServices) {
         service.prepareStopping();
       }
 
@@ -434,7 +435,7 @@ public class LensServices extends CompositeService implements ServiceProvider {
     return (T) services.get(sName);
   }
 
-  public List<LensService> getLensServices() {
+  public List<BaseLensService> getLensServices() {
     return lensServices;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/healthcheck/LensServiceHealthCheck.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/healthcheck/LensServiceHealthCheck.java b/lens-server/src/main/java/org/apache/lens/server/healthcheck/LensServiceHealthCheck.java
new file mode 100644
index 0000000..b811b66
--- /dev/null
+++ b/lens-server/src/main/java/org/apache/lens/server/healthcheck/LensServiceHealthCheck.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.lens.server.healthcheck;
+
+import org.apache.lens.server.LensServices;
+import org.apache.lens.server.api.LensService;
+import org.apache.lens.server.api.health.HealthStatus;
+
+import com.codahale.metrics.health.HealthCheck;
+
+public class LensServiceHealthCheck extends HealthCheck {
+
+  private final LensService service;
+
+  public LensServiceHealthCheck(String serviceName) {
+    this.service = LensServices.get().getService(serviceName);
+  }
+
+  @Override
+  protected Result check() throws Exception {
+    HealthStatus status = service.getHealthStatus();
+
+    if (status.isHealthy()) {
+      return Result.healthy();
+    } else {
+      return Result.unhealthy(status.getDetails());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
index 646de8c..cb7a530 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
@@ -28,8 +28,10 @@ import org.apache.lens.api.metastore.*;
 import org.apache.lens.cube.metadata.*;
 import org.apache.lens.cube.metadata.Dimension;
 import org.apache.lens.cube.metadata.timeline.PartitionTimeline;
-import org.apache.lens.server.LensService;
+import org.apache.lens.server.BaseLensService;
+import org.apache.lens.server.LensServerConf;
 import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.health.HealthStatus;
 import org.apache.lens.server.api.metastore.CubeMetastoreService;
 import org.apache.lens.server.session.LensSessionImpl;
 
@@ -47,7 +49,7 @@ import com.google.common.collect.Lists;
 import lombok.extern.slf4j.Slf4j;
 
 @Slf4j
-public class CubeMetastoreServiceImpl extends LensService implements CubeMetastoreService {
+public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMetastoreService {
 
   public CubeMetastoreServiceImpl(CLIService cliService) {
     super(NAME, cliService);
@@ -1470,4 +1472,34 @@ public class CubeMetastoreServiceImpl extends LensService implements CubeMetasto
       release(sessionHandle);
     }
   }
+
+  /**
+   * @inheritDoc
+   */
+  @Override
+  public HealthStatus getHealthStatus() {
+    boolean isHealthy = true;
+    StringBuilder details = new StringBuilder();
+
+    try{
+      /** Try to issue command on hive **/
+      Hive.get(LensServerConf.getHiveConf()).getAllDatabases();
+    } catch (HiveException e) {
+      isHealthy = false;
+      details.append("Could not connect to Hive.");
+      log.error("Could not connect to Hive.", e);
+    }
+
+    /** Check if service is up **/
+    if (!this.getServiceState().equals(STATE.STARTED)) {
+      isHealthy = false;
+      details.append("Cube metastore service is down");
+      log.error("Cube metastore service is down");
+    }
+
+    return isHealthy
+        ? new HealthStatus(isHealthy, "Cube metastore service is healthy.")
+        : new HealthStatus(isHealthy, details.toString());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java
index 2455232..e430324 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java
@@ -29,11 +29,14 @@ import java.util.Locale;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.lens.api.query.QueryStatus.Status;
-import org.apache.lens.server.LensService;
+import org.apache.lens.server.BaseLensService;
+import org.apache.lens.server.EventServiceImpl;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.events.AsyncEventListener;
 import org.apache.lens.server.api.events.LensEventService;
+import org.apache.lens.server.api.health.HealthStatus;
+import org.apache.lens.server.api.metastore.CubeMetastoreService;
 import org.apache.lens.server.api.metrics.*;
 import org.apache.lens.server.api.query.QueryExecutionService;
 import org.apache.lens.server.api.query.StatusChange;
@@ -42,7 +45,12 @@ import org.apache.lens.server.api.session.SessionEvent;
 import org.apache.lens.server.api.session.SessionExpired;
 import org.apache.lens.server.api.session.SessionOpened;
 import org.apache.lens.server.api.session.SessionService;
+import org.apache.lens.server.healthcheck.LensServiceHealthCheck;
+import org.apache.lens.server.query.QueryExecutionServiceImpl;
+import org.apache.lens.server.quota.QuotaServiceImpl;
+import org.apache.lens.server.scheduler.QuerySchedulerServiceImpl;
 import org.apache.lens.server.session.DatabaseResourceService;
+import org.apache.lens.server.session.HiveSessionService;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hive.service.AbstractService;
@@ -252,6 +260,13 @@ public class MetricsServiceImpl extends AbstractService implements MetricsServic
     methodMetricsFactory = new MethodMetricsFactory(metricRegistry);
     setEnableResourceMethodMetering(hiveConf.getBoolean(LensConfConstants.ENABLE_RESOURCE_METHOD_METERING, false));
     healthCheck = new HealthCheckRegistry();
+    healthCheck.register(CubeMetastoreService.NAME, new LensServiceHealthCheck(CubeMetastoreService.NAME));
+    healthCheck.register(HiveSessionService.NAME, new LensServiceHealthCheck(HiveSessionService.NAME));
+    healthCheck.register(QueryExecutionServiceImpl.NAME, new LensServiceHealthCheck(QueryExecutionServiceImpl.NAME));
+    healthCheck.register(QuerySchedulerServiceImpl.NAME, new LensServiceHealthCheck(QuerySchedulerServiceImpl.NAME));
+    healthCheck.register(QuotaServiceImpl.NAME, new LensServiceHealthCheck(QuotaServiceImpl.NAME));
+    healthCheck.register(MetricsServiceImpl.NAME, new LensServiceHealthCheck(MetricsServiceImpl.NAME));
+    healthCheck.register(EventServiceImpl.NAME, new LensServiceHealthCheck(EventServiceImpl.NAME));
     initCounters();
     timeBetweenPolls = hiveConf.getInt(LensConfConstants.REPORTING_PERIOD, 10);
 
@@ -313,7 +328,7 @@ public class MetricsServiceImpl extends AbstractService implements MetricsServic
       new Gauge<Integer>() {
         @Override
         public Integer getValue() {
-          return LensService.getNumberOfSessions();
+          return BaseLensService.getNumberOfSessions();
         }
       });
 
@@ -554,6 +569,31 @@ public class MetricsServiceImpl extends AbstractService implements MetricsServic
   }
 
   @Override
+  public HealthStatus getHealthStatus() {
+    boolean isHealthy = true;
+    StringBuilder details = new StringBuilder();
+
+    if (!this.getServiceState().equals(STATE.STARTED)) {
+      details.append("Metric service is down.");
+      isHealthy = false;
+    }
+
+    // Also unhealthy if 30% of queries have failed.
+    if (getTotalFailedQueries()/(float)getTotalAcceptedQueries() > 0.3) {
+      details.append("30% of queries have failed.");
+      isHealthy = false;
+    }
+
+    if (!isHealthy) {
+      log.error(details.toString());
+    }
+
+    return isHealthy
+        ? new HealthStatus(true, "Metric service is healthy.")
+        : new HealthStatus(false, details.toString());
+  }
+
+  @Override
   public long getTotalOpenedSessions() {
     return totalOpenedSessions.getCount();
   }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
index 5e0f24a..1316257 100644
--- a/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
@@ -44,13 +44,14 @@ import org.apache.lens.api.query.*;
 import org.apache.lens.api.query.QueryStatus.Status;
 import org.apache.lens.api.result.LensErrorTO;
 import org.apache.lens.driver.hive.HiveDriver;
+import org.apache.lens.server.BaseLensService;
 import org.apache.lens.server.LensServerConf;
-import org.apache.lens.server.LensService;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.driver.*;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.error.LensMultiCauseException;
 import org.apache.lens.server.api.events.LensEventListener;
+import org.apache.lens.server.api.health.HealthStatus;
 import org.apache.lens.server.api.metrics.MethodMetricsContext;
 import org.apache.lens.server.api.metrics.MethodMetricsFactory;
 import org.apache.lens.server.api.metrics.MetricsService;
@@ -97,7 +98,7 @@ import lombok.extern.slf4j.Slf4j;
  * The Class QueryExecutionServiceImpl.
  */
 @Slf4j
-public class QueryExecutionServiceImpl extends LensService implements QueryExecutionService {
+public class QueryExecutionServiceImpl extends BaseLensService implements QueryExecutionService {
 
   /**
    * The Constant PREPARED_QUERIES_COUNTER.
@@ -2436,6 +2437,58 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
     }
   }
 
+  /**
+   * @inheritDoc
+   */
+  @Override
+  public HealthStatus getHealthStatus() {
+    boolean isHealthy = true;
+    StringBuilder details = new StringBuilder();
+
+    if (!this.getServiceState().equals(STATE.STARTED)) {
+      isHealthy = false;
+      details.append("Query execution service is down.");
+    }
+
+    if (!this.statusPoller.isAlive()) {
+      isHealthy = false;
+      details.append("Status poller thread is dead.");
+    }
+
+    if (!this.prepareQueryPurger.isAlive()) {
+      isHealthy = false;
+      details.append("PrepareQuery purger thread is dead.");
+    }
+
+    if (!this.queryPurger.isAlive()) {
+      isHealthy = false;
+      details.append("Query purger thread is dead.");
+    }
+
+    if (!this.querySubmitter.isAlive()) {
+      isHealthy = false;
+      details.append("Query submitter thread is dead.");
+    }
+
+    if (this.estimatePool.isShutdown() || this.estimatePool.isTerminated()) {
+      isHealthy = false;
+      details.append("Estimate Pool is dead.");
+    }
+
+    if (querySubmitterRunnable.pausedForTest) {
+      isHealthy = false;
+      details.append("QuerySubmitter paused for test.");
+    }
+
+    if (!isHealthy) {
+      log.error(details.toString());
+    }
+
+    return isHealthy
+        ? new HealthStatus(isHealthy, "QueryExecution service is healthy.")
+        : new HealthStatus(isHealthy, details.toString());
+  }
+
   /*
    * (non-Javadoc)
    *

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/quota/QuotaServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/quota/QuotaServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/quota/QuotaServiceImpl.java
index 89cfe71..cc627cf 100644
--- a/lens-server/src/main/java/org/apache/lens/server/quota/QuotaServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/quota/QuotaServiceImpl.java
@@ -18,7 +18,8 @@
  */
 package org.apache.lens.server.quota;
 
-import org.apache.lens.server.LensService;
+import org.apache.lens.server.BaseLensService;
+import org.apache.lens.server.api.health.HealthStatus;
 import org.apache.lens.server.api.quota.QuotaService;
 
 import org.apache.hive.service.cli.CLIService;
@@ -26,7 +27,12 @@ import org.apache.hive.service.cli.CLIService;
 /**
  * The Class QuotaServiceImpl.
  */
-public class QuotaServiceImpl extends LensService implements QuotaService {
+public class QuotaServiceImpl extends BaseLensService implements QuotaService {
+
+  /**
+   * The constant name for quota service.
+   */
+  public static final String NAME = "quota";
 
   /**
    * Instantiates a new quota service impl.
@@ -34,7 +40,16 @@ public class QuotaServiceImpl extends LensService implements QuotaService {
    * @param cliService the cli service
    */
   public QuotaServiceImpl(CLIService cliService) {
-    super("quota", cliService);
+    super(NAME, cliService);
   }
 
+  /**
+   * @inheritDoc
+   */
+  @Override
+  public HealthStatus getHealthStatus() {
+    return this.getServiceState().equals(STATE.STARTED)
+        ? new HealthStatus(true, "Quota service is healthy.")
+        : new HealthStatus(false, "Quota service is down.");
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/scheduler/QuerySchedulerServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/scheduler/QuerySchedulerServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/scheduler/QuerySchedulerServiceImpl.java
index 32feeb9..3ea7524 100644
--- a/lens-server/src/main/java/org/apache/lens/server/scheduler/QuerySchedulerServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/scheduler/QuerySchedulerServiceImpl.java
@@ -18,7 +18,8 @@
  */
 package org.apache.lens.server.scheduler;
 
-import org.apache.lens.server.LensService;
+import org.apache.lens.server.BaseLensService;
+import org.apache.lens.server.api.health.HealthStatus;
 import org.apache.lens.server.api.scheduler.QuerySchedulerService;
 
 import org.apache.hive.service.cli.CLIService;
@@ -26,7 +27,12 @@ import org.apache.hive.service.cli.CLIService;
 /**
  * The Class QuerySchedulerServiceImpl.
  */
-public class QuerySchedulerServiceImpl extends LensService implements QuerySchedulerService {
+public class QuerySchedulerServiceImpl extends BaseLensService implements QuerySchedulerService {
+
+  /**
+   * The constant name for scheduler service.
+   */
+  public static final String NAME = "scheduler";
 
   /**
    * Instantiates a new query scheduler service impl.
@@ -34,6 +40,16 @@ public class QuerySchedulerServiceImpl extends LensService implements QuerySched
    * @param cliService the cli service
    */
   public QuerySchedulerServiceImpl(CLIService cliService) {
-    super("scheduler", cliService);
+    super(NAME, cliService);
+  }
+
+  /**
+   * @inheritDoc
+   */
+  @Override
+  public HealthStatus getHealthStatus() {
+    return this.getServiceState().equals(STATE.STARTED)
+        ? new HealthStatus(true, "Query scheduler service is healthy.")
+        : new HealthStatus(false, "Query scheduler service is down.");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java b/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java
index 3a8fc95..ab5be32 100644
--- a/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java
+++ b/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java
@@ -32,10 +32,11 @@ import javax.ws.rs.NotFoundException;
 import javax.ws.rs.WebApplicationException;
 
 import org.apache.lens.api.LensSessionHandle;
-import org.apache.lens.server.LensService;
+import org.apache.lens.server.BaseLensService;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.health.HealthStatus;
 import org.apache.lens.server.api.session.SessionClosed;
 import org.apache.lens.server.api.session.SessionExpired;
 import org.apache.lens.server.api.session.SessionOpened;
@@ -66,7 +67,8 @@ import lombok.extern.slf4j.Slf4j;
  * The Class HiveSessionService.
  */
 @Slf4j
-public class HiveSessionService extends LensService implements SessionService {
+public class HiveSessionService extends BaseLensService implements SessionService {
+
 
   /** The restorable sessions. */
   private List<LensSessionImpl.LensSessionPersistInfo> restorableSessions;
@@ -102,7 +104,7 @@ public class HiveSessionService extends LensService implements SessionService {
   public int addResourceToAllServices(LensSessionHandle sessionid, String type, String path) {
     int numAdded = 0;
     boolean error = false;
-    for (LensService service : LensServices.get().getLensServices()) {
+    for (BaseLensService service : LensServices.get().getLensServices()) {
       try {
         service.addResource(sessionid, type, path);
         numAdded++;
@@ -448,6 +450,16 @@ public class HiveSessionService extends LensService implements SessionService {
     log.info("Session service pesristed " + SESSION_MAP.size() + " sessions");
   }
 
+  /**
+   * @inheritDoc
+   */
+  @Override
+  public HealthStatus getHealthStatus() {
+    return this.getServiceState().equals(STATE.STARTED)
+        ? new HealthStatus(true, "Hive session service is healthy.")
+        : new HealthStatus(false, "Hive session service is down.");
+  }
+
   /*
    * (non-Javadoc)
    *
@@ -484,7 +496,7 @@ public class HiveSessionService extends LensService implements SessionService {
   private void closeInternal(LensSessionHandle sessionHandle) throws LensException {
     super.closeSession(sessionHandle);
     // Inform query service
-    LensService svc = LensServices.get().getService(QueryExecutionServiceImpl.NAME);
+    BaseLensService svc = LensServices.get().getService(QueryExecutionServiceImpl.NAME);
     if (svc instanceof QueryExecutionServiceImpl) {
       ((QueryExecutionServiceImpl) svc).closeDriverSessions(sessionHandle);
     }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java b/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
index 0635e95..63d7b9c 100644
--- a/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
+++ b/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
@@ -30,7 +30,7 @@ import org.apache.lens.api.APIResult.Status;
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.StringList;
-import org.apache.lens.server.LensService;
+import org.apache.lens.server.BaseLensService;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.session.SessionService;
@@ -201,7 +201,7 @@ public class SessionResource {
     int numDeleted = 0;
 
     for(String matchedPath : scannedPaths) {
-      for (LensService service : LensServices.get().getLensServices()) {
+      for (BaseLensService service : LensServices.get().getLensServices()) {
         try {
           if (matchedPath.startsWith("file:") && !matchedPath.startsWith("file://")) {
             matchedPath = "file://" + matchedPath.substring("file:".length());

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java b/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
index 697e867..b5235ad 100644
--- a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
+++ b/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
@@ -121,6 +121,8 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
     log.info("Server restart test");
 
     QueryExecutionServiceImpl queryService = (QueryExecutionServiceImpl) LensServices.get().getService("query");
+    Assert.assertTrue(queryService.getHealthStatus().isHealthy());
+
     LensSessionHandle lensSessionId = queryService.openSession("foo", "bar", new HashMap<String, String>());
     // Create data file
     createRestartTestDataFile();
@@ -143,6 +145,7 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
         try {
           queryService.pauseQuerySubmitter();
           log.info("Stopped query submitter");
+          Assert.assertFalse(queryService.getHealthStatus().isHealthy());
         } catch (Exception exc) {
           log.error("Could not kill query submitter", exc);
         }
@@ -173,6 +176,7 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
     restartLensServer();
     log.info("Restarted lens server!");
     queryService = (QueryExecutionServiceImpl) LensServices.get().getService("query");
+    Assert.assertTrue(queryService.getHealthStatus().isHealthy());
 
     // All queries should complete after server restart
     for (QueryHandle handle : launchedQueries) {
@@ -213,6 +217,8 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
   @Test
   public void testHiveServerRestart() throws Exception {
     QueryExecutionServiceImpl queryService = (QueryExecutionServiceImpl) LensServices.get().getService("query");
+    Assert.assertTrue(queryService.getHealthStatus().isHealthy());
+
     LensSessionHandle lensSessionId = queryService.openSession("foo", "bar", new HashMap<String, String>());
 
     // set params
@@ -223,6 +229,8 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
 
     // Add a resource to check if its added after server restart.
     HiveSessionService sessionService = (HiveSessionService) LensServices.get().getService(SessionService.NAME);
+    Assert.assertTrue(sessionService.getHealthStatus().isHealthy());
+
     sessionService.addResource(lensSessionId, "FILE", dataFile.toURI().toString());
     queryService.getSession(lensSessionId).addResource("FILE", dataFile.toURI().toString());
     log.info("@@ Added resource {}", dataFile.toURI());
@@ -379,6 +387,8 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
     verifyParamOnRestart(restartTestSession);
 
     HiveSessionService sessionService = LensServices.get().getService("session");
+    Assert.assertTrue(sessionService.getHealthStatus().isHealthy());
+
     LensSessionImpl session = sessionService.getSession(restartTestSession);
     Assert.assertEquals(session.getLensSessionPersistInfo().getResources().size(), 1);
     LensSessionImpl.ResourceEntry resourceEntry = session.getLensSessionPersistInfo().getResources().get(0);

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/lens-server/src/test/java/org/apache/lens/server/healthcheck/TestHealthChecks.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/healthcheck/TestHealthChecks.java b/lens-server/src/test/java/org/apache/lens/server/healthcheck/TestHealthChecks.java
new file mode 100644
index 0000000..3573a1e
--- /dev/null
+++ b/lens-server/src/test/java/org/apache/lens/server/healthcheck/TestHealthChecks.java
@@ -0,0 +1,127 @@
+/**
+ * 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.lens.server.healthcheck;
+
+import static org.testng.Assert.*;
+
+import javax.ws.rs.core.Application;
+
+import org.apache.lens.server.EventServiceImpl;
+import org.apache.lens.server.LensJerseyTest;
+import org.apache.lens.server.LensServices;
+import org.apache.lens.server.api.LensService;
+import org.apache.lens.server.api.health.HealthStatus;
+import org.apache.lens.server.api.query.QueryExecutionService;
+import org.apache.lens.server.metastore.CubeMetastoreServiceImpl;
+import org.apache.lens.server.metastore.MetastoreApp;
+import org.apache.lens.server.metrics.MetricsServiceImpl;
+import org.apache.lens.server.quota.QuotaServiceImpl;
+import org.apache.lens.server.scheduler.QuerySchedulerServiceImpl;
+import org.apache.lens.server.session.HiveSessionService;
+
+import org.apache.log4j.BasicConfigurator;
+
+import org.glassfish.jersey.client.ClientConfig;
+import org.glassfish.jersey.media.multipart.MultiPartFeature;
+
+import org.testng.annotations.AfterTest;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+import com.codahale.metrics.health.HealthCheck;
+
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@Test(groups = "unit-test")
+public class TestHealthChecks extends LensJerseyTest {
+  @BeforeTest
+  public void setUp() throws Exception {
+    super.setUp();
+    BasicConfigurator.configure();
+  }
+
+  @AfterTest
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  @Override
+  protected Application configure() {
+    return new MetastoreApp();
+  }
+
+  @Override
+  protected void configureClient(ClientConfig config) {
+    config.register(MultiPartFeature.class);
+  }
+
+  @Test
+  public void testCubeMetastoreServiceHealth() throws Exception {
+    checkHealth(CubeMetastoreServiceImpl.NAME);
+  }
+
+  @Test
+  public void testEventServiceHealth() throws Exception {
+    checkHealth(EventServiceImpl.NAME);
+  }
+
+  @Test
+  public void testHiveSessionServiceHealth() throws Exception {
+    checkHealth(HiveSessionService.NAME);
+  }
+
+  @Test
+  public void testMetricsServiceHealth() throws Exception {
+    checkHealth(MetricsServiceImpl.NAME);
+  }
+
+  @Test
+  public void testQueryExecutionServiceHealth() throws Exception {
+    checkHealth(QueryExecutionService.NAME);
+  }
+
+  @Test
+  public void testQuerySchedulerServiceHealth() throws Exception {
+    checkHealth(QuerySchedulerServiceImpl.NAME);
+  }
+
+  @Test
+  public void testQuotaServiceHealth() throws Exception {
+    checkHealth(QuotaServiceImpl.NAME);
+  }
+
+  /**
+   * Utility method to check health for provided service name.
+   *
+   * @param serviceName
+   * @throws Exception
+   */
+  private void checkHealth(String serviceName) throws Exception {
+    /** Test via LensServiceHealthCheck **/
+    new LensServiceHealthCheck(serviceName).check().equals(HealthCheck.Result.healthy());
+
+    /** Also check directly via service **/
+    HealthStatus status;
+    LensService service = LensServices.get().getService(serviceName);
+    status = service.getHealthStatus();
+    assertTrue(status.isHealthy(), serviceName + " Service should had been healthy.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/651a9b6a/src/site/apt/user/cli.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/user/cli.apt b/src/site/apt/user/cli.apt
index 63f2b3f..51257e0 100644
--- a/src/site/apt/user/cli.apt
+++ b/src/site/apt/user/cli.apt
@@ -237,9 +237,9 @@ User CLI Commands
 *--+--+
 |fact timelines [--fact_name] \<fact_name\> [--storage_name \<storage_name\>] [--update_period \<update_period\>] [--time_dimension \<time_dimension\>]|get timelines for fact. Can optionally specify storage, update period and time dimension to filter by. Instead of time dimension, partition column can be directly passed as <<<time_dimension>>>|
 *--+--+
-|fact update partitions [--fact_name] \<fact_name\> [--storage_name] \<storage_name\> [--path] \<partition-list-spec-path\>|update multiple partition to fact <<<fact_name>>>'s storage <<<storage_name>>>, reading partition list spec from <<<partition-list-spec-path>>>|
+|fact update partitions [--fact_name] \<fact_name\> [--storage_name] \<storage_name\> [--path] \<partition-list-spec-path\>|update multiple partition of fact <<<fact_name>>>'s storage <<<storage_name>>>, reading partition list spec from <<<partition-list-spec-path>>> The partitions have to exist to be eligible for updation.|
 *--+--+
-|fact update single-partition [--fact_name] \<fact_name\> [--storage_name] \<storage_name\> [--path] \<partition-spec-path\>|update single partition to fact <<<fact_name>>>'s storage <<<storage_name>>>, reading spec from <<<partition-spec-path>>>|
+|fact update single-partition [--fact_name] \<fact_name\> [--storage_name] \<storage_name\> [--path] \<partition-spec-path\>|update single partition to fact <<<fact_name>>>'s storage <<<storage_name>>>, reading spec from <<<partition-spec-path>>> The partition has to exist to be eligible for updation.|
 *--+--+
 |show facts [[--cube_name] \<cube_name\>]|display list of fact tables in current database. If optional <<<cube_name>>> is supplied, only facts belonging to cube <<<cube_name>>> will be displayed|
 *--+--+
@@ -278,9 +278,9 @@ User CLI Commands
 *--+--+
 |dimtable list storages [--dimtable_name] \<dimtable_name\>|display list of storage associated to dimtable <<<dimtable_name>>>|
 *--+--+
-|dimtable update partitions [--dimtable_name] \<dimtable_name\> [--storage_name] \<storage_name\> [--path] \<partition-list-spec-path\>|update multiple partition to dimtable <<<dimtable_name>>>'s storage <<<storage_name>>>, reading partition list spec from <<<partition-list-spec-path>>>|
+|dimtable update partitions [--dimtable_name] \<dimtable_name\> [--storage_name] \<storage_name\> [--path] \<partition-list-spec-path\>|update multiple partition to dimtable <<<dimtable_name>>>'s storage <<<storage_name>>>, reading partition list spec from <<<partition-list-spec-path>>> The partitions have to exist to be eligible for updation.|
 *--+--+
-|dimtable update single-partition [--dimtable_name] \<dimtable_name\> [--storage_name] \<storage_name\> [--path] \<partition-spec-path\>|update single partition to dimtable <<<dimtable_name>>>'s storage <<<storage_name>>>, reading spec from <<<partition-spec-path>>>|
+|dimtable update single-partition [--dimtable_name] \<dimtable_name\> [--storage_name] \<storage_name\> [--path] \<partition-spec-path\>|update single partition to dimtable <<<dimtable_name>>>'s storage <<<storage_name>>>, reading spec from <<<partition-spec-path>>> The partition has to exist to be eligible for updation.|
 *--+--+
 |drop dimtable [--dimtable_name] \<dimtable_name\> [--cascade \<cascade\>]|drop dimtable <<<dimtable_name>>>.  If <<<cascade>>> is true, all the storage tables associated with the dimtable <<<dimtable_name>>> are also dropped. By default <<<cascade>>> is false|
 *--+--+