You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ng...@apache.org on 2022/02/17 19:11:40 UTC

[hive] branch master updated: HIVE-25957: [Addendum to HIVE-25875] Fix password authentication with SAML (Yu-Wen Lai via Naveen Gangam)

This is an automated email from the ASF dual-hosted git repository.

ngangam pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new e82c355  HIVE-25957: [Addendum to HIVE-25875] Fix password authentication with SAML (Yu-Wen Lai via Naveen Gangam)
e82c355 is described below

commit e82c355644fdd29b57f8fa2aeca812d0bc951471
Author: Yu-Wen Lai <yu...@cloudera.com>
AuthorDate: Mon Feb 14 14:58:37 2022 -0800

    HIVE-25957: [Addendum to HIVE-25875] Fix password authentication with SAML (Yu-Wen Lai via Naveen Gangam)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   2 +-
 .../org/apache/hive/service/auth/AuthType.java     | 112 +++++++++++++++++++++
 .../hive/service/cli/thrift/ThriftHttpServlet.java |  59 +++++------
 .../org/apache/hive/service/auth/TestAuthType.java | 112 +++++++++++++++++++++
 .../service/cli/thrift/ThriftHttpServletTest.java  |   2 +-
 5 files changed, 253 insertions(+), 34 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index e0fcf3c..d8398b3 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -4125,7 +4125,7 @@ public class HiveConf extends Configuration {
         "          (Use with property hive.server2.custom.authentication.class)\n" +
         "  PAM: Pluggable authentication module\n" +
         "  NOSASL:  Raw transport\n" +
-        "  SAML2: SAML 2.0 compliant authentication. This is only supported in http transport mode."),
+        "  SAML: SAML 2.0 compliant authentication. This is only supported in http transport mode."),
     HIVE_SERVER2_TRUSTED_DOMAIN("hive.server2.trusted.domain", "",
         "Specifies the host or a domain to trust connections from. Authentication is skipped " +
         "for any connection coming from a host whose hostname ends with the value of this" +
diff --git a/service/src/java/org/apache/hive/service/auth/AuthType.java b/service/src/java/org/apache/hive/service/auth/AuthType.java
new file mode 100644
index 0000000..b0c5711
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/AuthType.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service.auth;
+
+import org.apache.commons.lang3.EnumUtils;
+
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+/**
+ * AuthType is used to parse and verify
+ * {@link org.apache.hadoop.hive.conf.HiveConf.ConfVars#HIVE_SERVER2_AUTHENTICATION}.
+ * Throws an exception if the config value is not allowed.
+ */
+public class AuthType {
+  static final Set<HiveAuthConstants.AuthTypes> PASSWORD_BASED_TYPES = new HashSet<>(Arrays.asList(
+      HiveAuthConstants.AuthTypes.LDAP, HiveAuthConstants.AuthTypes.CUSTOM, HiveAuthConstants.AuthTypes.PAM));
+  private final BitSet typeBits;
+
+  public AuthType(String authTypes) throws Exception {
+    typeBits = new BitSet();
+    parseTypes(authTypes);
+    verifyTypes(authTypes);
+  }
+
+  private void parseTypes(String authTypes) throws Exception {
+    String[] types = authTypes.split(",");
+    for (String type : types) {
+      if (!EnumUtils.isValidEnumIgnoreCase(HiveAuthConstants.AuthTypes.class, type)) {
+        throw new Exception(type + " is not a valid authentication type.");
+      }
+      typeBits.set(EnumUtils.getEnumIgnoreCase(HiveAuthConstants.AuthTypes.class, type).ordinal());
+    }
+  }
+
+  private void verifyTypes(String authTypes) throws Exception {
+    if (typeBits.cardinality() == 1) {
+      // single authentication type has no conflicts
+      return;
+    }
+    if (typeBits.get(HiveAuthConstants.AuthTypes.SAML.ordinal()) &&
+        !typeBits.get(HiveAuthConstants.AuthTypes.NOSASL.ordinal()) &&
+        !typeBits.get(HiveAuthConstants.AuthTypes.KERBEROS.ordinal()) &&
+        !typeBits.get(HiveAuthConstants.AuthTypes.NONE.ordinal()) &&
+        (!areAnyEnabled(PASSWORD_BASED_TYPES) || isExactlyOneEnabled(PASSWORD_BASED_TYPES))) {
+      // SAML can be enabled with another password based authentication types
+      return;
+    }
+    throw new Exception("The authentication types have conflicts: " + authTypes);
+  }
+
+  private boolean isExactlyOneEnabled(Collection<HiveAuthConstants.AuthTypes> types) {
+    boolean areAnyEnabled = false;
+    boolean areTwoEnabled = false;
+    Iterator<HiveAuthConstants.AuthTypes> it = types.iterator();
+    while (!areTwoEnabled && it.hasNext()) {
+      boolean isCurrentTypeEnabled = isEnabled(it.next());
+      areTwoEnabled = areAnyEnabled && isCurrentTypeEnabled;
+      areAnyEnabled |= isCurrentTypeEnabled;
+    }
+    return areAnyEnabled && !areTwoEnabled;
+  }
+
+  private boolean areAnyEnabled(Collection<HiveAuthConstants.AuthTypes> types) {
+    boolean areAnyEnabled = false;
+    Iterator<HiveAuthConstants.AuthTypes> it = types.iterator();
+    while (!areAnyEnabled && it.hasNext()) {
+      areAnyEnabled = isEnabled(it.next());
+    }
+    return areAnyEnabled;
+  }
+
+  public boolean isEnabled(HiveAuthConstants.AuthTypes type) {
+    return typeBits.get(type.ordinal());
+  }
+
+
+  public String getPasswordBasedAuthStr() {
+    if (isEnabled(HiveAuthConstants.AuthTypes.NOSASL)) {
+      return HiveAuthConstants.AuthTypes.NOSASL.getAuthName();
+    }
+    if (isEnabled(HiveAuthConstants.AuthTypes.NONE)) {
+      return HiveAuthConstants.AuthTypes.NONE.getAuthName();
+    }
+    for (HiveAuthConstants.AuthTypes type : PASSWORD_BASED_TYPES) {
+      if (isEnabled(type)) {
+        return type.getAuthName();
+      }
+    }
+    return "";
+  }
+}
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
index 61832f0..244bd3a 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.service.CookieSigner;
+import org.apache.hive.service.auth.AuthType;
 import org.apache.hive.service.auth.AuthenticationProviderFactory;
 import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods;
 import org.apache.hive.service.auth.HiveAuthConstants;
@@ -90,7 +91,7 @@ public class ThriftHttpServlet extends TServlet {
 
   private static final long serialVersionUID = 1L;
   public static final Logger LOG = LoggerFactory.getLogger(ThriftHttpServlet.class.getName());
-  private final String authType;
+  private final AuthType authType;
   private final UserGroupInformation serviceUGI;
   private final UserGroupInformation httpUGI;
   private final HiveConf hiveConf;
@@ -111,10 +112,10 @@ public class ThriftHttpServlet extends TServlet {
 
   public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory,
       String authType, UserGroupInformation serviceUGI, UserGroupInformation httpUGI,
-      HiveAuthFactory hiveAuthFactory, HiveConf hiveConf) {
+      HiveAuthFactory hiveAuthFactory, HiveConf hiveConf) throws Exception {
     super(processor, protocolFactory);
     this.hiveConf = hiveConf;
-    this.authType = authType;
+    this.authType = new AuthType(authType);
     this.serviceUGI = serviceUGI;
     this.httpUGI = httpUGI;
     this.hiveAuthFactory = hiveAuthFactory;
@@ -212,15 +213,15 @@ public class ThriftHttpServlet extends TServlet {
             } else {
               clientUserName = doKerberosAuth(request);
             }
-          } else if (HiveSamlUtils.isSamlAuthMode(authType)) {
+          } else if (authType.isEnabled(HiveAuthConstants.AuthTypes.SAML)) {
             // check if this request needs a SAML redirect
             String authHeader = request.getHeader(HttpAuthUtils.AUTHORIZATION);
             if ((authHeader == null || authHeader.isEmpty()) && needsRedirect(request, response)) {
               doSamlRedirect(request, response);
               return;
             } else if(authHeader.toLowerCase().startsWith(HttpAuthUtils.BASIC.toLowerCase())) {
-              //LDAP Authentication if the header starts with Basic
-              clientUserName = doPasswdAuth(request, HiveAuthConstants.AuthTypes.NONE.toString());
+              // fall back to password based authentication if the header starts with Basic
+              clientUserName = doPasswdAuth(request, authType.getPasswordBasedAuthStr());
             } else {
               // redirect is not needed. Do SAML auth.
               clientUserName = doSamlAuth(request, response);
@@ -228,10 +229,10 @@ public class ThriftHttpServlet extends TServlet {
           } else {
             String proxyHeader = HiveConf.getVar(hiveConf, ConfVars.HIVE_SERVER2_TRUSTED_PROXY_TRUSTHEADER).trim();
             if (!proxyHeader.equals("") && request.getHeader(proxyHeader) != null) { //Trusted header is present, which means the user is already authorized.
-              clientUserName = getUsername(request, authType);
+              clientUserName = getUsername(request);
             } else {
               // For password based authentication
-              clientUserName = doPasswdAuth(request, authType);
+              clientUserName = doPasswdAuth(request, authType.getPasswordBasedAuthStr());
             }
           }
         }
@@ -251,7 +252,7 @@ public class ThriftHttpServlet extends TServlet {
 
       // Generate new cookie and add it to the response
       if (requireNewCookie &&
-          !authType.toLowerCase().contains(HiveAuthConstants.AuthTypes.NOSASL.toString().toLowerCase())) {
+          !authType.isEnabled(HiveAuthConstants.AuthTypes.NOSASL)) {
         String cookieToken = HttpAuthUtils.createCookieToken(clientUserName);
         Cookie hs2Cookie = createCookie(signer.signCookie(cookieToken));
 
@@ -285,7 +286,7 @@ public class ThriftHttpServlet extends TServlet {
       } else {
         try {
           LOG.error("Login attempt is failed for user : " +
-              getUsername(request, authType) + ". Error Messsage :" + e.getMessage());
+              getUsername(request) + ". Error Messsage :" + e.getMessage());
         } catch (Exception ex) {
           // Ignore Exception
         }
@@ -510,15 +511,14 @@ public class ThriftHttpServlet extends TServlet {
    */
   private String doPasswdAuth(HttpServletRequest request, String authType)
       throws HttpAuthenticationException {
-    String userName = getUsername(request, authType);
+    String userName = getUsername(request);
     // No-op when authType is NOSASL
     if (!authType.toLowerCase().contains(HiveAuthConstants.AuthTypes.NOSASL.toString().toLowerCase())) {
       try {
         AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType);
         PasswdAuthenticationProvider provider =
             AuthenticationProviderFactory.getAuthenticationProvider(authMethod, hiveConf);
-        provider.Authenticate(userName, getPassword(request, authType));
-
+        provider.Authenticate(userName, getPassword(request));
       } catch (Exception e) {
         throw new HttpAuthenticationException(e);
       }
@@ -551,7 +551,7 @@ public class ThriftHttpServlet extends TServlet {
       throws HttpAuthenticationException {
     // Each http request must have an Authorization header
     // Check before trying to do kerberos authentication twice
-    getAuthHeader(request, authType);
+    getAuthHeader(request);
 
     // Try authenticating with the HTTP/_HOST principal
     if (httpUGI != null) {
@@ -608,7 +608,7 @@ public class ThriftHttpServlet extends TServlet {
         // Create a GSS context
         gssContext = manager.createContext(serverCreds);
         // Get service ticket from the authorization header
-        String serviceTicketBase64 = getAuthHeader(request, authType);
+        String serviceTicketBase64 = getAuthHeader(request);
         byte[] inToken = Base64.getDecoder().decode(serviceTicketBase64);
         gssContext.acceptSecContext(inToken, 0, inToken.length);
         // Authenticate or deny based on its context completion
@@ -616,12 +616,10 @@ public class ThriftHttpServlet extends TServlet {
           throw new HttpAuthenticationException("Kerberos authentication failed: " +
               "unable to establish context with the service ticket " +
               "provided by the client.");
-        }
-        else {
+        } else {
           return getPrincipalWithoutRealmAndHost(gssContext.getSrcName().toString());
         }
-      }
-      catch (GSSException e) {
+      } catch (GSSException e) {
         if (gssContext != null) {
           try {
             LOG.error("Login attempt is failed for user : " +
@@ -632,8 +630,7 @@ public class ThriftHttpServlet extends TServlet {
           }
         }
         throw new HttpAuthenticationException("Kerberos authentication failed: ", e);
-      }
-      finally {
+      } finally {
         if (gssContext != null) {
           try {
             gssContext.dispose();
@@ -673,9 +670,9 @@ public class ThriftHttpServlet extends TServlet {
     }
   }
 
-  private String getUsername(HttpServletRequest request, String authType)
+  private String getUsername(HttpServletRequest request)
       throws HttpAuthenticationException {
-    String creds[] = getAuthHeaderTokens(request, authType);
+    String creds[] = getAuthHeaderTokens(request);
     // Username must be present
     if (creds[0] == null || creds[0].isEmpty()) {
       throw new HttpAuthenticationException("Authorization header received " +
@@ -684,9 +681,9 @@ public class ThriftHttpServlet extends TServlet {
     return creds[0];
   }
 
-  private String getPassword(HttpServletRequest request, String authType)
+  private String getPassword(HttpServletRequest request)
       throws HttpAuthenticationException {
-    String creds[] = getAuthHeaderTokens(request, authType);
+    String[] creds = getAuthHeaderTokens(request);
     // Password must be present
     if (creds[1] == null || creds[1].isEmpty()) {
       throw new HttpAuthenticationException("Authorization header received " +
@@ -695,9 +692,8 @@ public class ThriftHttpServlet extends TServlet {
     return creds[1];
   }
 
-  private String[] getAuthHeaderTokens(HttpServletRequest request,
-      String authType) throws HttpAuthenticationException {
-    String authHeaderBase64Str = getAuthHeader(request, authType);
+  private String[] getAuthHeaderTokens(HttpServletRequest request) throws HttpAuthenticationException {
+    String authHeaderBase64Str = getAuthHeader(request);
     String authHeaderString = new String(Base64.getDecoder().decode(authHeaderBase64Str), StandardCharsets.UTF_8);
     return authHeaderString.split(":");
   }
@@ -705,11 +701,10 @@ public class ThriftHttpServlet extends TServlet {
   /**
    * Returns the base64 encoded auth header payload
    * @param request request to interrogate
-   * @param authType Either BASIC or NEGOTIATE
    * @return base64 encoded auth header payload
    * @throws HttpAuthenticationException exception if header is missing or empty
    */
-  private String getAuthHeader(HttpServletRequest request, String authType)
+  private String getAuthHeader(HttpServletRequest request)
       throws HttpAuthenticationException {
     String authHeader = request.getHeader(HttpAuthUtils.AUTHORIZATION);
     // Each http request must have an Authorization header
@@ -733,8 +728,8 @@ public class ThriftHttpServlet extends TServlet {
     return authHeaderBase64String;
   }
 
-  private boolean isKerberosAuthMode(String authType) {
-    return authType.equalsIgnoreCase(HiveAuthConstants.AuthTypes.KERBEROS.toString());
+  private boolean isKerberosAuthMode(AuthType authType) {
+    return authType.isEnabled(HiveAuthConstants.AuthTypes.KERBEROS);
   }
 
   private static String getDoAsQueryParam(String queryString) {
diff --git a/service/src/test/org/apache/hive/service/auth/TestAuthType.java b/service/src/test/org/apache/hive/service/auth/TestAuthType.java
new file mode 100644
index 0000000..08db27b
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/auth/TestAuthType.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service.auth;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Tests to test if AuthType can parse allowed values and reject disallowed combinations.
+ */
+public class TestAuthType {
+  @Test
+  public void testSingleAuth() throws Exception {
+    for (HiveAuthConstants.AuthTypes type : HiveAuthConstants.AuthTypes.values()) {
+      testSingleAuth(type);
+    }
+  }
+
+  private void testSingleAuth(HiveAuthConstants.AuthTypes type) throws Exception {
+    AuthType authType = new AuthType(type.getAuthName());
+    Assert.assertTrue(authType.isEnabled(type));
+    if (type == HiveAuthConstants.AuthTypes.NOSASL || type == HiveAuthConstants.AuthTypes.NONE ||
+        AuthType.PASSWORD_BASED_TYPES.contains(type)) {
+      Assert.assertEquals(type.getAuthName(), authType.getPasswordBasedAuthStr());
+    } else {
+      Assert.assertEquals("Should return empty string if no password based authentication is set.",
+          "", authType.getPasswordBasedAuthStr());
+    }
+  }
+
+  @Test
+  public void testOnePasswordAuthWithSAML() throws Exception {
+    testOnePasswordAuthWithSAML(HiveAuthConstants.AuthTypes.LDAP);
+    testOnePasswordAuthWithSAML(HiveAuthConstants.AuthTypes.PAM);
+    testOnePasswordAuthWithSAML(HiveAuthConstants.AuthTypes.CUSTOM);
+  }
+
+  private void testOnePasswordAuthWithSAML(HiveAuthConstants.AuthTypes type) throws Exception {
+    AuthType authType = new AuthType("SAML," + type.getAuthName());
+    Assert.assertTrue(authType.isEnabled(HiveAuthConstants.AuthTypes.SAML));
+    Assert.assertTrue(authType.isEnabled(type));
+
+    Set<HiveAuthConstants.AuthTypes> disabledAuthTypes = Arrays.stream(HiveAuthConstants.AuthTypes.values())
+        .collect(Collectors.toSet());
+    disabledAuthTypes.remove(HiveAuthConstants.AuthTypes.SAML);
+    disabledAuthTypes.remove(type);
+    for (HiveAuthConstants.AuthTypes disabledType : disabledAuthTypes) {
+      Assert.assertFalse(authType.isEnabled(disabledType));
+    }
+    Assert.assertEquals(type.getAuthName(), authType.getPasswordBasedAuthStr());
+  }
+
+  @Test(expected = Exception.class)
+  public void testKerberosWithSAML() throws Exception {
+    AuthType authType = new AuthType("KERBEROS,SAML");
+  }
+
+  @Test(expected = Exception.class)
+  public void testKerberosWithSAMLAndLdap() throws Exception {
+    AuthType authType = new AuthType("KERBEROS,SAML,LDAP");
+  }
+
+  @Test(expected = Exception.class)
+  public void testKerberosWithLdap() throws Exception {
+    AuthType authType = new AuthType("KERBEROS,LDAP");
+  }
+
+  @Test(expected = Exception.class)
+  public void testNoneWithSAML() throws Exception {
+    AuthType authType = new AuthType("NONE,SAML");
+  }
+
+  @Test(expected = Exception.class)
+  public void testNoSaslWithSAML() throws Exception {
+    AuthType authType = new AuthType("NOSASL,SAML");
+  }
+
+  @Test(expected = Exception.class)
+  public void testMultiPasswordAuthWithSAML() throws Exception {
+    AuthType authType = new AuthType("SAML,LDAP,PAM,CUSTOM");
+  }
+
+  @Test(expected = Exception.class)
+  public void testMultiPasswordAuth() throws Exception {
+    AuthType authType = new AuthType("LDAP,PAM,CUSTOM");
+  }
+
+  @Test(expected = Exception.class)
+  public void testNotExistAuth() throws Exception {
+    AuthType authType = new AuthType("SAML,OTHER");
+  }
+}
diff --git a/service/src/test/org/apache/hive/service/cli/thrift/ThriftHttpServletTest.java b/service/src/test/org/apache/hive/service/cli/thrift/ThriftHttpServletTest.java
index 5695de2..c490cc2 100644
--- a/service/src/test/org/apache/hive/service/cli/thrift/ThriftHttpServletTest.java
+++ b/service/src/test/org/apache/hive/service/cli/thrift/ThriftHttpServletTest.java
@@ -43,7 +43,7 @@ public class ThriftHttpServletTest {
   private ThriftHttpServlet thriftHttpServlet;
 
   @Before
-  public void setUp() {
+  public void setUp() throws Exception {
     String authType = HiveAuthConstants.AuthTypes.KERBEROS.toString();
     thriftHttpServlet = new ThriftHttpServlet(null, null, authType, null, null, null,
         new HiveConf());