You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by sohami <gi...@git.apache.org> on 2017/11/17 00:33:49 UTC

[GitHub] drill pull request #1040: Drill 5425

GitHub user sohami opened a pull request:

    https://github.com/apache/drill/pull/1040

    Drill 5425

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/sohami/drill DRILL-5425

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/1040.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1040
    
----
commit abf8eee1e267871af78bb81d7f41708f961c71d7
Author: Sindhuri Rayavaram <sr...@mapr.com>
Date:   2017-09-11T23:56:22Z

    DRILL-5425: Support HTTP Kerberos auth using SPNEGO

commit 5e54f4ab2768f687f86d78e1992aa9a2c797840a
Author: Sorabh Hamirwasia <sh...@maprtech.com>
Date:   2017-11-13T22:43:52Z

    DRILL-5425: Refactor, Add tests and bunch of fixes

----


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158135822
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoUtil.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.CommonConfigurationKeys;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +public class SpnegoUtil {
    --- End diff --
    
    Renamed to SpnegoConfig


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156660850
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SPNEGOSecurityHandler.java ---
    @@ -0,0 +1,37 @@
    +/*
    + * 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;
    +
    +public class SPNEGOSecurityHandler extends DrillHttpConstraintSecurityHandler {
    +  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SPNEGOSecurityHandler.class);
    +
    +  public static final String HANDLER_NAME = "SPNEGO";
    +
    +  @Override
    +  public String getImplName() {
    +    return HANDLER_NAME;
    --- End diff --
    
    You can use `org.eclipse.jetty.util.security.Constraint.__SPNEGO_AUTH` instead.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158164495
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    --- End diff --
    
    Refactored this method to be more readable.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156651931
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    --- End diff --
    
    Why 5?


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156670865
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    +      logger.debug("Logging out user {}", req.getRemoteAddr());
    +      session.removeAttribute("org.eclipse.jetty.security.UserIdentity");
    +      return null;
    +    } else if (authentication != null) { // Since already logged in just return the session attribute
    +      return authentication;
    +    } else { // The session is not yet authenticated
    +      final String header = req.getHeader(HttpHeader.AUTHORIZATION.asString());
    +      if (!mandatory) {
    +        return new DeferredAuthentication(this);
    +      } else 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("SPNEGOAuthenticator: Sending challenge to client {}", req.getRemoteAddr());
    +            return Authentication.SEND_CONTINUE;
    +          }
    +        } catch (IOException var9) {
    --- End diff --
    
    `var9`? From where this code was copied? Is it public resource?


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156673473
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    +
    +  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(FORMSecurityHanlder.HANDLER_NAME);
    +    }
    +
    +      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 | IllegalAccessException |
    +            InstantiationException | InvocationTargetException | 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("org.eclipse.jetty.security.UserIdentity");
    --- End diff --
    
    Can be replaced with org.eclipse.jetty.security.authentication.SessionAuthentication,__J_AUTHENTICATED


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158164861
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    --- End diff --
    
    Changed. 


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158099938
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    +      return null;
    +    }
    +
    +    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
    +    MainLoginPageModel model = new MainLoginPageModel(errorString);
    +    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 {
    +    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.
    +      final HttpSession session = request.getSession(true);
    +      final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
    +      session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
    +    }
    +
    +    MainLoginPageModel model = new MainLoginPageModel(null);
    +    return ViewableWithPermissions.createMainLoginPage(model);
    +  }
    +
    +  public class MainLoginPageModel {
    +
    +    public String error;
    +
    +    MainLoginPageModel(String error) {
    --- End diff --
    
    Since the class is used only by `LogInLogOutResources` I have made it private and left constructor as package-private


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158112333
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    +
    +  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(FORMSecurityHanlder.HANDLER_NAME);
    +    }
    +
    +      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 | IllegalAccessException |
    +            InstantiationException | InvocationTargetException | 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("org.eclipse.jetty.security.UserIdentity");
    --- End diff --
    
    Done.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158137486
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    +      logger.debug("Logging out user {}", req.getRemoteAddr());
    +      session.removeAttribute("org.eclipse.jetty.security.UserIdentity");
    --- End diff --
    
    Changed in all the places.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158099736
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    +      return null;
    +    }
    +
    +    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
    +    MainLoginPageModel model = new MainLoginPageModel(errorString);
    +    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 {
    +    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.
    +      final HttpSession session = request.getSession(true);
    +      final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
    +      session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
    +    }
    --- End diff --
    
    Done. Added private method `updateSessionRedirectInfo`


---

[GitHub] drill issue #1040: Drill 5425

Posted by priteshm <gi...@git.apache.org>.
Github user priteshm commented on the issue:

    https://github.com/apache/drill/pull/1040
  
    @arina-ielchiieva can you please review this?


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156661450
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SPNEGOSecurityHandler.java ---
    @@ -0,0 +1,37 @@
    +/*
    + * 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;
    +
    +public class SPNEGOSecurityHandler extends DrillHttpConstraintSecurityHandler {
    --- End diff --
    
    I suggest we use initcap for spnego (jetty does the same, i.e. `SpnegoAuthenticator`) -> `SpnegoSecurityHandler`


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156670981
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    +      logger.debug("Logging out user {}", req.getRemoteAddr());
    +      session.removeAttribute("org.eclipse.jetty.security.UserIdentity");
    +      return null;
    +    } else if (authentication != null) { // Since already logged in just return the session attribute
    +      return authentication;
    +    } else { // The session is not yet authenticated
    +      final String header = req.getHeader(HttpHeader.AUTHORIZATION.asString());
    +      if (!mandatory) {
    +        return new DeferredAuthentication(this);
    +      } else 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("SPNEGOAuthenticator: Sending challenge to client {}", req.getRemoteAddr());
    +            return Authentication.SEND_CONTINUE;
    +          }
    +        } catch (IOException var9) {
    +          throw new ServerAuthException(var9);
    +        }
    +      } else {
    +        logger.debug("SPNEGOAuthenticator: 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) {
    +              e.printStackTrace();
    --- End diff --
    
    Please log exception or re-throw it.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158100823
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java ---
    @@ -183,17 +180,19 @@ public void start() throws Exception {
         }
       }
     
    -  private ServletContextHandler createServletContextHandler(final boolean authEnabled) {
    +  private ServletContextHandler createServletContextHandler(final boolean authEnabled) throws Exception {
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158166373
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    +      logger.debug("Logging out user {}", req.getRemoteAddr());
    +      session.removeAttribute("org.eclipse.jetty.security.UserIdentity");
    +      return null;
    +    } else if (authentication != null) { // Since already logged in just return the session attribute
    +      return authentication;
    +    } else { // The session is not yet authenticated
    +      final String header = req.getHeader(HttpHeader.AUTHORIZATION.asString());
    +      if (!mandatory) {
    +        return new DeferredAuthentication(this);
    +      } else 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("SPNEGOAuthenticator: Sending challenge to client {}", req.getRemoteAddr());
    +            return Authentication.SEND_CONTINUE;
    +          }
    +        } catch (IOException var9) {
    --- End diff --
    
    This code was developed taking help from the base class implementation of Jetty and adding bunch of other functionality as mentioned in the class level documentation. I have changed the variable name and moved documentation from class level to method level. Licensing for Jetty code is under Apache license. See [here](https://github.com/eclipse/jetty.project/blob/jetty-9.4.x/jetty-security/src/main/java/org/eclipse/jetty/security/authentication/SpnegoAuthenticator.java)


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156671425
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    +      logger.debug("Logging out user {}", req.getRemoteAddr());
    +      session.removeAttribute("org.eclipse.jetty.security.UserIdentity");
    +      return null;
    +    } else if (authentication != null) { // Since already logged in just return the session attribute
    +      return authentication;
    +    } else { // The session is not yet authenticated
    +      final String header = req.getHeader(HttpHeader.AUTHORIZATION.asString());
    +      if (!mandatory) {
    +        return new DeferredAuthentication(this);
    +      } else 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("SPNEGOAuthenticator: Sending challenge to client {}", req.getRemoteAddr());
    --- End diff --
    
    SPNEGOAuthenticator -> DrillSpnegoAuthenticator


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158184891
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/TestSpnegoAuthentication.java ---
    @@ -0,0 +1,597 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.collect.Lists;
    +import com.typesafe.config.ConfigValueFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.common.scanner.ClassPathScanner;
    +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.AuthenticatorProviderImpl;
    +import org.apache.drill.exec.rpc.security.KerberosHelper;
    +import org.apache.drill.exec.rpc.security.plain.PlainFactory;
    +import org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
    +import org.apache.drill.exec.server.rest.WebServerConstants;
    +import org.apache.drill.exec.server.rest.auth.DrillHttpSecurityHandlerProvider;
    +import org.apache.drill.exec.server.rest.auth.DrillSpnegoAuthenticator;
    +import org.apache.drill.exec.server.rest.auth.DrillSpnegoLoginService;
    +import org.apache.drill.exec.server.rest.auth.SpnegoUtil;
    +import org.apache.drill.test.BaseDirTestWatcher;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.hadoop.security.authentication.util.KerberosName;
    +import org.apache.hadoop.security.authentication.util.KerberosUtil;
    +import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
    +import org.eclipse.jetty.http.HttpHeader;
    +import org.eclipse.jetty.security.Authenticator;
    +import org.eclipse.jetty.security.DefaultIdentityService;
    +import org.eclipse.jetty.security.UserAuthentication;
    +import org.eclipse.jetty.server.Authentication;
    +import org.eclipse.jetty.server.UserIdentity;
    +import org.ietf.jgss.GSSContext;
    +import org.ietf.jgss.GSSManager;
    +import org.ietf.jgss.GSSName;
    +import org.ietf.jgss.Oid;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +import org.mockito.Mockito;
    +import sun.security.jgss.GSSUtil;
    +
    +import javax.security.auth.Subject;
    +import javax.servlet.http.HttpServletRequest;
    +import javax.servlet.http.HttpServletResponse;
    +import javax.servlet.http.HttpSession;
    +import java.lang.reflect.Field;
    +import java.security.PrivilegedExceptionAction;
    +
    +import static junit.framework.TestCase.fail;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Mockito.never;
    +import static org.mockito.Mockito.verify;
    +
    +/**
    + * Test {@link SpnegoUtil}, {@link DrillSpnegoAuthenticator} and {@link DrillSpnegoLoginService}
    + */
    +public class TestSpnegoAuthentication {
    +
    +  private static KerberosHelper spnegoHelper;
    +
    +  private static final String primaryName = "HTTP";
    +
    +  private static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
    +
    +
    +  @BeforeClass
    +  public static void setupTest() throws Exception {
    +    spnegoHelper = new KerberosHelper(TestSpnegoAuthentication.class.getSimpleName(), primaryName);
    +    spnegoHelper.setupKdc(dirTestWatcher.getTmpDir());
    +
    +
    +    sun.security.krb5.Config.refresh();
    +
    +    // (2) Reset the default realm.
    +    final Field defaultRealm = KerberosName.class.getDeclaredField("defaultRealm");
    +    defaultRealm.setAccessible(true);
    +    defaultRealm.set(null, KerberosUtil.getDefaultRealm());
    +  }
    +
    +  /**
    +   * Both SPNEGO and FORM mechanism is enabled for WebServer in configuration. Test to see if the respective security
    +   * handlers are created successfully or not.
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testSPNEGOAndFORMEnabled() throws Exception {
    +
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +            ConfigValueFactory.fromIterable(Lists.newArrayList("form", "spnego")))
    +        .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +        .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +        false);
    +
    +    final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +    final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +    Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(true);
    +
    +    final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +    Mockito.when(context.getConfig()).thenReturn(newConfig);
    +    Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +    final DrillHttpSecurityHandlerProvider securityProvider = new DrillHttpSecurityHandlerProvider(newConfig, context);
    +    assertTrue(securityProvider.isFormEnabled());
    +    assertTrue(securityProvider.isSpnegoEnabled());
    +  }
    +
    +  /**
    +   * Validate if FORM security handler is created successfully when only form is configured as auth mechanism
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testOnlyFORMEnabled() throws Exception {
    +
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +            ConfigValueFactory.fromIterable(Lists.newArrayList("form")))
    +        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +        .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +        false);
    +
    +    final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +    final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +    Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(true);
    +
    +    final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +    Mockito.when(context.getConfig()).thenReturn(newConfig);
    +    Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +    final DrillHttpSecurityHandlerProvider securityProvider = new DrillHttpSecurityHandlerProvider(newConfig, context);
    +    assertTrue(securityProvider.isFormEnabled());
    +    assertTrue(!securityProvider.isSpnegoEnabled());
    +  }
    +
    +  /**
    +   * Validate failure in creating FORM security handler when PAM authenticator is absent. PAM authenticator is provided
    +   * via {@link PlainFactory#getAuthenticator()}
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testFORMEnabledWithPlainDisabled() throws Exception {
    +    try {
    +      final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +          .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +              ConfigValueFactory.fromAnyRef(true))
    +          .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +              ConfigValueFactory.fromIterable(Lists.newArrayList("form")))
    +          .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +              ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +          .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +              ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +          false);
    +
    +      final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +      final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +      Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(false);
    +
    +      final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +      Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +      Mockito.when(context.getConfig()).thenReturn(newConfig);
    +      Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +      final DrillHttpSecurityHandlerProvider securityProvider =
    +          new DrillHttpSecurityHandlerProvider(newConfig, context);
    +      fail();
    +    } catch(Exception ex) {
    +      assertTrue(ex instanceof DrillbitStartupException);
    +    }
    +  }
    +
    +  /**
    +   * Validate only SPNEGO security handler is configured properly when enabled via configuration
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testOnlySPNEGOEnabled() throws Exception {
    +
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +            ConfigValueFactory.fromIterable(Lists.newArrayList("spnego")))
    +        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +        .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +        false);
    +
    +    final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +    final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +    Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(false);
    +
    +    final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +    Mockito.when(context.getConfig()).thenReturn(newConfig);
    +    Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +    final DrillHttpSecurityHandlerProvider securityProvider = new DrillHttpSecurityHandlerProvider(newConfig, context);
    +
    +    assertTrue(!securityProvider.isFormEnabled());
    +    assertTrue(securityProvider.isSpnegoEnabled());
    +  }
    +
    +  /**
    +   * Validate when none of the security mechanism is specified in the
    +   * {@link ExecConstants#HTTP_AUTHENTICATION_MECHANISMS}, FORM security handler is still configured correctly when
    +   * authentication is enabled along with PAM authenticator module.
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testConfigBackwardCompatibility() throws Exception {
    +
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true)),
    +        false);
    +
    +    final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +    final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +    Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(true);
    +
    +    final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +    Mockito.when(context.getConfig()).thenReturn(newConfig);
    +    Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +    final DrillHttpSecurityHandlerProvider securityProvider = new DrillHttpSecurityHandlerProvider(newConfig, context);
    +
    +    assertTrue(securityProvider.isFormEnabled());
    +    assertTrue(!securityProvider.isSpnegoEnabled());
    +  }
    +
    +  /**
    +   * Validate behavior of {@link SpnegoUtil} class when provided with different configuration's for SPNEGO via
    +   * DrillConfig
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testSpnegoUtil() throws Exception {
    --- End diff --
    
    I kept all the tests in same file to avoid the need for setup again. But have now moved into separate class files as per your feedback.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158113700
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158112439
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    +
    +  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(FORMSecurityHanlder.HANDLER_NAME);
    +    }
    +
    +      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 | IllegalAccessException |
    +            InstantiationException | InvocationTargetException | 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("org.eclipse.jetty.security.UserIdentity");
    +    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 request will enforce going via /spnegoLogin before authentication is
    --- End diff --
    
    Done.


---

[GitHub] drill issue #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEGO

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on the issue:

    https://github.com/apache/drill/pull/1040
  
    @sohami thanks for making the changes. +1, LGTM.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158113430
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    +
    +  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(FORMSecurityHanlder.HANDLER_NAME);
    +    }
    +
    +      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 | IllegalAccessException |
    +            InstantiationException | InvocationTargetException | 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("org.eclipse.jetty.security.UserIdentity");
    +    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 request will enforce going via /spnegoLogin before authentication is
    +    // done, this is to ensure we don't have to authenticate again and again 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 ((!securityHandlers.containsKey(FORMSecurityHanlder.HANDLER_NAME)) ||
    +          (securityHandlers.containsKey(SPNEGOSecurityHandler.HANDLER_NAME) &&
    +              uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH))) {
    +        securityHandler = securityHandlers.get(SPNEGOSecurityHandler.HANDLER_NAME);
    +        securityHandler.handle(target, baseRequest, request, response);
    +      } else if (securityHandlers.containsKey(FORMSecurityHanlder.HANDLER_NAME)) {
    +        securityHandler = securityHandlers.get(FORMSecurityHanlder.HANDLER_NAME);
    +        securityHandler.handle(target, baseRequest, request, response);
    +      }
    +    }
    --- End diff --
    
    The suggested change doesn't handle the condition 1) in the comment. I have tried to simplify the if condition.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158111965
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    --- End diff --
    
    I just chose a little higher number than 2 to avoid resizing of hashtable (based on default loadfactor of 0.75), but looked into the implementation and seems like initial size of 2 will also work since it will allocate 4 space for it.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158106496
  
    --- Diff: 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 handler -
    --- End diff --
    
    Changed


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158117082
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoLoginService.java ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/**
    + * 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 SpnegoUtil spnegoUtil;
    +
    +  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
    +    spnegoUtil = new SpnegoUtil(drillBitContext.getConfig());
    +    spnegoUtil.validateSpnegoConfig();
    +    loggedInUgi = spnegoUtil.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, spnegoUtil.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(username, credentials);
    +        }
    +      });
    +    } catch (Exception e) {
    +      logger.error("Failed to login using SPNEGO");
    --- End diff --
    
    Done.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158135735
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoUtil.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.CommonConfigurationKeys;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +public class SpnegoUtil {
    +
    +  private UserGroupInformation loggedInUgi;
    +
    +  //private String realm;
    +
    +  private String principal;
    +
    +  private String keytab;
    +
    +  public SpnegoUtil(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());
    +  }
    +
    +  //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));
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158117860
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoLoginService.java ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/**
    + * 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 SpnegoUtil spnegoUtil;
    +
    +  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
    +    spnegoUtil = new SpnegoUtil(drillBitContext.getConfig());
    +    spnegoUtil.validateSpnegoConfig();
    +    loggedInUgi = spnegoUtil.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, spnegoUtil.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(username, credentials);
    +        }
    +      });
    +    } catch (Exception e) {
    +      logger.error("Failed to login using SPNEGO");
    +    }
    +
    +    return identity;
    +  }
    +
    +  private UserIdentity spnegoLogin(String username, Object credentials) {
    --- End diff --
    
    Removed.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154902705
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoUtil.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.CommonConfigurationKeys;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +public class SpnegoUtil {
    +
    +  private UserGroupInformation loggedInUgi;
    +
    +  //private String realm;
    +
    +  private String principal;
    +
    +  private String keytab;
    +
    +  public SpnegoUtil(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());
    +  }
    +
    +  //Performs the Server login to KDC for SPNEGO
    +  private UserGroupInformation loginAndReturnUgi() throws DrillException {
    --- End diff --
    
    Usually I put private methods in the end of the class.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158348706
  
    --- Diff: 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)) {
    --- End diff --
    
    There is a reason for doing it like this for handling a special case which is when server send 401 error code (standard HTTP error page) along with Negotiate header to client and expect client to respond back with SPNEGO token. In case when client is not configured for SPNEGO it will not respond and server will not get chance to reply back with an error page. In this scenario we wanted to provide a message to client on what to do next and hence the special handling.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158118993
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FORMSecurityHanlder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +public class FORMSecurityHanlder extends DrillHttpConstraintSecurityHandler {
    +  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FORMSecurityHanlder.class);
    +
    +  public static final String HANDLER_NAME = "FORM";
    +
    +  @Override
    +  public String getImplName() {
    +    return HANDLER_NAME;
    --- End diff --
    
    Didn't used it earlier since it was not used consistently by Jetty itself. See `org.eclipse.jetty.security.authentication.FormAuthenticator::getAuthMethod()`


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154902468
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoUtil.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.CommonConfigurationKeys;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +public class SpnegoUtil {
    +
    +  private UserGroupInformation loggedInUgi;
    +
    +  //private String realm;
    +
    +  private String principal;
    +
    +  private String keytab;
    +
    +  public SpnegoUtil(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());
    +  }
    +
    +  //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));
    --- End diff --
    
    Please add original exception to `DrillException` as well to have full stacktrace.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154902157
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoUtil.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.CommonConfigurationKeys;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +public class SpnegoUtil {
    +
    +  private UserGroupInformation loggedInUgi;
    +
    +  //private String realm;
    --- End diff --
    
    Please remove.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158184949
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/TestSpnegoAuthentication.java ---
    @@ -0,0 +1,597 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.collect.Lists;
    +import com.typesafe.config.ConfigValueFactory;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.common.scanner.ClassPathScanner;
    +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.AuthenticatorProviderImpl;
    +import org.apache.drill.exec.rpc.security.KerberosHelper;
    +import org.apache.drill.exec.rpc.security.plain.PlainFactory;
    +import org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
    +import org.apache.drill.exec.server.rest.WebServerConstants;
    +import org.apache.drill.exec.server.rest.auth.DrillHttpSecurityHandlerProvider;
    +import org.apache.drill.exec.server.rest.auth.DrillSpnegoAuthenticator;
    +import org.apache.drill.exec.server.rest.auth.DrillSpnegoLoginService;
    +import org.apache.drill.exec.server.rest.auth.SpnegoUtil;
    +import org.apache.drill.test.BaseDirTestWatcher;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.hadoop.security.authentication.util.KerberosName;
    +import org.apache.hadoop.security.authentication.util.KerberosUtil;
    +import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
    +import org.eclipse.jetty.http.HttpHeader;
    +import org.eclipse.jetty.security.Authenticator;
    +import org.eclipse.jetty.security.DefaultIdentityService;
    +import org.eclipse.jetty.security.UserAuthentication;
    +import org.eclipse.jetty.server.Authentication;
    +import org.eclipse.jetty.server.UserIdentity;
    +import org.ietf.jgss.GSSContext;
    +import org.ietf.jgss.GSSManager;
    +import org.ietf.jgss.GSSName;
    +import org.ietf.jgss.Oid;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +import org.mockito.Mockito;
    +import sun.security.jgss.GSSUtil;
    +
    +import javax.security.auth.Subject;
    +import javax.servlet.http.HttpServletRequest;
    +import javax.servlet.http.HttpServletResponse;
    +import javax.servlet.http.HttpSession;
    +import java.lang.reflect.Field;
    +import java.security.PrivilegedExceptionAction;
    +
    +import static junit.framework.TestCase.fail;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Mockito.never;
    +import static org.mockito.Mockito.verify;
    +
    +/**
    + * Test {@link SpnegoUtil}, {@link DrillSpnegoAuthenticator} and {@link DrillSpnegoLoginService}
    + */
    +public class TestSpnegoAuthentication {
    +
    +  private static KerberosHelper spnegoHelper;
    +
    +  private static final String primaryName = "HTTP";
    +
    +  private static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
    +
    +
    +  @BeforeClass
    +  public static void setupTest() throws Exception {
    +    spnegoHelper = new KerberosHelper(TestSpnegoAuthentication.class.getSimpleName(), primaryName);
    +    spnegoHelper.setupKdc(dirTestWatcher.getTmpDir());
    +
    +
    +    sun.security.krb5.Config.refresh();
    +
    +    // (2) Reset the default realm.
    +    final Field defaultRealm = KerberosName.class.getDeclaredField("defaultRealm");
    +    defaultRealm.setAccessible(true);
    +    defaultRealm.set(null, KerberosUtil.getDefaultRealm());
    +  }
    +
    +  /**
    +   * Both SPNEGO and FORM mechanism is enabled for WebServer in configuration. Test to see if the respective security
    +   * handlers are created successfully or not.
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testSPNEGOAndFORMEnabled() throws Exception {
    +
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +            ConfigValueFactory.fromIterable(Lists.newArrayList("form", "spnego")))
    +        .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +        .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +        false);
    +
    +    final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +    final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +    Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(true);
    +
    +    final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +    Mockito.when(context.getConfig()).thenReturn(newConfig);
    +    Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +    final DrillHttpSecurityHandlerProvider securityProvider = new DrillHttpSecurityHandlerProvider(newConfig, context);
    +    assertTrue(securityProvider.isFormEnabled());
    +    assertTrue(securityProvider.isSpnegoEnabled());
    +  }
    +
    +  /**
    +   * Validate if FORM security handler is created successfully when only form is configured as auth mechanism
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testOnlyFORMEnabled() throws Exception {
    +
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +            ConfigValueFactory.fromIterable(Lists.newArrayList("form")))
    +        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +        .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +        false);
    +
    +    final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +    final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +    Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(true);
    +
    +    final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +    Mockito.when(context.getConfig()).thenReturn(newConfig);
    +    Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +    final DrillHttpSecurityHandlerProvider securityProvider = new DrillHttpSecurityHandlerProvider(newConfig, context);
    +    assertTrue(securityProvider.isFormEnabled());
    +    assertTrue(!securityProvider.isSpnegoEnabled());
    +  }
    +
    +  /**
    +   * Validate failure in creating FORM security handler when PAM authenticator is absent. PAM authenticator is provided
    +   * via {@link PlainFactory#getAuthenticator()}
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testFORMEnabledWithPlainDisabled() throws Exception {
    +    try {
    +      final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +          .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +              ConfigValueFactory.fromAnyRef(true))
    +          .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +              ConfigValueFactory.fromIterable(Lists.newArrayList("form")))
    +          .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +              ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +          .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +              ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +          false);
    +
    +      final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +      final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +      Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(false);
    +
    +      final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +      Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +      Mockito.when(context.getConfig()).thenReturn(newConfig);
    +      Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +      final DrillHttpSecurityHandlerProvider securityProvider =
    +          new DrillHttpSecurityHandlerProvider(newConfig, context);
    +      fail();
    +    } catch(Exception ex) {
    +      assertTrue(ex instanceof DrillbitStartupException);
    +    }
    +  }
    +
    +  /**
    +   * Validate only SPNEGO security handler is configured properly when enabled via configuration
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testOnlySPNEGOEnabled() throws Exception {
    +
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +            ConfigValueFactory.fromIterable(Lists.newArrayList("spnego")))
    +        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +        .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +        false);
    +
    +    final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +    final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +    Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(false);
    +
    +    final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +    Mockito.when(context.getConfig()).thenReturn(newConfig);
    +    Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +    final DrillHttpSecurityHandlerProvider securityProvider = new DrillHttpSecurityHandlerProvider(newConfig, context);
    +
    +    assertTrue(!securityProvider.isFormEnabled());
    +    assertTrue(securityProvider.isSpnegoEnabled());
    +  }
    +
    +  /**
    +   * Validate when none of the security mechanism is specified in the
    +   * {@link ExecConstants#HTTP_AUTHENTICATION_MECHANISMS}, FORM security handler is still configured correctly when
    +   * authentication is enabled along with PAM authenticator module.
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testConfigBackwardCompatibility() throws Exception {
    +
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true)),
    +        false);
    +
    +    final ScanResult scanResult = ClassPathScanner.fromPrescan(newConfig);
    +    final AuthenticatorProviderImpl authenticatorProvider = Mockito.mock(AuthenticatorProviderImpl.class);
    +    Mockito.when(authenticatorProvider.containsFactory(PlainFactory.SIMPLE_NAME)).thenReturn(true);
    +
    +    final DrillbitContext context = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(context.getClasspathScan()).thenReturn(scanResult);
    +    Mockito.when(context.getConfig()).thenReturn(newConfig);
    +    Mockito.when(context.getAuthProvider()).thenReturn(authenticatorProvider);
    +
    +    final DrillHttpSecurityHandlerProvider securityProvider = new DrillHttpSecurityHandlerProvider(newConfig, context);
    +
    +    assertTrue(securityProvider.isFormEnabled());
    +    assertTrue(!securityProvider.isSpnegoEnabled());
    +  }
    +
    +  /**
    +   * Validate behavior of {@link SpnegoUtil} class when provided with different configuration's for SPNEGO via
    +   * DrillConfig
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testSpnegoUtil() throws Exception {
    +
    +    DrillConfig newConfig;
    +    SpnegoUtil spnegoUtil;
    +
    +    // Invalid configuration for SPNEGO
    +    try {
    +      newConfig = new DrillConfig(DrillConfig.create()
    +          .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +              ConfigValueFactory.fromAnyRef(true))
    +          .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
    +              ConfigValueFactory.fromIterable(Lists.newArrayList("plain")))
    +          .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
    +              ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE)),
    +          false);
    +
    +      spnegoUtil = new SpnegoUtil(newConfig);
    +      spnegoUtil.validateSpnegoConfig();
    +      fail();
    +    } catch (Exception ex) {
    +      assertTrue(ex instanceof DrillException);
    +    }
    +
    +    // Configuration with keytab only
    +    try {
    +      newConfig = new DrillConfig(DrillConfig.create()
    +          .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +              ConfigValueFactory.fromAnyRef(true))
    +          .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
    +              ConfigValueFactory.fromIterable(Lists.newArrayList("plain")))
    +          .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +              ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString()))
    +          .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
    +              ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE)),
    +          false);
    +
    +      spnegoUtil = new SpnegoUtil(newConfig);
    +      spnegoUtil.validateSpnegoConfig();
    +      fail();
    +    } catch (Exception ex) {
    +      assertTrue(ex instanceof DrillException);
    +    }
    +
    +    // Configuration with principal only
    +    try {
    +      newConfig = new DrillConfig(DrillConfig.create()
    +          .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +              ConfigValueFactory.fromAnyRef(true))
    +          .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
    +              ConfigValueFactory.fromIterable(Lists.newArrayList("plain")))
    +          .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +              ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +          .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
    +              ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE)),
    +          false);
    +
    +      spnegoUtil = new SpnegoUtil(newConfig);
    +      spnegoUtil.validateSpnegoConfig();
    +      fail();
    +    } catch (Exception ex) {
    +      assertTrue(ex instanceof DrillException);
    +    }
    +
    +    // Valid Configuration with both keytab & principal
    +    try {
    +      newConfig = new DrillConfig(DrillConfig.create()
    +          .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
    +              ConfigValueFactory.fromAnyRef(true))
    +          .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
    +              ConfigValueFactory.fromIterable(Lists.newArrayList("plain")))
    +          .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +              ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +          .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +              ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString()))
    +          .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
    +              ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE)),
    +          false);
    +
    +      spnegoUtil = new SpnegoUtil(newConfig);
    +      spnegoUtil.validateSpnegoConfig();
    +      UserGroupInformation ugi = spnegoUtil.getLoggedInUgi();
    +      assertEquals(primaryName, ugi.getShortUserName());
    +      assertEquals(spnegoHelper.SERVER_PRINCIPAL, ugi.getUserName());
    +    } catch (Exception ex) {
    +      fail();
    +    }
    +  }
    +
    +  /**
    +   * Validate successful {@link DrillSpnegoLoginService#login(String, Object)} when provided with client token for a
    +   * configured service principal.
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testDrillSpnegoLoginService() throws Exception {
    +
    +    // Create client subject using it's principal and keytab
    +    final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(spnegoHelper.CLIENT_PRINCIPAL,
    +            spnegoHelper.clientKeytab.getAbsoluteFile());
    +
    +    // Generate a SPNEGO token for the peer SERVER_PRINCIPAL from this CLIENT_PRINCIPAL
    +    final String token = Subject.doAs(clientSubject, new PrivilegedExceptionAction<String>() {
    +      @Override
    +      public String run() throws Exception {
    +
    +        final GSSManager gssManager = GSSManager.getInstance();
    +        GSSContext gssContext = null;
    +        try {
    +          final Oid oid = GSSUtil.GSS_SPNEGO_MECH_OID;
    +          final GSSName serviceName = gssManager.createName(spnegoHelper.SERVER_PRINCIPAL, GSSName.NT_USER_NAME, oid);
    +
    +          gssContext = gssManager.createContext(serviceName, oid, null, GSSContext.DEFAULT_LIFETIME);
    +          gssContext.requestCredDeleg(true);
    +          gssContext.requestMutualAuth(true);
    +
    +          byte[] outToken = new byte[0];
    +          outToken = gssContext.initSecContext(outToken, 0, outToken.length);
    +          return Base64.encodeBase64String(outToken);
    +
    +        } finally {
    +          if (gssContext != null) {
    +            gssContext.dispose();
    +          }
    +        }
    +      }
    +    });
    +
    +    // Create a DrillbitContext with service principal and keytab for DrillSpnegoLoginService
    +    final DrillConfig newConfig = new DrillConfig(DrillConfig.create()
    +        .withValue(ExecConstants.HTTP_AUTHENTICATION_MECHANISMS,
    +            ConfigValueFactory.fromIterable(Lists.newArrayList("spnego")))
    +        .withValue(ExecConstants.HTTP_SPNEGO_PRINCIPAL,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.SERVER_PRINCIPAL))
    +        .withValue(ExecConstants.HTTP_SPNEGO_KEYTAB,
    +            ConfigValueFactory.fromAnyRef(spnegoHelper.serverKeytab.toString())),
    +        false);
    +
    +
    +    final SystemOptionManager optionManager = Mockito.mock(SystemOptionManager.class);
    +    Mockito.when(optionManager.getOption(ExecConstants.ADMIN_USERS_VALIDATOR))
    +        .thenReturn(ExecConstants.ADMIN_USERS_VALIDATOR.DEFAULT_ADMIN_USERS);
    +    Mockito.when(optionManager.getOption(ExecConstants.ADMIN_USER_GROUPS_VALIDATOR))
    +        .thenReturn(ExecConstants.ADMIN_USER_GROUPS_VALIDATOR.DEFAULT_ADMIN_USER_GROUPS);
    +
    +    final DrillbitContext drillbitContext = Mockito.mock(DrillbitContext.class);
    +    Mockito.when(drillbitContext.getConfig()).thenReturn(newConfig);
    +    Mockito.when(drillbitContext.getOptionManager()).thenReturn(optionManager);
    +
    +    final DrillSpnegoLoginService loginService = new DrillSpnegoLoginService(drillbitContext);
    +
    +    // Authenticate the client using its SPNEGO token
    +    final UserIdentity user = loginService.login(null, token);
    +
    +    // Validate the UserIdentity of authenticated client
    +    assertTrue(user != null);
    +    assertTrue(user.getUserPrincipal().getName().equals(spnegoHelper.CLIENT_PRINCIPAL));
    +    assertTrue(user.isUserInRole("authenticated", null));
    +  }
    +
    +  /**
    +   * Validate {@link DrillSpnegoAuthenticator} with request of different nature and from authenticated and
    +   * unauthenticated session.
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testDrillSpnegoAuthenticator() throws Exception {
    --- End diff --
    
    Same as above.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158118536
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FORMSecurityHanlder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +public class FORMSecurityHanlder extends DrillHttpConstraintSecurityHandler {
    --- End diff --
    
    Changed.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156658831
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    +
    +  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(FORMSecurityHanlder.HANDLER_NAME);
    +    }
    +
    +      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 | IllegalAccessException |
    +            InstantiationException | InvocationTargetException | 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("org.eclipse.jetty.security.UserIdentity");
    +    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 request will enforce going via /spnegoLogin before authentication is
    +    // done, this is to ensure we don't have to authenticate again and again 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 ((!securityHandlers.containsKey(FORMSecurityHanlder.HANDLER_NAME)) ||
    +          (securityHandlers.containsKey(SPNEGOSecurityHandler.HANDLER_NAME) &&
    +              uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH))) {
    +        securityHandler = securityHandlers.get(SPNEGOSecurityHandler.HANDLER_NAME);
    +        securityHandler.handle(target, baseRequest, request, response);
    +      } else if (securityHandlers.containsKey(FORMSecurityHanlder.HANDLER_NAME)) {
    +        securityHandler = securityHandlers.get(FORMSecurityHanlder.HANDLER_NAME);
    +        securityHandler.handle(target, baseRequest, request, response);
    +      }
    +    }
    --- End diff --
    
    I am not sure that the above `if` implements condition described in the comment.
    I think the below code will be more correct:
    ```
    if (isSpnego && isSpnegoUri) {
    // use spnego
    } else if (isForm) {
    // use form
    }
    ```
    



---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156668713
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    --- End diff --
    
    This `if` has many nested if-s, else-ifs which may not be needs since we have return statements there and overall looks too complex and hard to read. Please consider splitting it for standalone if-s or methods.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156648355
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    +
    +  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(FORMSecurityHanlder.HANDLER_NAME);
    +    }
    +
    +      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 | IllegalAccessException |
    +            InstantiationException | InvocationTargetException | DrillException e) {
    --- End diff --
    
    Most of these exceptions can be replaced with their parent: `ReflectiveOperationException`.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158137533
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    +      logger.debug("Logging out user {}", req.getRemoteAddr());
    +      session.removeAttribute("org.eclipse.jetty.security.UserIdentity");
    +      return null;
    +    } else if (authentication != null) { // Since already logged in just return the session attribute
    +      return authentication;
    +    } else { // The session is not yet authenticated
    +      final String header = req.getHeader(HttpHeader.AUTHORIZATION.asString());
    +      if (!mandatory) {
    +        return new DeferredAuthentication(this);
    +      } else 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("SPNEGOAuthenticator: Sending challenge to client {}", req.getRemoteAddr());
    --- End diff --
    
    Changed in all the places.


---

[GitHub] drill issue #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on the issue:

    https://github.com/apache/drill/pull/1040
  
    @sohami could you please update pull request header to include Jira title? It would be just easier to identify it among others. Thanks.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156664691
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    --- End diff --
    
    1. I suggest we create local variable instead of overwriting given `mandatory` parameter.
    2. Do we need bit wise operator here?


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158121657
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java ---
    @@ -202,6 +202,9 @@ public int bind(final int initialPort, boolean allowPortHunting) {
             if (e instanceof BindException && allowPortHunting) {
               continue;
             }
    +
    +        e.printStackTrace();
    --- End diff --
    
    Not sure how this got introduced. Thanks for catching.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158112289
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    +
    +  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(FORMSecurityHanlder.HANDLER_NAME);
    +    }
    +
    +      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 | IllegalAccessException |
    +            InstantiationException | InvocationTargetException | DrillException e) {
    --- End diff --
    
    Replaced.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156660683
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FORMSecurityHanlder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +public class FORMSecurityHanlder extends DrillHttpConstraintSecurityHandler {
    +  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FORMSecurityHanlder.class);
    +
    +  public static final String HANDLER_NAME = "FORM";
    +
    +  @Override
    +  public String getImplName() {
    +    return HANDLER_NAME;
    --- End diff --
    
    You can use `org.eclipse.jetty.util.security.Constraint.__FORM_AUTH` instead.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158100206
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    +      return null;
    +    }
    +
    +    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
    +    MainLoginPageModel model = new MainLoginPageModel(errorString);
    +    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 {
    +    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.
    +      final HttpSession session = request.getSession(true);
    +      final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
    +      session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
    +    }
    +
    +    MainLoginPageModel model = new MainLoginPageModel(null);
    +    return ViewableWithPermissions.createMainLoginPage(model);
    +  }
    +
    +  public class MainLoginPageModel {
    +
    +    public String error;
    +
    +    MainLoginPageModel(String error) {
    +      this.error = error;
    +    }
    +
    +    public boolean isSpnegoEnabled() {
    +      return LogInLogOutResources.isSpnegoEnabled(workManager.getContext().getConfig());
    +    }
    +
    +    public boolean isFormEnabled() {
    +      return LogInLogOutResources.isFormEnabled(workManager.getContext().getConfig());
    +    }
    +
    +    public String getError() {
    +      return error;
    +    }
    +  }
    +
    +  private static boolean isSpnegoEnabled(DrillConfig config) {
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158099889
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    +      return null;
    +    }
    +
    +    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
    +    MainLoginPageModel model = new MainLoginPageModel(errorString);
    +    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 {
    +    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.
    +      final HttpSession session = request.getSession(true);
    +      final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
    +      session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
    +    }
    +
    +    MainLoginPageModel model = new MainLoginPageModel(null);
    +    return ViewableWithPermissions.createMainLoginPage(model);
    +  }
    +
    +  public class MainLoginPageModel {
    +
    +    public String error;
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154691438
  
    --- Diff: 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)) {
    --- End diff --
    
    We need to redirect user to error ftl rather then constructing formatting inside of Java. Formatting should be done on freemarker side.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156662613
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    --- End diff --
    
    also trigger -> also triggers


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154901206
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java ---
    @@ -202,6 +202,9 @@ public int bind(final int initialPort, boolean allowPortHunting) {
             if (e instanceof BindException && allowPortHunting) {
               continue;
             }
    +
    +        e.printStackTrace();
    --- End diff --
    
    Please remove.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158120686
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SPNEGOSecurityHandler.java ---
    @@ -0,0 +1,37 @@
    +/*
    + * 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;
    +
    +public class SPNEGOSecurityHandler extends DrillHttpConstraintSecurityHandler {
    +  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SPNEGOSecurityHandler.class);
    +
    +  public static final String HANDLER_NAME = "SPNEGO";
    +
    +  @Override
    +  public String getImplName() {
    +    return HANDLER_NAME;
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156673206
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    +      logger.debug("Logging out user {}", req.getRemoteAddr());
    +      session.removeAttribute("org.eclipse.jetty.security.UserIdentity");
    --- End diff --
    
    Can be replaced with `org.eclipse.jetty.security.authentication.SessionAuthentication,__J_AUTHENTICATED`


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/1040


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156642213
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    +      return null;
    +    }
    +
    +    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
    +    MainLoginPageModel model = new MainLoginPageModel(errorString);
    +    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 {
    +    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.
    +      final HttpSession session = request.getSession(true);
    +      final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
    +      session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
    +    }
    --- End diff --
    
    This redirect logic is the same as in `getLoginPage`, please factor out it in separate private method.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154690645
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServerConstants.java ---
    @@ -0,0 +1,44 @@
    +/*
    + * 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 class WebServerConstants {
    +  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WebServerConstants.class);
    --- End diff --
    
    1. Please remove logger.
    2. Usually interface is used to hold constants.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158120552
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SPNEGOSecurityHandler.java ---
    @@ -0,0 +1,37 @@
    +/*
    + * 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;
    +
    +public class SPNEGOSecurityHandler extends DrillHttpConstraintSecurityHandler {
    --- End diff --
    
    Renamed.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156645220
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java ---
    @@ -183,17 +180,19 @@ public void start() throws Exception {
         }
       }
     
    -  private ServletContextHandler createServletContextHandler(final boolean authEnabled) {
    +  private ServletContextHandler createServletContextHandler(final boolean authEnabled) throws Exception {
    --- End diff --
    
    `DrillHttpSecurityHandlerProvider` constructor throws `DrillbitStartupException`, it seems better to re-throw exact exception rather then broad `Exception`.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154896548
  
    --- Diff: 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 handler -
    --- End diff --
    
    2 security handlers (plural)


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156661626
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FORMSecurityHanlder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +public class FORMSecurityHanlder extends DrillHttpConstraintSecurityHandler {
    --- End diff --
    
    I suggest we use initcap for form (jetty does the same, i.e. `FormAuthenticator`) -> `FormSecurityHandler`


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156642357
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    +      return null;
    +    }
    +
    +    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
    +    MainLoginPageModel model = new MainLoginPageModel(errorString);
    +    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 {
    +    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.
    +      final HttpSession session = request.getSession(true);
    +      final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
    +      session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
    +    }
    +
    +    MainLoginPageModel model = new MainLoginPageModel(null);
    +    return ViewableWithPermissions.createMainLoginPage(model);
    +  }
    +
    +  public class MainLoginPageModel {
    +
    +    public String error;
    --- End diff --
    
    private final?


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156642388
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    +      return null;
    +    }
    +
    +    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
    +    MainLoginPageModel model = new MainLoginPageModel(errorString);
    +    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 {
    +    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.
    +      final HttpSession session = request.getSession(true);
    +      final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
    +      session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
    +    }
    +
    +    MainLoginPageModel model = new MainLoginPageModel(null);
    +    return ViewableWithPermissions.createMainLoginPage(model);
    +  }
    +
    +  public class MainLoginPageModel {
    +
    +    public String error;
    +
    +    MainLoginPageModel(String error) {
    --- End diff --
    
    public?


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154688969
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    --- End diff --
    
    1. `getSpnegologin` -> `getSpnegoLogin`
    2. "/" -> replace with constant `WebServerConstants.FORM_LOGIN_RESOURCE_PATH`


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158166736
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/KerberosHelper.java ---
    @@ -43,16 +45,20 @@
       public String SERVER_PRINCIPAL;
       private final String testName;
     
    -  private File keytabDir;
    +  public File keytabDir;
    --- End diff --
    
    Have changed this back to private since change was not required.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154689865
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    +      return null;
    +    }
    +
    +    final String errorString = "Invalid SPNEGO credentials or SPNEGO is not configured";
    +    MainLoginPageModel model = new MainLoginPageModel(errorString);
    +    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 {
    +    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.
    +      final HttpSession session = request.getSession(true);
    +      final URI destURI = UriBuilder.fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
    +      session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
    +    }
    +
    +    MainLoginPageModel model = new MainLoginPageModel(null);
    +    return ViewableWithPermissions.createMainLoginPage(model);
    +  }
    +
    +  public class MainLoginPageModel {
    +
    +    public String error;
    +
    +    MainLoginPageModel(String error) {
    +      this.error = error;
    +    }
    +
    +    public boolean isSpnegoEnabled() {
    +      return LogInLogOutResources.isSpnegoEnabled(workManager.getContext().getConfig());
    +    }
    +
    +    public boolean isFormEnabled() {
    +      return LogInLogOutResources.isFormEnabled(workManager.getContext().getConfig());
    +    }
    +
    +    public String getError() {
    +      return error;
    +    }
    +  }
    +
    +  private static boolean isSpnegoEnabled(DrillConfig config) {
    --- End diff --
    
    I think rather then having two private static method and call them from inner class, you can move their logic directly into private class, and when creating `MainLoginPageModel` pass `DrillConfig config` into constructor.


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r154950842
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoUtil.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.CommonConfigurationKeys;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +public class SpnegoUtil {
    +
    +  private UserGroupInformation loggedInUgi;
    +
    +  //private String realm;
    +
    +  private String principal;
    --- End diff --
    
    final 


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158099695
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/LogInLogOutResources.java ---
    @@ -69,23 +79,94 @@ public Viewable getLoginPage(@Context HttpServletRequest request, @Context HttpS
         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("/").forward(request, response);
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156671926
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillSpnegoAuthenticator.java ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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 which provides following:
    + * 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 trigger the
    + *    SPNEGO authentication.
    + */
    +public class DrillSpnegoAuthenticator extends SpnegoAuthenticator {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSpnegoAuthenticator.class);
    +
    +  public DrillSpnegoAuthenticator(String authMethod) {
    +    super(authMethod);
    +  }
    +
    +  @Override
    +  public Authentication validateRequest(ServletRequest request, ServletResponse response, boolean mandatory)
    +      throws ServerAuthException {
    +
    +    HttpServletRequest req = (HttpServletRequest) request;
    +    HttpServletResponse res = (HttpServletResponse) response;
    +    HttpSession session = req.getSession(true);
    +    final Authentication authentication =
    +        (Authentication) session.getAttribute("org.eclipse.jetty.security.UserIdentity");
    +    String uri = req.getRequestURI();
    +
    +    //If the Request URI is for /spnegoLogin then perform login
    +    mandatory |= uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH);
    +
    +    //For logout remove the attribute from the session that holds UserIdentity
    +    if (authentication != null && uri.equals(WebServerConstants.LOGOUT_RESOURCE_PATH)) {
    +      logger.debug("Logging out user {}", req.getRemoteAddr());
    +      session.removeAttribute("org.eclipse.jetty.security.UserIdentity");
    +      return null;
    +    } else if (authentication != null) { // Since already logged in just return the session attribute
    +      return authentication;
    +    } else { // The session is not yet authenticated
    +      final String header = req.getHeader(HttpHeader.AUTHORIZATION.asString());
    +      if (!mandatory) {
    +        return new DeferredAuthentication(this);
    +      } else 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("SPNEGOAuthenticator: Sending challenge to client {}", req.getRemoteAddr());
    +            return Authentication.SEND_CONTINUE;
    +          }
    +        } catch (IOException var9) {
    +          throw new ServerAuthException(var9);
    +        }
    +      } else {
    +        logger.debug("SPNEGOAuthenticator: 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) {
    +              e.printStackTrace();
    +            }
    +
    +            logger.debug("SPNEGOAuthenticator: Successfully authenticated this client session: {}",
    +                user.getUserPrincipal().getName());
    +            return new UserAuthentication(this.getAuthMethod(), user);
    +          }
    +        }
    +        logger.debug("SPNEGOAuthenticator: Authentication failed for client session: {}", req.getRemoteAddr());
    --- End diff --
    
    SPNEGOAuthenticator -> DrillSpnegoAuthenticator


