You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ct...@apache.org on 2016/04/03 03:26:37 UTC

hive git commit: HIVE-13401:Kerberized HS2 with LDAP auth enabled fails kerberos/delegation token authentication (Chaoyu Tang, reviewed by Szehon Ho)

Repository: hive
Updated Branches:
  refs/heads/master d94e8d08d -> 6a1f8a835


HIVE-13401:Kerberized HS2 with LDAP auth enabled fails kerberos/delegation token authentication (Chaoyu Tang, reviewed by Szehon Ho)


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

Branch: refs/heads/master
Commit: 6a1f8a835e13d0e0d3500fce02da8c14de34a023
Parents: d94e8d0
Author: ctang <ct...@cloudera.com>
Authored: Sat Apr 2 21:26:22 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Sat Apr 2 21:26:22 2016 -0400

----------------------------------------------------------------------
 .../org/apache/hive/minikdc/MiniHiveKdc.java    |  46 +++++++--
 .../minikdc/TestJdbcNonKrbSASLWithMiniKdc.java  | 103 +++++++++++++++++++
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |  14 ++-
 .../hive/service/auth/HiveAuthFactory.java      |  13 ++-
 .../service/cli/thrift/ThriftCLIService.java    |  15 +--
 5 files changed, 162 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java
----------------------------------------------------------------------
diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java
index 4e3a9c5..bbec37e 100644
--- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java
+++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/MiniHiveKdc.java
@@ -50,6 +50,7 @@ public class MiniHiveKdc {
   public static String HIVE_TEST_USER_1 = "user1";
   public static String HIVE_TEST_USER_2 = "user2";
   public static String HIVE_TEST_SUPER_USER = "superuser";
+  public static String AUTHENTICATION_TYPE = "KERBEROS";
 
   private final MiniKdc miniKdc;
   private final File workDir;
@@ -170,16 +171,29 @@ public class MiniHiveKdc {
    * @throws Exception
    */
   public static MiniHS2 getMiniHS2WithKerb(MiniHiveKdc miniHiveKdc, HiveConf hiveConf) throws Exception {
-    String hivePrincipal =
-        miniHiveKdc.getFullyQualifiedServicePrincipal(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL);
-    String hiveKeytab = miniHiveKdc.getKeyTabFile(
-        miniHiveKdc.getServicePrincipalForUser(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL));
-
-    return new MiniHS2.Builder().withConf(hiveConf).
-        withMiniKdc(hivePrincipal, hiveKeytab).build();
+    return getMiniHS2WithKerb(miniHiveKdc, hiveConf, AUTHENTICATION_TYPE);
   }
 
   /**
+  * Create a MiniHS2 with the hive service principal and keytab in MiniHiveKdc
+  * @param miniHiveKdc
+  * @param hiveConf
+  * @param authType
+  * @return new MiniHS2 instance
+  * @throws Exception
+  */
+ public static MiniHS2 getMiniHS2WithKerb(MiniHiveKdc miniHiveKdc, HiveConf hiveConf,
+     String authType) throws Exception {
+   String hivePrincipal =
+       miniHiveKdc.getFullyQualifiedServicePrincipal(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL);
+   String hiveKeytab = miniHiveKdc.getKeyTabFile(
+       miniHiveKdc.getServicePrincipalForUser(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL));
+
+   return new MiniHS2.Builder().withConf(hiveConf).withMiniKdc(hivePrincipal, hiveKeytab).
+       withAuthenticationType(authType).build();
+ }
+
+  /**
    * Create a MiniHS2 with the hive service principal and keytab in MiniHiveKdc
    * @param miniHiveKdc
    * @param hiveConf
@@ -187,12 +201,26 @@ public class MiniHiveKdc {
    * @throws Exception
    */
   public static MiniHS2 getMiniHS2WithKerbWithRemoteHMS(MiniHiveKdc miniHiveKdc, HiveConf hiveConf) throws Exception {
+    return getMiniHS2WithKerbWithRemoteHMS(miniHiveKdc, hiveConf, AUTHENTICATION_TYPE);
+  }
+
+  /**
+   * Create a MiniHS2 with the hive service principal and keytab in MiniHiveKdc. It uses remote HMS
+   * and can support a different Sasl authType
+   * @param miniHiveKdc
+   * @param hiveConf
+   * @param authType
+   * @return new MiniHS2 instance
+   * @throws Exception
+   */
+  public static MiniHS2 getMiniHS2WithKerbWithRemoteHMS(MiniHiveKdc miniHiveKdc, HiveConf hiveConf,
+      String authType) throws Exception {
     String hivePrincipal =
         miniHiveKdc.getFullyQualifiedServicePrincipal(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL);
     String hiveKeytab = miniHiveKdc.getKeyTabFile(
         miniHiveKdc.getServicePrincipalForUser(MiniHiveKdc.HIVE_SERVICE_PRINCIPAL));
 
     return new MiniHS2.Builder().withConf(hiveConf).withRemoteMetastore().
-        withMiniKdc(hivePrincipal, hiveKeytab).build();
+        withMiniKdc(hivePrincipal, hiveKeytab).withAuthenticationType(authType).build();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcNonKrbSASLWithMiniKdc.java
----------------------------------------------------------------------
diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcNonKrbSASLWithMiniKdc.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcNonKrbSASLWithMiniKdc.java
new file mode 100644
index 0000000..1c1beda
--- /dev/null
+++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcNonKrbSASLWithMiniKdc.java
@@ -0,0 +1,103 @@
+/**
+ * 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.minikdc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+
+import javax.security.sasl.AuthenticationException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.hive.service.auth.PasswdAuthenticationProvider;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestJdbcNonKrbSASLWithMiniKdc extends TestJdbcWithMiniKdc{
+
+  public static class CustomAuthenticator implements PasswdAuthenticationProvider {
+    @Override
+    public void Authenticate(String user, String password) throws AuthenticationException {
+      if (!("nonkrbuser".equals(user) && "mypwd".equals(password))) {
+        throw new AuthenticationException("Authentication failed");
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    Class.forName(MiniHS2.getJdbcDriverName());
+    confOverlay.put(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname,
+        SessionHookTest.class.getName());
+    confOverlay.put(ConfVars.HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS.varname,
+        CustomAuthenticator.class.getName());
+    HiveConf hiveConf = new HiveConf();
+    miniHiveKdc = MiniHiveKdc.getMiniHiveKdc(hiveConf);
+    miniHS2 = MiniHiveKdc.getMiniHS2WithKerbWithRemoteHMS(miniHiveKdc, hiveConf, "CUSTOM");
+    miniHS2.start(confOverlay);
+  }
+
+  /***
+   * Test a nonkrb user could login the kerberized HS2 with authentication type SASL NONE
+   * @throws Exception
+   */
+  @Test
+  public void testNonKrbSASLAuth() throws Exception {
+    hs2Conn = DriverManager.getConnection(miniHS2.getBaseJdbcURL() + "default;user=nonkrbuser;password=mypwd");
+    verifyProperty(SESSION_USER_NAME, "nonkrbuser");
+    hs2Conn.close();
+  }
+
+  /***
+   * Negative test, verify that connection to secure HS2 fails if it is noSasl
+   * @throws Exception
+   */
+  @Test
+  public void testNoSaslConnectionNeg() throws Exception {
+    try {
+      String url = miniHS2.getBaseJdbcURL() + "default;auth=noSasl";
+      hs2Conn = DriverManager.getConnection(url);
+      fail("noSasl connection should fail");
+    } catch (SQLException e) {
+      // expected error
+      assertEquals("08S01", e.getSQLState().trim());
+    }
+  }
+
+  /***
+   * Negative test, verify that NonKrb connection to secure HS2 fails if it is
+   * user/pwd do not match.
+   * @throws Exception
+   */
+  @Test
+  public void testNoKrbConnectionNeg() throws Exception {
+    try {
+      String url = miniHS2.getBaseJdbcURL() + "default;user=wronguser;pwd=mypwd";
+      hs2Conn = DriverManager.getConnection(url);
+      fail("noSasl connection should fail");
+    } catch (SQLException e) {
+      // expected error
+      assertEquals("08S01", e.getSQLState().trim());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index eca2317..6141a1a 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -85,6 +85,7 @@ public class MiniHS2 extends AbstractHiveService {
     private boolean isHTTPTransMode = false;
     private boolean isMetastoreRemote;
     private boolean usePortsFromConf = false;
+    private String authType = "KERBEROS";
 
     public Builder() {
     }
@@ -101,6 +102,11 @@ public class MiniHS2 extends AbstractHiveService {
       return this;
     }
 
+    public Builder withAuthenticationType(String authType) {
+      this.authType = authType;
+      return this;
+    }
+
     public Builder withRemoteMetastore() {
       this.isMetastoreRemote = true;
       return this;
@@ -131,7 +137,7 @@ public class MiniHS2 extends AbstractHiveService {
         hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_BINARY_MODE);
       }
       return new MiniHS2(hiveConf, miniClusterType, useMiniKdc, serverPrincipal, serverKeytab,
-          isMetastoreRemote, usePortsFromConf);
+          isMetastoreRemote, usePortsFromConf, authType);
     }
   }
 
@@ -169,7 +175,7 @@ public class MiniHS2 extends AbstractHiveService {
 
   private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean useMiniKdc,
       String serverPrincipal, String serverKeytab, boolean isMetastoreRemote,
-      boolean usePortsFromConf) throws Exception {
+      boolean usePortsFromConf, String authType) throws Exception {
     super(hiveConf, "localhost",
         (usePortsFromConf ? hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT) : MetaStoreUtils.findFreePort()),
         (usePortsFromConf ? hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT) : MetaStoreUtils.findFreePort()));
@@ -217,7 +223,7 @@ public class MiniHS2 extends AbstractHiveService {
     if (useMiniKdc) {
       hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL, serverPrincipal);
       hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB, serverKeytab);
-      hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, "KERBEROS");
+      hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, authType);
     }
     String metaStoreURL =  "jdbc:derby:" + baseDir.getAbsolutePath() + File.separator + "test_metastore-" +
         hs2Counter.incrementAndGet() + ";create=true";
