You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2018/01/02 14:50:14 UTC

[2/7] drill git commit: DRILL-5425: Support HTTP Kerberos auth using SPNEGO

DRILL-5425: Support HTTP Kerberos auth using SPNEGO

closes #1040


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

Branch: refs/heads/master
Commit: adee46149734908ad20568951c683e49e88a67a3
Parents: e25c58f
Author: Sindhuri Rayavaram <sr...@mapr.com>
Authored: Mon Sep 11 16:56:22 2017 -0700
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Tue Jan 2 15:43:10 2018 +0200

----------------------------------------------------------------------
 .../org/apache/drill/exec/ExecConstants.java    |   4 +-
 .../exec/server/rest/LogInLogOutResources.java  | 124 +++++--
 .../server/rest/ViewableWithPermissions.java    |  21 +-
 .../drill/exec/server/rest/WebServer.java       |  52 ++-
 .../exec/server/rest/WebServerConstants.java    |  45 +++
 .../server/rest/auth/AuthDynamicFeature.java    |   9 +-
 .../server/rest/auth/DrillErrorHandler.java     |  45 +++
 .../DrillHttpConstraintSecurityHandler.java     |  61 ++++
 .../auth/DrillHttpSecurityHandlerProvider.java  | 184 +++++++++++
 .../rest/auth/DrillSpnegoAuthenticator.java     | 192 +++++++++++
 .../rest/auth/DrillSpnegoLoginService.java      | 149 +++++++++
 .../server/rest/auth/DrillUserPrincipal.java    |   6 +-
 .../server/rest/auth/FormSecurityHanlder.java   |  49 +++
 .../exec/server/rest/auth/SpnegoConfig.java     | 112 +++++++
 .../server/rest/auth/SpnegoSecurityHandler.java |  36 ++
 .../src/main/resources/drill-module.conf        |   4 +-
 .../src/main/resources/rest/generic.ftl         |   2 +-
 .../src/main/resources/rest/mainLogin.ftl       |  32 ++
 .../drill/exec/rpc/data/TestBitBitKerberos.java |   3 +-
 .../drill/exec/rpc/security/KerberosHelper.java |   8 +-
 .../rpc/user/security/TestUserBitKerberos.java  |   2 +-
 .../security/TestUserBitKerberosEncryption.java |   2 +-
 .../spnego/TestDrillSpnegoAuthenticator.java    | 286 ++++++++++++++++
 .../rest/spnego/TestSpnegoAuthentication.java   | 326 +++++++++++++++++++
 .../server/rest/spnego/TestSpnegoConfig.java    | 167 ++++++++++
 .../org/apache/drill/exec/rpc/BasicServer.java  |   1 +
 26 files changed, 1850 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 52aa52d..5059b4f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -30,7 +30,6 @@ import org.apache.drill.exec.server.options.TypeValidators.PowerOfTwoLongValidat
 import org.apache.drill.exec.server.options.TypeValidators.RangeDoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.RangeLongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
-import org.apache.drill.exec.server.options.TypeValidators.MaxWidthValidator;
 import org.apache.drill.exec.server.options.TypeValidators.AdminUsersValidator;
 import org.apache.drill.exec.server.options.TypeValidators.AdminUserGroupsValidator;
 import org.apache.drill.exec.testing.ExecutionControls;
