You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2018/01/21 19:44:05 UTC

[1/4] drill git commit: DRILL-5967: Fixed memory leak in OrderedPartitionSender

Repository: drill
Updated Branches:
  refs/heads/master 9e944c97e -> a2a56af3e


DRILL-5967: Fixed memory leak in OrderedPartitionSender

closes apache/drill#1073


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

Branch: refs/heads/master
Commit: 81f926b5ea5c8537c7c65db397efc0fdf0a1d589
Parents: 9e944c9
Author: Timothy Farkas <ti...@apache.org>
Authored: Thu Dec 14 10:48:27 2017 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sun Jan 21 08:32:38 2018 -0800

----------------------------------------------------------------------
 .../OrderedPartitionSenderCreator.java          | 35 +++++++++++++++++---
 .../PartitionSenderRootExec.java                | 14 ++++++++
 2 files changed, 44 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/81f926b5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java
index 5c953b1..d2e07e7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionSenderCreator.java
@@ -29,8 +29,34 @@ import org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootEx
 import org.apache.drill.exec.record.RecordBatch;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
+/**
+ * <br/>
+ * <h1>Known Issues:</h1>
+ * <h2>Creation of batches</h2>
+ * <p>
+ *   The {@link org.apache.drill.exec.work.fragment.FragmentExecutor} is only aware of the operators in the tree that it has a reference too. In the case of the
+ *   {@link OrderedPartitionSenderCreator}, an upstream {@link org.apache.drill.exec.record.RecordBatch} is wrapped in an
+ *   {@link org.apache.drill.exec.physical.impl.orderedpartitioner.OrderedPartitionRecordBatch}. Since the
+ *   {@link org.apache.drill.exec.physical.impl.orderedpartitioner.OrderedPartitionRecordBatch} is instantiated in the creator the
+ *   {@link org.apache.drill.exec.work.fragment.FragmentExecutor} does not have a reference to it. So when the {@link org.apache.drill.exec.work.fragment.FragmentExecutor}
+ *   closes the operators it closes the original operator, but not not the wrapping {@link OrderedPartitionSenderCreator}. This is an issue since the
+ *   {@link OrderedPartitionSenderCreator} allocates {@link org.apache.drill.exec.record.VectorContainer}s which are consequentially never released.
+ *   <br/>
+ *   <ol>
+ *     <li>
+ *       We change the Creators in some way to communicate to the FragmentExecutor that they have wrapped an operator, so the FragmentExecutor can close the wrapped operator
+ *       instead of the original operator.
+ *     </li>
+ *     <li>
+ *       Or we take a less invasive approach and simply tell the {@link org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec} whether to close the wrapped
+ *       operator.
+ *     </li>
+ *   </ol>
+ *   <br/>
+ *   For now we've taken approach 2. In the future we should we should implement approach 1.
+ * </p>
+ */
 public class OrderedPartitionSenderCreator implements RootCreator<OrderedPartitionSender> {
 
   @SuppressWarnings("resource")
@@ -39,10 +65,9 @@ public class OrderedPartitionSenderCreator implements RootCreator<OrderedPartiti
       List<RecordBatch> children) throws ExecutionSetupException {
     Preconditions.checkArgument(children.size() == 1);
 
-    List<RecordBatch> ordered_children = Lists.newArrayList();
-    ordered_children.add(new OrderedPartitionRecordBatch(config, children.iterator().next(), context));
-    HashPartitionSender hpc = new HashPartitionSender(config.getOppositeMajorFragmentId(), config, config.getRef(), config.getDestinations());
-    return new PartitionSenderRootExec(context, ordered_children.iterator().next(), hpc);
+    final OrderedPartitionRecordBatch recordBatch = new OrderedPartitionRecordBatch(config, children.iterator().next(), context);
+    final HashPartitionSender hpc = new HashPartitionSender(config.getOppositeMajorFragmentId(), config, config.getRef(), config.getDestinations());
+    return new PartitionSenderRootExec(context, recordBatch, hpc, true);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/81f926b5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index 7684e94..108d539 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -43,6 +43,7 @@ import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.CloseableRecordBatch;
 import org.apache.drill.exec.record.FragmentWritableBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
@@ -72,6 +73,7 @@ public class PartitionSenderRootExec extends BaseRootExec {
   private final AtomicInteger remaingReceiverCount;
   private volatile boolean done = false;
   private boolean first = true;
+  private boolean closeIncoming;
 
   long minReceiverRecordCount = Long.MAX_VALUE;
   long maxReceiverRecordCount = Long.MIN_VALUE;
@@ -99,9 +101,17 @@ public class PartitionSenderRootExec extends BaseRootExec {
   public PartitionSenderRootExec(FragmentContext context,
                                  RecordBatch incoming,
                                  HashPartitionSender operator) throws OutOfMemoryException {
+    this(context, incoming, operator, false);
+  }
+
+  public PartitionSenderRootExec(FragmentContext context,
+                                 RecordBatch incoming,
+                                 HashPartitionSender operator,
+                                 boolean closeIncoming) throws OutOfMemoryException {
     super(context, context.newOperatorContext(operator, null), operator);
     this.incoming = incoming;
     this.operator = operator;
+    this.closeIncoming = closeIncoming;
     this.context = context;
     outGoingBatchCount = operator.getDestinations().size();
     popConfig = operator;
@@ -341,6 +351,10 @@ public class PartitionSenderRootExec extends BaseRootExec {
       updateAggregateStats();
       partitioner.clear();
     }
+
+    if (closeIncoming) {
+      ((CloseableRecordBatch) incoming).close();
+    }
   }
 
   public void sendEmptyBatch(boolean isLast) {


[3/4] drill git commit: DRILL-6085: Fixed spontaneous vm exits on Travis.

Posted by am...@apache.org.
DRILL-6085: Fixed spontaneous vm exits on Travis.

close apache/drill#1095


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

Branch: refs/heads/master
Commit: dbaf048c7443e453a78cdb5a976ed19ebb9ed2c7
Parents: 3ff958c
Author: Timothy Farkas <ti...@apache.org>
Authored: Tue Jan 16 12:11:25 2018 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sun Jan 21 08:32:57 2018 -0800

----------------------------------------------------------------------
 .travis.yml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/dbaf048c/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 68d6814..a2f5bd8 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -14,6 +14,7 @@
 # limitations under the License.
 
 before_install: git fetch --unshallow
+sudo: required
 language: java
 jdk:
   - openjdk7


[2/4] drill git commit: DRILL-6081: Set query end time before writing final profile

Posted by am...@apache.org.
DRILL-6081: Set query end time before writing final profile

close apache/drill#1088


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

Branch: refs/heads/master
Commit: 3ff958cbcac644685f0c6e984eb4764c55e226f3
Parents: 81f926b
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Thu Jan 11 17:40:04 2018 +0200
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sun Jan 21 08:32:46 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/drill/exec/work/foreman/Foreman.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3ff958cb/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 276d839..5633022 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -746,6 +746,9 @@ public class Foreman implements Runnable {
         });
       }
 
+      // set query end time before writing final profile
+      queryStateProcessor.close();
+
       /*
        * Construct the response based on the latest resultState. The builder shouldn't fail.
        */
@@ -813,8 +816,6 @@ public class Foreman implements Runnable {
         logger.warn("unable to close query manager", e);
       }
 
-      queryStateProcessor.close();
-
       try {
         queryRM.exit();
       } finally {


[4/4] drill git commit: DRILL-6088: MainLoginPageModel errors out when http.auth.mechanisms is not configured

Posted by am...@apache.org.
DRILL-6088: MainLoginPageModel errors out when http.auth.mechanisms is not configured

DRILL-6088: Update based on review feedback

close apache/drill#1092


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

Branch: refs/heads/master
Commit: a2a56af3e3f2a3c92bbfafeeb7a1f285a64e4a73
Parents: dbaf048
Author: Sorabh Hamirwasia <sh...@maprtech.com>
Authored: Sun Jan 14 22:25:51 2018 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sun Jan 21 08:33:05 2018 -0800

----------------------------------------------------------------------
 .../src/resources/drill-override-example.conf   |  11 +-
 .../exec/server/rest/LogInLogOutResources.java  |  26 ++--
 .../auth/DrillHttpSecurityHandlerProvider.java  |  89 ++++++++-----
 .../server/rest/TestMainLoginPageModel.java     | 133 +++++++++++++++++++
 4 files changed, 211 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a2a56af3/distribution/src/resources/drill-override-example.conf
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-override-example.conf b/distribution/src/resources/drill-override-example.conf
index dbc9c8e..fa2e395 100644
--- a/distribution/src/resources/drill-override-example.conf
+++ b/distribution/src/resources/drill-override-example.conf
@@ -91,7 +91,16 @@ drill.exec: {
       allowedMethods: ["GET", "POST", "HEAD", "OPTIONS"],
       allowedHeaders: ["X-Requested-With", "Content-Type", "Accept", "Origin"],
       credentials: true
-    }
+    },
+    auth: {
+        # Http Auth mechanisms to configure. If not provided but user.auth is enabled
+        # then default value is FORM.
+        mechanisms: ["FORM", "SPNEGO"],
+        # Spnego principal to be used by WebServer when Spnego authentication is enabled.
+        spnego.principal: "HTTP://<localhost>"
+        # Location to keytab file for above spnego principal
+        spnego.keytab: "<keytab_file_location>";
+    },
   },
   # Below SSL parameters need to be set for custom transport layer settings.
   ssl: {

http://git-wip-us.apache.org/repos/asf/drill/blob/a2a56af3/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 34ac4d6..8b750ac 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
@@ -17,11 +17,12 @@
  */
 package org.apache.drill.exec.server.rest;
 
+import com.google.common.annotations.VisibleForTesting;
 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.server.rest.auth.DrillHttpSecurityHandlerProvider;
 import org.apache.drill.exec.work.WorkManager;
 import org.eclipse.jetty.security.authentication.FormAuthenticator;
 import org.eclipse.jetty.util.security.Constraint;
@@ -57,8 +58,9 @@ public class LogInLogOutResources {
   /**
    * 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
+   * @param request  - Http Servlet Request
    * @throws Exception
    */
   private void updateSessionRedirectInfo(String redirect, HttpServletRequest request) throws Exception {
@@ -100,8 +102,7 @@ public class LogInLogOutResources {
     }
 
     final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
-    final DrillConfig drillConfig = workManager.getContext().getConfig();
-    MainLoginPageModel model = new MainLoginPageModel(errorString, drillConfig);
+    final MainLoginPageModel model = new MainLoginPageModel(errorString);
     return ViewableWithPermissions.createMainLoginPage(model);
   }
 
@@ -132,12 +133,15 @@ public class LogInLogOutResources {
                                    @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);
+    final MainLoginPageModel model = new MainLoginPageModel(null);
     return ViewableWithPermissions.createMainLoginPage(model);
   }
 
-  private class MainLoginPageModel {
+  /**
+   * This class should be public for it's method's to be accessible by mainLogin.ftl file
+   */
+  @VisibleForTesting
+  public class MainLoginPageModel {
 
     private final String error;
 
@@ -145,11 +149,11 @@ public class LogInLogOutResources {
 
     private final Set<String> configuredMechs;
 
-    MainLoginPageModel(String error, DrillConfig drillConfig) {
+    MainLoginPageModel(String error) {
       this.error = error;
-      authEnabled = drillConfig.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED);
-      configuredMechs = AuthStringUtil.asSet(
-          drillConfig.getStringList(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS));
+      final DrillConfig config = workManager.getContext().getConfig();
+      authEnabled = config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED);
+      configuredMechs = DrillHttpSecurityHandlerProvider.getHttpAuthMechanisms(config);
     }
 
     public boolean isSpnegoEnabled() {

http://git-wip-us.apache.org/repos/asf/drill/blob/a2a56af3/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
index 3d77596..9eda5c4 100644
--- 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
@@ -58,51 +58,46 @@ public class DrillHttpSecurityHandlerProvider extends ConstraintSecurityHandler
       throws DrillbitStartupException {
 
     Preconditions.checkState(config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED));
-    final Set<String> configuredMechanisms = new HashSet<>();
+    final Set<String> configuredMechanisms = getHttpAuthMechanisms(config);
 
-    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);
 
-      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) {
+    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;
-        }
+      // 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;
-          }
+      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;
-        }
+      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);
+      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 " +
@@ -181,4 +176,26 @@ public class DrillHttpSecurityHandlerProvider extends ConstraintSecurityHandler
   public boolean isFormEnabled() {
     return securityHandlers.containsKey(Constraint.__FORM_AUTH);
   }