@@ -260,7 +266,7 @@ public class MiniHS2 extends AbstractHiveService {
 
   public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType,
       boolean usePortsFromConf) throws Exception {
-    this(hiveConf, clusterType, false, null, null, false, usePortsFromConf);
+    this(hiveConf, clusterType, false, null, null, false, usePortsFromConf, "KERBEROS");
   }
 
   public void start(Map<String, String> confOverlay) throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java b/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
index 6992f62..8af9d0a 100644
--- a/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
+++ b/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
@@ -116,8 +116,7 @@ public class HiveAuthFactory {
         authTypeStr = AuthTypes.NONE.getAuthName();
       }
     }
-    if (hadoopAuth.equalsIgnoreCase("kerberos")
-        && !authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) {
+    if (isSASLWithKerberizedHadoop()) {
       saslServer =
           ShimLoader.getHadoopThriftAuthBridge().createServer(
               conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB),
@@ -162,8 +161,7 @@ public class HiveAuthFactory {
     TTransportFactory transportFactory;
     TSaslServerTransport.Factory serverTransportFactory;
 
-    if (hadoopAuth.equalsIgnoreCase("kerberos") && !authTypeStr.equalsIgnoreCase(
-          AuthTypes.NOSASL.getAuthName())) {
+    if (isSASLWithKerberizedHadoop()) {
       try {
         serverTransportFactory = saslServer.createSaslServerTransportFactory(
             getSaslProperties());
@@ -207,7 +205,7 @@ public class HiveAuthFactory {
    * @throws LoginException
    */
   public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException {
-    if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
+    if (isSASLWithKerberizedHadoop()) {
       return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service);
     } else {
       return PlainSaslHelper.getPlainProcessorFactory(service);
@@ -226,6 +224,11 @@ public class HiveAuthFactory {
     }
   }
 
+  public boolean isSASLWithKerberizedHadoop() {
+    return "kerberos".equalsIgnoreCase(hadoopAuth)
+        && !authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName());
+  }
+
   // Perform kerberos login using the hadoop shim API if the configuration is available
   public static void loginFromKeytab(HiveConf hiveConf) throws IOException {
     String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL);

http://git-wip-us.apache.org/repos/asf/hive/blob/6a1f8a83/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
index 62fcde5..0a2a761 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
@@ -387,11 +387,10 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       clientIpAddress = SessionManager.getIpAddress();
     }
     else {
-      // Kerberos
-      if (isKerberosAuthMode()) {
+      if (hiveAuthFactory != null && hiveAuthFactory.isSASLWithKerberizedHadoop()) {
         clientIpAddress = hiveAuthFactory.getIpAddress();
       }
-      // Except kerberos, NOSASL
+      // NOSASL
       else {
         clientIpAddress = TSetIpAddressProcessor.getUserIpAddress();
       }
@@ -412,11 +411,10 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
    */
   private String getUserName(TOpenSessionReq req) throws HiveSQLException, IOException {
     String userName = null;
-    // Kerberos
-    if (isKerberosAuthMode()) {
+    if (hiveAuthFactory != null && hiveAuthFactory.isSASLWithKerberizedHadoop()) {
       userName = hiveAuthFactory.getRemoteUser();
     }
-    // Except kerberos, NOSASL
+    // NOSASL
     if (userName == null) {
       userName = TSetIpAddressProcessor.getUserName();
     }
@@ -803,9 +801,4 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
     LOG.debug("Verified proxy user: " + proxyUser);
     return proxyUser;
   }
-
-  private boolean isKerberosAuthMode() {
-    return cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION)
-        .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString());
-  }
 }