@@ -149,6 +148,9 @@ public final class ExecConstants {
   public static final String HTTP_KEYSTORE_PASSWORD = SSL_KEYSTORE_PASSWORD;
   public static final String HTTP_TRUSTSTORE_PATH = SSL_TRUSTSTORE_PATH;
   public static final String HTTP_TRUSTSTORE_PASSWORD = SSL_TRUSTSTORE_PASSWORD;
+  public static final String HTTP_AUTHENTICATION_MECHANISMS = "drill.exec.http.auth.mechanisms";
+  public static final String HTTP_SPNEGO_PRINCIPAL = "drill.exec.http.auth.spnego.principal";
+  public static final String HTTP_SPNEGO_KEYTAB = "drill.exec.http.auth.spnego.keytab";
   public static final String SYS_STORE_PROVIDER_CLASS = "drill.exec.sys.store.provider.class";
   public static final String SYS_STORE_PROVIDER_LOCAL_PATH = "drill.exec.sys.store.provider.local.path";
   public static final String SYS_STORE_PROVIDER_LOCAL_ENABLE_WRITE = "drill.exec.sys.store.provider.local.write";

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java
index 20cd6da..34ac4d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,7 +17,18 @@
  */
 package org.apache.drill.exec.server.rest;
 
+import org.apache.commons.lang3.StringUtils;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.rpc.security.AuthStringUtil;
+import org.apache.drill.exec.server.rest.auth.AuthDynamicFeature;
+import org.apache.drill.exec.work.WorkManager;
+import org.eclipse.jetty.security.authentication.FormAuthenticator;
+import org.eclipse.jetty.util.security.Constraint;
+import org.glassfish.jersey.server.mvc.Viewable;
+
 import javax.annotation.security.PermitAll;
+import javax.inject.Inject;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpSession;
@@ -31,33 +42,26 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.SecurityContext;
 import javax.ws.rs.core.UriBuilder;
 import javax.ws.rs.core.UriInfo;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.drill.exec.server.rest.auth.AuthDynamicFeature;
-import org.eclipse.jetty.security.authentication.FormAuthenticator;
-import org.glassfish.jersey.server.mvc.Viewable;
-
 import java.net.URI;
 import java.net.URLDecoder;
+import java.util.Set;
 
-@Path("/")
+@Path(WebServerConstants.WEBSERVER_ROOT_PATH)
 @PermitAll
 public class LogInLogOutResources {
-  public static final String REDIRECT_QUERY_PARM = "redirect";
-  public static final String LOGIN_RESOURCE = "login";
 
-  @GET
-  @Path("/login")
-  @Produces(MediaType.TEXT_HTML)
-  public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpServletResponse response,
-      @Context SecurityContext sc, @Context UriInfo uriInfo, @QueryParam(REDIRECT_QUERY_PARM) String redirect)
-      throws Exception {
-    if (AuthDynamicFeature.isUserLoggedIn(sc)) {
-      // if the user is already login, forward the request to homepage.
-      request.getRequestDispatcher("/").forward(request, response);
-      return null;
-    }
+  @Inject
+  WorkManager workManager;
 
+
+  /**
+   * Update the destination URI to be redirect URI if specified in the request URL so that after the login is
+   * successful, request is forwarded to redirect page.
+   * @param redirect - Redirect parameter in the request URI
+   * @param request - Http Servlet Request
+   * @throws Exception
+   */
+  private void updateSessionRedirectInfo(String redirect, HttpServletRequest request) throws Exception {
     if (!StringUtils.isEmpty(redirect)) {
       // If the URL has redirect in it, set the redirect URI in session, so that after the login is successful, request
       // is forwarded to the redirect page.
@@ -65,27 +69,99 @@ public class LogInLogOutResources {
       final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
       session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
     }
+  }
+
+  @GET
+  @Path(WebServerConstants.FORM_LOGIN_RESOURCE_PATH)
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpServletResponse response,
+                               @Context SecurityContext sc, @Context UriInfo uriInfo,
+                               @QueryParam(WebServerConstants.REDIRECT_QUERY_PARM) String redirect) throws Exception {
+
+    if (AuthDynamicFeature.isUserLoggedIn(sc)) {
+      // if the user is already login, forward the request to homepage.
+      request.getRequestDispatcher(WebServerConstants.WEBSERVER_ROOT_PATH).forward(request, response);
+      return null;
+    }
 
+    updateSessionRedirectInfo(redirect, request);
     return ViewableWithPermissions.createLoginPage(null);
   }
 
+  @GET
+  @Path(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH)
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable getSpnegoLogin(@Context HttpServletRequest request, @Context HttpServletResponse response,
+                                 @Context SecurityContext sc, @Context UriInfo uriInfo,
+                                 @QueryParam(WebServerConstants.REDIRECT_QUERY_PARM) String redirect) throws Exception {
+    if (AuthDynamicFeature.isUserLoggedIn(sc)) {
+      request.getRequestDispatcher(WebServerConstants.WEBSERVER_ROOT_PATH).forward(request, response);
+      return null;
+    }
+
+    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
+    final DrillConfig drillConfig = workManager.getContext().getConfig();
+    MainLoginPageModel model = new MainLoginPageModel(errorString, drillConfig);
+    return ViewableWithPermissions.createMainLoginPage(model);
+  }
+
   // Request type is POST because POST request which contains the login credentials are invalid and the request is
   // dispatched here directly.
   @POST
-  @Path("/login")
+  @Path(WebServerConstants.FORM_LOGIN_RESOURCE_PATH)
   @Produces(MediaType.TEXT_HTML)
   public Viewable getLoginPageAfterValidationError() {
     return ViewableWithPermissions.createLoginPage("Invalid username/password credentials.");
   }
 
   @GET
-  @Path("/logout")
+  @Path(WebServerConstants.LOGOUT_RESOURCE_PATH)
   public void logout(@Context HttpServletRequest req, @Context HttpServletResponse resp) throws Exception {
     final HttpSession session = req.getSession();
     if (session != null) {
       session.invalidate();
     }
 
-    req.getRequestDispatcher("/").forward(req, resp);
+    req.getRequestDispatcher(WebServerConstants.WEBSERVER_ROOT_PATH).forward(req, resp);
+  }
+
+  @GET
+  @Path(WebServerConstants.MAIN_LOGIN_RESOURCE_PATH)
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable getMainLoginPage(@Context HttpServletRequest request, @Context HttpServletResponse response,
+                                   @Context SecurityContext sc, @Context UriInfo uriInfo,
+                                   @QueryParam(WebServerConstants.REDIRECT_QUERY_PARM) String redirect) throws Exception {
+    updateSessionRedirectInfo(redirect, request);
+    final DrillConfig drillConfig = workManager.getContext().getConfig();
+    MainLoginPageModel model = new MainLoginPageModel(null, drillConfig);
+    return ViewableWithPermissions.createMainLoginPage(model);
+  }
+
+  private class MainLoginPageModel {
+
+    private final String error;
+
+    private final boolean authEnabled;
+
+    private final Set<String> configuredMechs;
+
+    MainLoginPageModel(String error, DrillConfig drillConfig) {
+      this.error = error;
+      authEnabled = drillConfig.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED);
+      configuredMechs = AuthStringUtil.asSet(
+          drillConfig.getStringList(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS));
+    }
+
+    public boolean isSpnegoEnabled() {
+      return authEnabled && configuredMechs.contains(Constraint.__SPNEGO_AUTH);
+    }
+
+    public boolean isFormEnabled() {
+      return authEnabled && configuredMechs.contains(Constraint.__FORM_AUTH);
+    }
+
+    public String getError() {
+      return error;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ViewableWithPermissions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ViewableWithPermissions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ViewableWithPermissions.java
index 73019aa..2eed0b8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ViewableWithPermissions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ViewableWithPermissions.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -47,8 +47,8 @@ public class ViewableWithPermissions extends Viewable {
    * @param model
    * @return
    */
-  public static Viewable create(final boolean authEnabled, final String templateName, final SecurityContext sc,
-      final Object model) {
+  public static Viewable create(final boolean authEnabled, final String templateName,
+                                final SecurityContext sc, final Object model) {
     return new ViewableWithPermissions(authEnabled, templateName, sc, true, model);
   }
 
@@ -61,13 +61,18 @@ public class ViewableWithPermissions extends Viewable {
     return new ViewableWithPermissions(true, "/rest/login.ftl", null, false, errorMsg);
   }
 
-  private ViewableWithPermissions(final boolean authEnabled, final String templateName, final SecurityContext sc,
-      final boolean showControls, final Object model) throws IllegalArgumentException {
+  public static Viewable createMainLoginPage(Object mainPageModel) {
+    return new ViewableWithPermissions(true, "/rest/mainLogin.ftl", null, false, mainPageModel);
+  }
+
+  private ViewableWithPermissions(final boolean authEnabled, final String templateName,
+                                  final SecurityContext sc, final boolean showControls,
+                                  final Object model) throws IllegalArgumentException {
     super(templateName, createModel(authEnabled, sc, showControls, model));
   }
 
   private static Map<String, Object> createModel(final boolean authEnabled, final SecurityContext sc,
-      final boolean showControls, final Object pageModel) {
+                                                 final boolean showControls, final Object pageModel) {
 
     final boolean isAdmin = !authEnabled /* when auth is disabled every user is an admin user */
         || (showControls && sc.isUserInRole(DrillUserPrincipal.ADMIN_ROLE));
@@ -82,8 +87,8 @@ public class ViewableWithPermissions extends Viewable {
         .put("showLogin", authEnabled && showControls && !isUserLoggedIn)
         .put("showLogout", authEnabled && showControls && isUserLoggedIn)
         .put("loggedInUserName", authEnabled && showControls &&
-            isUserLoggedIn ? sc.getUserPrincipal().getName() : DrillUserPrincipal.ANONYMOUS_USER)
-        .put("showControls", showControls);
+            isUserLoggedIn ? sc.getUserPrincipal().getName()
+                           : DrillUserPrincipal.ANONYMOUS_USER).put("showControls", showControls);
 
     if (pageModel != null) {
       mapBuilder.put("model", pageModel);

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
index f0e822f..c702314 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
@@ -28,10 +28,11 @@ import org.apache.drill.common.exceptions.DrillException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ssl.SSLConfig;
 import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.rpc.security.plain.PlainFactory;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.rest.auth.DrillErrorHandler;
 import org.apache.drill.exec.server.rest.auth.DrillRestLoginService;
+import org.apache.drill.exec.server.rest.auth.DrillHttpSecurityHandlerProvider;
 import org.apache.drill.exec.ssl.SSLConfigBuilder;
 import org.apache.drill.exec.work.WorkManager;
 import org.bouncycastle.asn1.x500.X500NameBuilder;
@@ -114,7 +115,7 @@ public class WebServer implements AutoCloseable {
   /**
    * Create Jetty based web server.
    *
-   * @param context Bootstrap context.
+   * @param context     Bootstrap context.
    * @param workManager WorkManager instance.
    */
   public WebServer(final BootStrapContext context, final WorkManager workManager, final Drillbit drillbit) {
@@ -136,7 +137,8 @@ public class WebServer implements AutoCloseable {
    * @return true if impersonation without authentication is enabled, false otherwise
    */
   public static boolean isImpersonationOnlyEnabled(DrillConfig config) {
-    return !config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED) && config.getBoolean(ExecConstants.IMPERSONATION_ENABLED);
+    return !config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED)
+        && config.getBoolean(ExecConstants.IMPERSONATION_ENABLED);
   }
 
   /**
@@ -150,11 +152,6 @@ public class WebServer implements AutoCloseable {
     }
 
     final boolean authEnabled = config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED);
-    if (authEnabled && !context.getAuthProvider().containsFactory(PlainFactory.SIMPLE_NAME)) {
-      logger.warn("Not starting web server. Currently Drill supports web authentication only through " +
-          "username/password. But PLAIN mechanism is not configured.");
-      return;
-    }
 
     port = config.getInt(ExecConstants.HTTP_PORT);
     boolean portHunt = config.getBoolean(ExecConstants.HTTP_PORT_HUNT);
@@ -187,9 +184,10 @@ public class WebServer implements AutoCloseable {
     }
   }
 
-  private ServletContextHandler createServletContextHandler(final boolean authEnabled) {
+  private ServletContextHandler createServletContextHandler(final boolean authEnabled) throws DrillbitStartupException {
     // Add resources
-    final ErrorHandler errorHandler = new ErrorHandler();
+    final ErrorHandler errorHandler = new DrillErrorHandler();
+
     errorHandler.setShowStacks(true);
     errorHandler.setShowMessageInTitle(true);
 
@@ -197,7 +195,8 @@ public class WebServer implements AutoCloseable {
     servletContextHandler.setErrorHandler(errorHandler);
     servletContextHandler.setContextPath("/");
 
-    final ServletHolder servletHolder = new ServletHolder(new ServletContainer(new DrillRestServer(workManager, servletContextHandler.getServletContext(), drillbit)));
+    final ServletHolder servletHolder = new ServletHolder(new ServletContainer(
+        new DrillRestServer(workManager, servletContextHandler.getServletContext(), drillbit)));
     servletHolder.setInitOrder(1);
     servletContextHandler.addServlet(servletHolder, "/*");
 
@@ -207,16 +206,16 @@ public class WebServer implements AutoCloseable {
     final ServletHolder staticHolder = new ServletHolder("static", DefaultServlet.class);
     // Get resource URL for Drill static assets, based on where Drill icon is located
     String drillIconResourcePath =
-      Resource.newClassPathResource(BASE_STATIC_PATH + DRILL_ICON_RESOURCE_RELATIVE_PATH).getURL().toString();
-    staticHolder.setInitParameter(
-      "resourceBase",
-      drillIconResourcePath.substring(0,  drillIconResourcePath.length() - DRILL_ICON_RESOURCE_RELATIVE_PATH.length()));
+        Resource.newClassPathResource(BASE_STATIC_PATH + DRILL_ICON_RESOURCE_RELATIVE_PATH).getURL().toString();
+    staticHolder.setInitParameter("resourceBase",
+        drillIconResourcePath.substring(0, drillIconResourcePath.length() - DRILL_ICON_RESOURCE_RELATIVE_PATH.length()));
     staticHolder.setInitParameter("dirAllowed", "false");
     staticHolder.setInitParameter("pathInfoOnly", "true");
     servletContextHandler.addServlet(staticHolder, "/static/*");
 
     if (authEnabled) {
-      servletContextHandler.setSecurityHandler(createSecurityHandler());
+      //DrillSecurityHandler is used to support SPNEGO and FORM authentication together
+      servletContextHandler.setSecurityHandler(new DrillHttpSecurityHandlerProvider(config, workManager.getContext()));
       servletContextHandler.setSessionHandler(createSessionHandler(servletContextHandler.getSecurityHandler()));
     }
 
@@ -229,13 +228,13 @@ public class WebServer implements AutoCloseable {
     if (config.getBoolean(ExecConstants.HTTP_CORS_ENABLED)) {
       FilterHolder holder = new FilterHolder(CrossOriginFilter.class);
       holder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM,
-        StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_ORIGINS), ","));
+          StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_ORIGINS), ","));
       holder.setInitParameter(CrossOriginFilter.ALLOWED_METHODS_PARAM,
-        StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_METHODS), ","));
+          StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_METHODS), ","));
       holder.setInitParameter(CrossOriginFilter.ALLOWED_HEADERS_PARAM,
-        StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_HEADERS), ","));
+          StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_HEADERS), ","));
       holder.setInitParameter(CrossOriginFilter.ALLOW_CREDENTIALS_PARAM,
-        String.valueOf(config.getBoolean(ExecConstants.HTTP_CORS_CREDENTIALS)));
+          String.valueOf(config.getBoolean(ExecConstants.HTTP_CORS_CREDENTIALS)));
 
       for (String path : new String[]{"*.json", "/storage/*/enable/*", "/status*"}) {
         servletContextHandler.addFilter(holder, path, EnumSet.of(DispatcherType.REQUEST));
@@ -314,8 +313,7 @@ public class WebServer implements AutoCloseable {
     if (config.getBoolean(ExecConstants.HTTP_ENABLE_SSL)) {
       try {
         serverConnector = createHttpsConnector(port);
-      }
-      catch(DrillException e){
+      } catch (DrillException e) {
         throw new DrillbitStartupException(e.getMessage(), e);
       }
     } else {
@@ -366,11 +364,10 @@ public class WebServer implements AutoCloseable {
       final DateTime now = DateTime.now();
 
       // Create builder for certificate attributes
-      final X500NameBuilder nameBuilder =
-          new X500NameBuilder(BCStyle.INSTANCE)
-              .addRDN(BCStyle.OU, "Apache Drill (auth-generated)")
-              .addRDN(BCStyle.O, "Apache Software Foundation (auto-generated)")
-              .addRDN(BCStyle.CN, workManager.getContext().getEndpoint().getAddress());
+      final X500NameBuilder nameBuilder = new X500NameBuilder(BCStyle.INSTANCE)
+          .addRDN(BCStyle.OU, "Apache Drill (auth-generated)")
+          .addRDN(BCStyle.O, "Apache Software Foundation (auto-generated)")
+          .addRDN(BCStyle.CN, workManager.getContext().getEndpoint().getAddress());
 
       final Date notBefore = now.minusMinutes(1).toDate();
       final Date notAfter = now.plusYears(5).toDate();
@@ -422,6 +419,7 @@ public class WebServer implements AutoCloseable {
 
   /**
    * Create HTTP connector.
+   *
    * @return Initialized {@link ServerConnector} instance for HTTP connections.
    * @throws Exception
    */

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServerConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServerConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServerConstants.java
new file mode 100644
index 0000000..5650d43
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServerConstants.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.drill.exec.server.rest;
+
+/**
+ * Holds various constants used by WebServer components.
+ */
+public final class WebServerConstants {
+
+  private WebServerConstants() {}
+
+  public static final String REDIRECT_QUERY_PARM = "redirect";
+  public static final String WEBSERVER_ROOT_PATH = "/";
+
+  // Main Login page which help to choose between Form and Spnego authentication
+  public static final String MAIN_LOGIN_RESOURCE_NAME = "mainLogin";
+  public static final String MAIN_LOGIN_RESOURCE_PATH = WEBSERVER_ROOT_PATH + MAIN_LOGIN_RESOURCE_NAME;
+
+  // Login page for FORM authentication
+  public static final String FORM_LOGIN_RESOURCE_NAME = "login";
+  public static final String FORM_LOGIN_RESOURCE_PATH = WEBSERVER_ROOT_PATH + FORM_LOGIN_RESOURCE_NAME;
+
+  // Login page for SPNEGO authentication
+  public static final String SPENGO_LOGIN_RESOURCE_NAME = "spnegoLogin";
+  public static final String SPENGO_LOGIN_RESOURCE_PATH = WEBSERVER_ROOT_PATH + SPENGO_LOGIN_RESOURCE_NAME;
+
+  // Logout page
+  public static final String LOGOUT_RESOURCE_NAME = "logout";
+  public static final String LOGOUT_RESOURCE_PATH = WEBSERVER_ROOT_PATH + LOGOUT_RESOURCE_NAME;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/AuthDynamicFeature.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/AuthDynamicFeature.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/AuthDynamicFeature.java
index bee0c9d..7ca739b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/AuthDynamicFeature.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/AuthDynamicFeature.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.server.rest.auth;
 
-import org.apache.drill.exec.server.rest.LogInLogOutResources;
+import org.apache.drill.exec.server.rest.WebServerConstants;
 import org.glassfish.jersey.server.model.AnnotatedMethod;
 
 import javax.annotation.Priority;
@@ -55,6 +55,9 @@ public class AuthDynamicFeature implements DynamicFeature {
     }
 
     // PermitAll takes precedence over RolesAllowed on the class
+    // This avoids putting AuthCheckFilter in the request flow for all path's which
+    // are defined under PermitAll annotation. That is requests for "/", "/login", "/mainLogin" and "/spnegoLogin"
+    // path's doesn't go through AuthCheckFilter.
     if (am.isAnnotationPresent(PermitAll.class)) {
       // Do nothing.
       return;
@@ -79,8 +82,8 @@ public class AuthDynamicFeature implements DynamicFeature {
           final String destResource =
               URLEncoder.encode(requestContext.getUriInfo().getRequestUri().toString(), "UTF-8");
           final URI loginURI = requestContext.getUriInfo().getBaseUriBuilder()
-              .path(LogInLogOutResources.LOGIN_RESOURCE)
-              .queryParam(LogInLogOutResources.REDIRECT_QUERY_PARM, destResource)
+              .path(WebServerConstants.MAIN_LOGIN_RESOURCE_NAME)
+              .queryParam(WebServerConstants.REDIRECT_QUERY_PARM, destResource)
               .build();
           requestContext.abortWith(Response.temporaryRedirect(loginURI).build()
           );

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillErrorHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillErrorHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillErrorHandler.java
new file mode 100644
index 0000000..df4825f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillErrorHandler.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.drill.exec.server.rest.auth;
+
+import org.apache.drill.exec.server.rest.WebServerConstants;
+import org.eclipse.jetty.server.handler.ErrorHandler;
+
+import javax.servlet.http.HttpServletRequest;
+import java.io.IOException;
+import java.io.Writer;
+
+/**
+ * Custom ErrorHandler class for Drill's WebServer to have better error message in case when SPNEGO login failed and
+ * what to do next. In all other cases this would use the generic error page.
+ */
+public class DrillErrorHandler extends ErrorHandler {
+
+  @Override
+  protected void writeErrorPageMessage(HttpServletRequest request, Writer writer,
+                                       int code, String message, String uri) throws IOException {
+
+    super.writeErrorPageMessage(request, writer, code, message, uri);
+
+    if (uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH)) {
+      writer.write("<p>SPNEGO Login Failed</p>");
+      writer.write("<p>Please check the requirements or use below link to use Form Authentication instead</p>");
+      writer.write("<a href='/login'> login </a>");
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpConstraintSecurityHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpConstraintSecurityHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpConstraintSecurityHandler.java
new file mode 100644
index 0000000..0b095fb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpConstraintSecurityHandler.java
@@ -0,0 +1,61 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.server.rest.auth;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.drill.common.exceptions.DrillException;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.eclipse.jetty.security.ConstraintMapping;
+import org.eclipse.jetty.security.ConstraintSecurityHandler;
+import org.eclipse.jetty.security.LoginService;
+import org.eclipse.jetty.security.authentication.LoginAuthenticator;
+
+import java.util.Collections;
+import java.util.Set;
+
+import static org.apache.drill.exec.server.rest.auth.DrillUserPrincipal.ADMIN_ROLE;
+import static org.apache.drill.exec.server.rest.auth.DrillUserPrincipal.AUTHENTICATED_ROLE;
+
+/**
+ * Accessor class that extends the ConstraintSecurityHandler to expose protected method's for start and stop of Handler.
+ * This is needed since now {@link DrillHttpSecurityHandlerProvider} composes of 2 security handlers -
+ * For FORM and SPNEGO and has responsibility to start/stop of those handlers.
+ **/
+public abstract class DrillHttpConstraintSecurityHandler extends ConstraintSecurityHandler {
+
+    @Override
+    public void doStart() throws Exception {
+        super.doStart();
+    }
+
+    @Override
+    public void doStop() throws Exception {
+        super.doStop();
+    }
+
+    public abstract void doSetup(DrillbitContext dbContext) throws DrillException;
+
+    public void setup(LoginAuthenticator authenticator, LoginService loginService) {
+      final Set<String> knownRoles = ImmutableSet.of(AUTHENTICATED_ROLE, ADMIN_ROLE);
+      setConstraintMappings(Collections.<ConstraintMapping>emptyList(), knownRoles);
+      setAuthenticator(authenticator);
+      setLoginService(loginService);
+    }
+
+    public abstract String getImplName();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java
new file mode 100644
index 0000000..3d77596
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java
@@ -0,0 +1,184 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.server.rest.auth;
+
+import com.google.common.base.Preconditions;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillException;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.AuthStringUtil;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.rest.WebServerConstants;
+import org.eclipse.jetty.security.ConstraintSecurityHandler;
+import org.eclipse.jetty.security.authentication.SessionAuthentication;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.util.security.Constraint;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpSession;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+
+public class DrillHttpSecurityHandlerProvider extends ConstraintSecurityHandler {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillHttpSecurityHandlerProvider.class);
+
+  private final Map<String, DrillHttpConstraintSecurityHandler> securityHandlers =
+      CaseInsensitiveMap.newHashMapWithExpectedSize(2);
+
+  public DrillHttpSecurityHandlerProvider(DrillConfig config, DrillbitContext drillContext)
+      throws DrillbitStartupException {
+
+    Preconditions.checkState(config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED));
+    final Set<String> configuredMechanisms = new HashSet<>();
+
+    if (config.hasPath(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS)) {
+      configuredMechanisms.addAll(AuthStringUtil.asSet(config.getStringList(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS)));
+    } else { // for backward compatibility
+      configuredMechanisms.add(Constraint.__FORM_AUTH);
+    }
+
+      final ScanResult scan = drillContext.getClasspathScan();
+      final Collection<Class<? extends DrillHttpConstraintSecurityHandler>> factoryImpls =
+          scan.getImplementations(DrillHttpConstraintSecurityHandler.class);
+      logger.debug("Found DrillHttpConstraintSecurityHandler implementations: {}", factoryImpls);
+      for (final Class<? extends DrillHttpConstraintSecurityHandler> clazz : factoryImpls) {
+
+        // If all the configured mechanisms handler is added then break out of this loop
+        if (configuredMechanisms.isEmpty()) {
+          break;
+        }
+
+        Constructor<? extends DrillHttpConstraintSecurityHandler> validConstructor = null;
+        for (final Constructor<?> c : clazz.getConstructors()) {
+          final Class<?>[] params = c.getParameterTypes();
+          if (params.length == 0) {
+            validConstructor = (Constructor<? extends DrillHttpConstraintSecurityHandler>) c; // unchecked
+            break;
+          }
+        }
+
+        if (validConstructor == null) {
+          logger.warn("Skipping DrillHttpConstraintSecurityHandler class {}. It must implement at least one" +
+              " constructor with signature [{}()]", clazz.getCanonicalName(), clazz.getName());
+          continue;
+        }
+
+        try {
+          final DrillHttpConstraintSecurityHandler instance = validConstructor.newInstance();
+          if (configuredMechanisms.remove(instance.getImplName())) {
+            instance.doSetup(drillContext);
+            securityHandlers.put(instance.getImplName(), instance);
+          }
+        } catch (IllegalArgumentException | ReflectiveOperationException | DrillException e) {
+          logger.warn(String.format("Failed to create DrillHttpConstraintSecurityHandler of type '%s'",
+              clazz.getCanonicalName()), e);
+        }
+      }
+
+    if (securityHandlers.size() == 0) {
+      throw new DrillbitStartupException("Authentication is enabled for WebServer but none of the security mechanism " +
+          "was configured properly. Please verify the configurations and try again.");
+    }
+
+    logger.info("Configure auth mechanisms for WebServer are: {}", securityHandlers.keySet());
+  }
+
+  @Override
+  public void doStart() throws Exception {
+    super.doStart();
+    for (DrillHttpConstraintSecurityHandler securityHandler : securityHandlers.values()) {
+      securityHandler.doStart();
+    }
+  }
+
+  @Override
+  public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response)
+      throws IOException, ServletException {
+
+    Preconditions.checkState(securityHandlers.size() > 0);
+
+    HttpSession session = request.getSession(true);
+    SessionAuthentication authentication =
+        (SessionAuthentication) session.getAttribute(SessionAuthentication.__J_AUTHENTICATED);
+    String uri = request.getRequestURI();
+    final DrillHttpConstraintSecurityHandler securityHandler;
+
+    // Before authentication, all requests go through the FormAuthenticator if configured except for /spnegoLogin
+    // request. For SPNEGO authentication all requests will be forced going via /spnegoLogin before authentication is
+    // done, this is to ensure that we don't have to authenticate same client session multiple times for each resource.
+    //
+    // If this authentication is null, user hasn't logged in yet
+    if (authentication == null) {
+
+      // 1) If only SPNEGOSecurity handler then use SPNEGOSecurity
+      // 2) If both but uri equals spnegoLogin then use SPNEGOSecurity
+      // 3) If both but uri doesn't equals spnegoLogin then use FORMSecurity
+      // 4) If only FORMSecurity handler then use FORMSecurity
+      if (isSpnegoEnabled() && (!isFormEnabled() || uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH))) {
+        securityHandler = securityHandlers.get(Constraint.__SPNEGO_AUTH);
+        securityHandler.handle(target, baseRequest, request, response);
+      } else if (isFormEnabled()) {
+        securityHandler = securityHandlers.get(Constraint.__FORM_AUTH);
+        securityHandler.handle(target, baseRequest, request, response);
+      }
+    }
+    // If user has logged in, use the corresponding handler to handle the request
+    else {
+      final String authMethod = authentication.getAuthMethod();
+      securityHandler = securityHandlers.get(authMethod);
+      securityHandler.handle(target, baseRequest, request, response);
+    }
+  }
+
+  @Override
+  public void setHandler(Handler handler) {
+    super.setHandler(handler);
+    for (DrillHttpConstraintSecurityHandler securityHandler : securityHandlers.values()) {
+      securityHandler.setHandler(handler);
+    }
+  }
+
+  public void doStop() throws Exception {
+    super.doStop();
+    for (DrillHttpConstraintSecurityHandler securityHandler : securityHandlers.values()) {
+      securityHandler.doStop();
+    }
+  }
+
+  public boolean isSpnegoEnabled() {
+    return securityHandlers.containsKey(Constraint.__SPNEGO_AUTH);
+  }
+
+  public boolean isFormEnabled() {
+    return securityHandlers.containsKey(Constraint.__FORM_AUTH);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java
new file mode 100644
index 0000000..10f21ac
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java
@@ -0,0 +1,192 @@
+/*
+ * 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.drill.exec.server.rest.auth;
+
+
+import org.apache.drill.exec.server.rest.WebServerConstants;
+import org.apache.parquet.Strings;
+import org.eclipse.jetty.http.HttpHeader;
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.security.ServerAuthException;
+import org.eclipse.jetty.security.UserAuthentication;
+import org.eclipse.jetty.security.authentication.DeferredAuthentication;
+import org.eclipse.jetty.security.authentication.SessionAuthentication;
+import org.eclipse.jetty.security.authentication.SpnegoAuthenticator;
+import org.eclipse.jetty.server.Authentication;
+import org.eclipse.jetty.server.HttpChannel;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.Response;
+import org.eclipse.jetty.server.UserIdentity;
+
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpSession;
+import java.io.IOException;
+
+/**
+ * Custom SpnegoAuthenticator for Drill
+ */
+public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
+
+  public DrillSpnegoAuthenticator(String authMethod) {
+    super(authMethod);
+  }
+
+  /**
+   * Updated logic as compared to default implementation in
+   * {@link SpnegoAuthenticator#validateRequest(ServletRequest, ServletResponse, boolean)} to handle below cases:
+   * 1) Perform SPNEGO authentication only when spnegoLogin resource is requested. This helps to avoid authentication
+   *    for each and every resource which the JETTY provided authenticator does.
+   * 2) Helps to redirect to the target URL after authentication is done successfully.
+   * 3) Clear-Up in memory session information once LogOut is triggered such that any future request also triggers SPNEGO
+   *    authentication.
+   * @param request
+   * @param response
+   * @param mandatoryAuth
+   * @return
+   * @throws ServerAuthException
+   */
+  @Override
+  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatoryAuth)
+      throws ServerAuthException {
+
+    final HttpServletRequest req = (HttpServletRequest) request;
+    final HttpSession session = req.getSession(true);
+    final Authentication authentication = (Authentication) session.getAttribute(SessionAuthentication.__J_AUTHENTICATED);
+    final String uri = req.getRequestURI();
+
+    // If the Request URI is for /spnegoLogin then perform login
+    final boolean mandatory = mandatoryAuth || uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
+
+    // For logout remove the attribute from the session that holds UserIdentity
+    if (authentication != null) {
+      if (uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
+        logger.debug("Logging out user {}", req.getRemoteAddr());
+        session.removeAttribute(SessionAuthentication.__J_AUTHENTICATED);
+        return null;
+      }
+
+      // Already logged in so just return the session attribute.
+      return authentication;
+    }
+
+    // Try to authenticate an unauthenticated session.
+    return authenticateSession(request, response, mandatory);
+  }
+
+  /**
+   * Method to authenticate a user session using the SPNEGO token passed in AUTHORIZATION header of request.
+   * @param request
+   * @param response
+   * @param mandatory
+   * @return
+   * @throws ServerAuthException
+   */
+  private Authentication authenticateSession(ServletRequest request, ServletResponse response, boolean mandatory)
+      throws ServerAuthException {
+
+    final HttpServletRequest req = (HttpServletRequest) request;
+    final HttpServletResponse res = (HttpServletResponse) response;
+    final HttpSession session = req.getSession(true);
+
+    // Defer the authentication if not mandatory.
+    if (!mandatory) {
+      return new DeferredAuthentication(this);
+    }
+
+    // Authentication is mandatory, get the Authorization header
+    final String header = req.getHeader(HttpHeader.AUTHORIZATION.asString());
+
+    // Authorization header is null, so send the 401 error code to client along with negotiate header
+    if (header == null) {
+      try {
+        if (DeferredAuthentication.isDeferred(res)) {
+          return Authentication.UNAUTHENTICATED;
+        } else {
+          res.setHeader(HttpHeader.WWW_AUTHENTICATE.asString(), HttpHeader.NEGOTIATE.asString());
+          res.sendError(401);
+          logger.debug("DrillSpnegoAuthenticator: Sending challenge to client {}", req.getRemoteAddr());
+          return Authentication.SEND_CONTINUE;
+        }
+      } catch (IOException e) {
+        logger.error("DrillSpnegoAuthenticator: Failed while sending challenge to client {}", req.getRemoteAddr(), e);
+        throw new ServerAuthException(e);
+      }
+    }
+
+    // Valid Authorization header received. Get the SPNEGO token sent by client and try to authenticate
+    logger.debug("DrillSpnegoAuthenticator: Received NEGOTIATE Response back from client {}", req.getRemoteAddr());
+    final String negotiateString = HttpHeader.NEGOTIATE.asString();
+
+    if (header.startsWith(negotiateString)) {
+      final String spnegoToken = header.substring(negotiateString.length() + 1);
+      final UserIdentity user = this.login(null, spnegoToken, request);
+
+      //redirect the request to the desired page after successful login
+      if (user != null) {
+        String newUri = (String) session.getAttribute("org.eclipse.jetty.security.form_URI");
+        if (Strings.isNullOrEmpty(newUri)) {
+          newUri = req.getContextPath();
+          if (Strings.isNullOrEmpty(newUri)) {
+            newUri = WebServerConstants.WEBSERVER_ROOT_PATH;
+          }
+        }
+
+        response.setContentLength(0);
+        final HttpChannel channel = HttpChannel.getCurrentHttpChannel();
+        final Response base_response = channel.getResponse();
+        final Request base_request = channel.getRequest();
+        final int redirectCode =
+            base_request.getHttpVersion().getVersion() < HttpVersion.HTTP_1_1.getVersion() ? 302 : 303;
+        try {
+          base_response.sendRedirect(redirectCode, res.encodeRedirectURL(newUri));
+        } catch (IOException e) {
+          logger.error("DrillSpnegoAuthenticator: Failed while using the redirect URL {} from client {}", newUri,
+              req.getRemoteAddr(), e);
+          throw new ServerAuthException(e);
+        }
+
+        logger.debug("DrillSpnegoAuthenticator: Successfully authenticated this client session: {}",
+            user.getUserPrincipal().getName());
+        return new UserAuthentication(this.getAuthMethod(), user);
+      }
+    }
+
+    logger.debug("DrillSpnegoAuthenticator: Authentication failed for client session: {}", req.getRemoteAddr());
+    return Authentication.UNAUTHENTICATED;
+
+  }
+
+  public UserIdentity login(String username, Object password, ServletRequest request) {
+    final UserIdentity user = super.login(username, password, request);
+
+    if (user != null) {
+      final HttpSession session = ((HttpServletRequest) request).getSession(true);
+      final Authentication cached = new SessionAuthentication(this.getAuthMethod(), user, password);
+      session.setAttribute(SessionAuthentication.__J_AUTHENTICATED, cached);
+    }
+
+    return user;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoLoginService.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoLoginService.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoLoginService.java
new file mode 100644
index 0000000..e7fbc16
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoLoginService.java
@@ -0,0 +1,149 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.server.rest.auth;
+
+
+import org.apache.drill.common.exceptions.DrillException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.SystemOptionManager;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.eclipse.jetty.security.DefaultIdentityService;
+import org.eclipse.jetty.security.SpnegoLoginService;
+import org.eclipse.jetty.server.UserIdentity;
+import org.eclipse.jetty.util.B64Code;
+import org.ietf.jgss.GSSContext;
+import org.ietf.jgss.GSSCredential;
+import org.ietf.jgss.GSSException;
+import org.ietf.jgss.GSSManager;
+import org.ietf.jgss.GSSName;
+import org.ietf.jgss.Oid;
+
+import javax.security.auth.Subject;
+import java.lang.reflect.Field;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+
+/**
+ * Custom implementation of DrillSpnegoLoginService to avoid the need of passing targetName in a config file,
+ * to include the SPNEGO OID and the way UserIdentity is created.
+ */
+public class DrillSpnegoLoginService extends SpnegoLoginService {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoLoginService.class);
+
+  private static final String TARGET_NAME_FIELD_NAME = "_targetName";
+
+  private final DrillbitContext drillContext;
+
+  private final SpnegoConfig spnegoConfig;
+
+  private final UserGroupInformation loggedInUgi;
+
+  public DrillSpnegoLoginService(DrillbitContext drillBitContext) throws DrillException {
+    super(DrillSpnegoLoginService.class.getName());
+    setIdentityService(new DefaultIdentityService());
+    drillContext = drillBitContext;
+
+    // Load and verify SPNEGO config. Then Login using creds to get an UGI instance
+    spnegoConfig = new SpnegoConfig(drillBitContext.getConfig());
+    spnegoConfig.validateSpnegoConfig();
+    loggedInUgi = spnegoConfig.getLoggedInUgi();
+  }
+
+  @Override
+  protected void doStart() throws Exception {
+    // Override the parent implementation, setting _targetName to be the serverPrincipal
+    // without the need for a one-line file to do the same thing.
+    final Field targetNameField = SpnegoLoginService.class.getDeclaredField(TARGET_NAME_FIELD_NAME);
+    targetNameField.setAccessible(true);
+    targetNameField.set(this, spnegoConfig.getSpnegoPrincipal());
+  }
+
+  @Override
+  public UserIdentity login(final String username, final Object credentials) {
+
+    UserIdentity identity = null;
+    try {
+      identity = loggedInUgi.doAs(new PrivilegedExceptionAction<UserIdentity>() {
+        @Override
+        public UserIdentity run() {
+          return spnegoLogin(credentials);
+        }
+      });
+    } catch (Exception e) {
+      logger.error("Failed to login using SPNEGO", e);
+    }
+
+    return identity;
+  }
+
+  private UserIdentity spnegoLogin(Object credentials) {
+
+    String encodedAuthToken = (String) credentials;
+    byte[] authToken = B64Code.decode(encodedAuthToken);
+    GSSManager manager = GSSManager.getInstance();
+
+    try {
+      // Providing both OID's is required here. If we provide only one,
+      // we're requiring that clients provide us the SPNEGO OID to authenticate via Kerberos.
+      Oid[] knownOids = new Oid[2];
+      knownOids[0] = new Oid("1.3.6.1.5.5.2"); // spnego
+      knownOids[1] = new Oid("1.2.840.113554.1.2.2"); // kerberos
+
+      GSSName gssName = manager.createName(spnegoConfig.getSpnegoPrincipal(), null);
+      GSSCredential serverCreds = manager.createCredential(gssName, GSSCredential.INDEFINITE_LIFETIME,
+          knownOids, GSSCredential.ACCEPT_ONLY);
+      GSSContext gContext = manager.createContext(serverCreds);
+
+      if (gContext == null) {
+        logger.debug("SPNEGOUserRealm: failed to establish GSSContext");
+      } else {
+        while (!gContext.isEstablished()) {
+          authToken = gContext.acceptSecContext(authToken, 0, authToken.length);
+        }
+
+        if (gContext.isEstablished()) {
+          String clientName = gContext.getSrcName().toString();
+          String role = clientName.substring(clientName.indexOf(64) + 1);
+
+          final SystemOptionManager sysOptions = drillContext.getOptionManager();
+          final boolean isAdmin = ImpersonationUtil.hasAdminPrivileges(role,
+              ExecConstants.ADMIN_USERS_VALIDATOR.getAdminUsers(sysOptions),
+              ExecConstants.ADMIN_USER_GROUPS_VALIDATOR.getAdminUserGroups(sysOptions));
+
+          final Principal user = new DrillUserPrincipal(clientName, isAdmin);
+          final Subject subject = new Subject();
+          subject.getPrincipals().add(user);
+
+          if (isAdmin) {
+            return this._identityService.newUserIdentity(subject, user, DrillUserPrincipal.ADMIN_USER_ROLES);
+          } else {
+            return this._identityService.newUserIdentity(subject, user, DrillUserPrincipal.NON_ADMIN_USER_ROLES);
+          }
+        }
+      }
+    } catch (GSSException gsse) {
+      logger.warn("Caught GSSException trying to authenticate the client", gsse);
+    }
+    return null;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillUserPrincipal.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillUserPrincipal.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillUserPrincipal.java
index a21977f..01206c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillUserPrincipal.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillUserPrincipal.java
@@ -38,9 +38,11 @@ public class DrillUserPrincipal implements Principal {
 
   public static final String[] NON_ADMIN_USER_ROLES = new String[]{AUTHENTICATED_ROLE};
 
-  public static final List<RolePrincipal> ADMIN_PRINCIPALS = ImmutableList.of(new RolePrincipal(AUTHENTICATED_ROLE), new RolePrincipal(ADMIN_ROLE));
+  public static final List<RolePrincipal> ADMIN_PRINCIPALS =
+      ImmutableList.of(new RolePrincipal(AUTHENTICATED_ROLE), new RolePrincipal(ADMIN_ROLE));
 
-  public static final List<RolePrincipal> NON_ADMIN_PRINCIPALS = ImmutableList.of(new RolePrincipal(AUTHENTICATED_ROLE));
+  public static final List<RolePrincipal> NON_ADMIN_PRINCIPALS =
+      ImmutableList.of(new RolePrincipal(AUTHENTICATED_ROLE));
 
   private final String userName;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHanlder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHanlder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHanlder.java
new file mode 100644
index 0000000..31d7cec
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHanlder.java
@@ -0,0 +1,49 @@
+/*
+ * 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.drill.exec.server.rest.auth;
+
+import org.apache.drill.common.exceptions.DrillException;
+import org.apache.drill.exec.rpc.security.plain.PlainFactory;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.rest.WebServerConstants;
+import org.eclipse.jetty.security.authentication.FormAuthenticator;
+import org.eclipse.jetty.util.security.Constraint;
+
+public class FormSecurityHanlder extends DrillHttpConstraintSecurityHandler {
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FormSecurityHanlder.class);
+
+  @Override
+  public String getImplName() {
+    return Constraint.__FORM_AUTH;
+  }
+
+  @Override
+  public void doSetup(DrillbitContext dbContext) throws DrillException {
+
+    // Check if PAMAuthenticator is available or not which is required for FORM authentication
+    if (!dbContext.getAuthProvider().containsFactory(PlainFactory.SIMPLE_NAME)) {
+      throw new DrillException("FORM mechanism was configured but PLAIN mechanism is not enabled to provide an " +
+          "authenticator. Please configure user authentication with PLAIN mechanism and authenticator to use " +
+          "FORM authentication");
+    }
+
+    setup(new FormAuthenticator(WebServerConstants.FORM_LOGIN_RESOURCE_PATH,
+        WebServerConstants.FORM_LOGIN_RESOURCE_PATH, true), new DrillRestLoginService(dbContext));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoConfig.java
new file mode 100644
index 0000000..a64d7de
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoConfig.java
@@ -0,0 +1,112 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.server.rest.auth;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+public class SpnegoConfig {
+
+  private UserGroupInformation loggedInUgi;
+
+  private final String principal;
+
+  private final String keytab;
+
+  public SpnegoConfig(DrillConfig config) {
+
+    keytab = config.hasPath(ExecConstants.HTTP_SPNEGO_KEYTAB) ?
+        config.getString(ExecConstants.HTTP_SPNEGO_KEYTAB) :
+        null;
+
+    principal = config.hasPath(ExecConstants.HTTP_SPNEGO_PRINCIPAL) ?
+        config.getString(ExecConstants.HTTP_SPNEGO_PRINCIPAL) :
+        null;
+  }
+
+  //Reads the SPNEGO principal from the config file
+  public String getSpnegoPrincipal() {
+    return principal;
+  }
+
+  public void validateSpnegoConfig() throws DrillException {
+
+    StringBuilder errorMsg = new StringBuilder();
+
+    if (principal != null && keytab != null) {
+      return;
+    }
+
+    if (principal == null) {
+      errorMsg.append("\nConfiguration ");
+      errorMsg.append(ExecConstants.HTTP_SPNEGO_PRINCIPAL);
+      errorMsg.append(" is not found");
+    }
+
+    if (keytab == null) {
+      errorMsg.append("\nConfiguration ");
+      errorMsg.append(ExecConstants.HTTP_SPNEGO_KEYTAB);
+      errorMsg.append(" is not found");
+    }
+
+    throw new DrillException(errorMsg.toString());
+  }
+
+  public UserGroupInformation getLoggedInUgi() throws DrillException {
+
+    if (loggedInUgi != null) {
+      return loggedInUgi;
+    }
+    loggedInUgi = loginAndReturnUgi();
+    return loggedInUgi;
+  }
+
+  //Performs the Server login to KDC for SPNEGO
+  private UserGroupInformation loginAndReturnUgi() throws DrillException {
+
+    validateSpnegoConfig();
+
+    UserGroupInformation ugi;
+    try {
+      // Check if security is not enabled and try to set the security parameter to login the principal.
+      // After the login is performed reset the static UGI state.
+      if (!UserGroupInformation.isSecurityEnabled()) {
+        final Configuration newConfig = new Configuration();
+        newConfig.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
+            UserGroupInformation.AuthenticationMethod.KERBEROS.toString());
+
+        UserGroupInformation.setConfiguration(newConfig);
+        ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab);
+
+        // Reset the original configuration for static UGI
+        UserGroupInformation.setConfiguration(new Configuration());
+      } else {
+        ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab);
+      }
+    } catch (Exception e) {
+      throw new DrillException(String.format("Login failed for %s with given keytab", principal), e);
+    }
+    return ugi;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoSecurityHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoSecurityHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoSecurityHandler.java
new file mode 100644
index 0000000..9e6acb1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoSecurityHandler.java
@@ -0,0 +1,36 @@
+/*
+ * 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.drill.exec.server.rest.auth;
+
+import org.apache.drill.common.exceptions.DrillException;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.eclipse.jetty.util.security.Constraint;
+
+public class SpnegoSecurityHandler extends DrillHttpConstraintSecurityHandler {
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SpnegoSecurityHandler.class);
+
+  @Override
+  public String getImplName() {
+    return Constraint.__SPNEGO_AUTH;
+  }
+
+  @Override
+  public void doSetup(DrillbitContext dbContext) throws DrillException {
+    setup(new DrillSpnegoAuthenticator(getImplName()), new DrillSpnegoLoginService(dbContext));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index c923e4f..a0cf643 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -27,6 +27,7 @@ drill {
       org.apache.drill.exec.physical.impl.RootCreator,
       org.apache.drill.exec.rpc.user.security.UserAuthenticator,
       org.apache.drill.exec.rpc.security.AuthenticatorFactory,
+      org.apache.drill.exec.server.rest.auth.DrillHttpConstraintSecurityHandler,
       org.apache.drill.exec.store.dfs.FormatPlugin,
       org.apache.drill.exec.store.StoragePlugin
     ],
@@ -38,7 +39,8 @@ drill {
           org.apache.drill.exec.physical,
           org.apache.drill.exec.store,
           org.apache.drill.exec.rpc.user.security,
-          org.apache.drill.exec.rpc.security
+          org.apache.drill.exec.rpc.security,
+          org.apache.drill.exec.server.rest.auth
     ]
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/resources/rest/generic.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/generic.ftl b/exec/java-exec/src/main/resources/rest/generic.ftl
index 9025adb..d04414f 100644
--- a/exec/java-exec/src/main/resources/rest/generic.ftl
+++ b/exec/java-exec/src/main/resources/rest/generic.ftl
@@ -77,7 +77,7 @@
               </#if>
               <li><a href="http://drill.apache.org/docs/">Documentation</a>
               <#if showLogin == true >
-              <li><a href="/login">Log In</a>
+              <li><a href="/mainLogin">Log In</a>
               </#if>
               <#if showLogout == true >
               <li><a href="/logout">Log Out (${loggedInUserName})</a>

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/main/resources/rest/mainLogin.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/mainLogin.ftl b/exec/java-exec/src/main/resources/rest/mainLogin.ftl
new file mode 100644
index 0000000..8deb156
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/mainLogin.ftl
@@ -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. -->
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+        <div class="page-header">
+        </div>
+        <div class="container container-table">
+        <div align="center" class="table-responsive">
+        <#if model?? && model.isFormEnabled()>
+        <a href ="/login" class="btn btn-primary"> Login using FORM AUTHENTICATION </a>
+        </#if>
+        <#if model?? && model.isSpnegoEnabled()>
+        <a href = "/spnegoLogin" class="btn btn-primary"> Login using SPNEGO </a>
+        </#if>
+        <#if model?? && model.getError()??>
+        <p style="color:red">${model.getError()}</p></br>
+        </#if>
+        </div>
+        </div>
+</#macro>
+<@page_html/>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
index 81b027f..a24b0db 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
@@ -95,7 +95,8 @@ public class TestBitBitKerberos extends BaseTestQuery {
   public static void setupTest() throws Exception {
 
     final Config config = DrillConfig.create(cloneDefaultTestConfigProperties());
-    krbHelper = new KerberosHelper(TestBitBitKerberos.class.getSimpleName());
+
+    krbHelper = new KerberosHelper(TestBitBitKerberos.class.getSimpleName(), null);
     krbHelper.setupKdc(dirTestWatcher.getTmpDir());
 
     newConfig = new DrillConfig(

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/KerberosHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/KerberosHelper.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/KerberosHelper.java
index 451e0aa..8ba4d18 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/KerberosHelper.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/KerberosHelper.java
@@ -49,10 +49,14 @@ public class KerberosHelper {
 
   private boolean kdcStarted;
 
-  public KerberosHelper(final String testName) {
+  public KerberosHelper(final String testName, String serverShortName) {
     final String realm = "EXAMPLE.COM";
     CLIENT_PRINCIPAL = CLIENT_SHORT_NAME + "@" + realm;
-    final String serverShortName = System.getProperty("user.name");
+
+    if (serverShortName == null) {
+      serverShortName = System.getProperty("user.name");
+    }
+
     SERVER_PRINCIPAL = serverShortName + "/" + HOSTNAME + "@" + realm;
     this.testName = testName;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
index d6495e9..065746c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
@@ -56,7 +56,7 @@ public class TestUserBitKerberos extends BaseTestQuery {
   @BeforeClass
   public static void setupTest() throws Exception {
 
-    krbHelper = new KerberosHelper(TestUserBitKerberos.class.getSimpleName());
+    krbHelper = new KerberosHelper(TestUserBitKerberos.class.getSimpleName(), null);
     krbHelper.setupKdc(dirTestWatcher.getTmpDir());
 
     // Create a new DrillConfig which has user authentication enabled and authenticator set to

http://git-wip-us.apache.org/repos/asf/drill/blob/adee4614/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java
index 4f411ae..ac60880 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java
@@ -60,7 +60,7 @@ public class TestUserBitKerberosEncryption extends BaseTestQuery {
 
   @BeforeClass
   public static void setupTest() throws Exception {
-    krbHelper = new KerberosHelper(TestUserBitKerberosEncryption.class.getSimpleName());
+    krbHelper = new KerberosHelper(TestUserBitKerberosEncryption.class.getSimpleName(), null);
     krbHelper.setupKdc(dirTestWatcher.getTmpDir());
 
     // Create a new DrillConfig which has user authentication enabled and authenticator set to