+
+  /**
+   * Return's list of configured mechanisms for HTTP authentication. For backward
+   * compatibility if authentication is enabled it will include FORM mechanism by default.
+   * @param config - {@link DrillConfig}
+   * @return
+   */
+  public static Set<String> getHttpAuthMechanisms(DrillConfig config) {
+    final Set<String> configuredMechs = new HashSet<>();
+    final boolean authEnabled = config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED);
+
+    if (authEnabled) {
+      if (config.hasPath(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS)) {
+        configuredMechs.addAll(
+            AuthStringUtil.asSet(config.getStringList(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS)));
+      } else {
+        // For backward compatibility
+        configuredMechs.add(Constraint.__FORM_AUTH);
+      }
+    }
+    return configuredMechs;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/a2a56af3/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/TestMainLoginPageModel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/TestMainLoginPageModel.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/TestMainLoginPageModel.java
new file mode 100644
index 0000000..3b36704
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/TestMainLoginPageModel.java
@@ -0,0 +1,133 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import com.typesafe.config.ConfigValueFactory;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.rest.LogInLogOutResources.MainLoginPageModel;
+import org.apache.drill.exec.work.WorkManager;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for {@link LogInLogOutResources.MainLoginPageModel} with various configurations done in DrillConfig
+ */
+public class TestMainLoginPageModel {
+
+  @Mock
+  WorkManager workManager;
+
+  @Mock
+  DrillbitContext context;
+
+  @InjectMocks
+  LogInLogOutResources logInLogOutResources = new LogInLogOutResources();
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    when(workManager.getContext()).thenReturn(context);
+  }
+
+  /**
+   * Test when auth is disabled then both Form and Spnego authentication is disabled.
+   */
+  @Test
+  public void testAuthDisabled() {
+    final DrillConfig config = DrillConfig.create();
+    when(context.getConfig()).thenReturn(config);
+    final MainLoginPageModel model = logInLogOutResources.new MainLoginPageModel(null);
+    assertTrue(!model.isFormEnabled());
+    assertTrue(!model.isSpnegoEnabled());
+  }
+
+  /**
+   * Test when auth is enabled with no http.auth.mechanisms configured then by default Form authentication is
+   * enabled but Spnego is disabled.
+   */
+  @Test
+  public void testAuthEnabledWithNoMech() {
+    final DrillConfig config = new DrillConfig(DrillConfig.create()
+      .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
+        ConfigValueFactory.fromAnyRef(true)), false);
+    when(context.getConfig()).thenReturn(config);
+    final MainLoginPageModel model = logInLogOutResources.new MainLoginPageModel(null);
+    assertTrue(model.isFormEnabled());
+    assertTrue(!model.isSpnegoEnabled());
+  }
+
+  /**
+   * Test when auth is enabled with http.auth.mechanisms configured as Form then only Form authentication is
+   * enabled but Spnego is disabled.
+   */
+  @Test
+  public void testAuthEnabledWithForm() {
+    final DrillConfig config = new DrillConfig(DrillConfig.create()
+      .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
+        ConfigValueFactory.fromAnyRef(true))
+      .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
+        ConfigValueFactory.fromIterable(Lists.newArrayList("form"))), false);
+    when(context.getConfig()).thenReturn(config);
+    final MainLoginPageModel model = logInLogOutResources.new MainLoginPageModel(null);
+    assertTrue(model.isFormEnabled());
+    assertTrue(!model.isSpnegoEnabled());
+  }
+
+  /**
+   * Test when auth is enabled with http.auth.mechanisms configured as Spnego then only Spnego authentication is
+   * enabled but Form is disabled.
+   */
+  @Test
+  public void testAuthEnabledWithSpnego() {
+    final DrillConfig config = new DrillConfig(DrillConfig.create()
+      .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
+        ConfigValueFactory.fromAnyRef(true))
+      .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
+        ConfigValueFactory.fromIterable(Lists.newArrayList("spnego"))), false);
+    when(context.getConfig()).thenReturn(config);
+    final MainLoginPageModel model = logInLogOutResources.new MainLoginPageModel(null);
+    assertTrue(!model.isFormEnabled());
+    assertTrue(model.isSpnegoEnabled());
+  }
+
+  /**
+   * Test when auth is enabled with http.auth.mechanisms configured as Form, Spnego then both Form and Spnego
+   * authentication are enabled.
+   */
+  @Test
+  public void testAuthEnabledWithFormSpnego() {
+    final DrillConfig config = new DrillConfig(DrillConfig.create()
+      .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
+        ConfigValueFactory.fromAnyRef(true))
+      .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
+        ConfigValueFactory.fromIterable(Lists.newArrayList("form", "spnego"))), false);
+    when(context.getConfig()).thenReturn(config);
+    final MainLoginPageModel model = logInLogOutResources.new MainLoginPageModel(null);
+    assertTrue(model.isFormEnabled());
+    assertTrue(model.isSpnegoEnabled());
+  }
+}
\ No newline at end of file