---

[GitHub] drill pull request #1040: Drill 5425

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r156652385
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * 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 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.lang.reflect.InvocationTargetException;
    +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(5);
    +
    +  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(FORMSecurityHanlder.HANDLER_NAME);
    +    }
    +
    +      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 | IllegalAccessException |
    +            InstantiationException | InvocationTargetException | 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("org.eclipse.jetty.security.UserIdentity");
    +    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 request will enforce going via /spnegoLogin before authentication is
    --- End diff --
    
    all requests (plural)


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158135221
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/SpnegoUtil.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.CommonConfigurationKeys;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +public class SpnegoUtil {
    +
    +  private UserGroupInformation loggedInUgi;
    +
    +  //private String realm;
    +
    +  private String principal;
    +
    +  private String keytab;
    +
    +  public SpnegoUtil(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());
    +  }
    +
    +  //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());
    --- End diff --
    
    There is no concurrency issue here since this will be called only once during WebServer setup time.


---

[GitHub] drill pull request #1040: DRILL-5425: Support HTTP Kerberos auth using SPNEG...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1040#discussion_r158102763
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServerConstants.java ---
    @@ -0,0 +1,44 @@
    +/*
    + * 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 class WebServerConstants {
    +  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WebServerConstants.class);
    --- End diff --
    
    1. Removed logger. 
    2. I prefer to use class since interface is mostly used for defining a service contract. If it's an interface then any class can implement it and will get access to all the constants which might not be required. I have changed the class to be final and made constructor private so that it can't be extended and instantiated by mistake.


---