You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2014/11/04 20:45:34 UTC

svn commit: r1636701 - in /hive/trunk: jdbc/src/java/org/apache/hive/jdbc/ service/src/java/org/apache/hive/service/auth/ service/src/java/org/apache/hive/service/cli/thrift/

Author: vgumashta
Date: Tue Nov  4 19:45:34 2014
New Revision: 1636701

URL: http://svn.apache.org/r1636701
Log:
HIVE-8705: Support using pre-authenticated subject in kerberized HiveServer2 HTTP mode (Vaibhav Gumashta reviewed by Thejas Nair)

Modified:
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpKerberosRequestInterceptor.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java

Modified: hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java?rev=1636701&r1=1636700&r2=1636701&view=diff
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java (original)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java Tue Nov  4 19:45:34 2014
@@ -99,6 +99,7 @@ public class HiveConnection implements j
   private JdbcConnectionParams connParams;
   private final boolean isEmbeddedMode;
   private TTransport transport;
+  private boolean assumeSubject;
   // TODO should be replaced by CliServiceClient
   private TCLIService.Iface client;
   private boolean isClosed = true;
@@ -177,6 +178,9 @@ public class HiveConnection implements j
   private void openTransport() throws SQLException {
     while (true) {
       try {
+        assumeSubject =
+            JdbcConnectionParams.AUTH_KERBEROS_AUTH_TYPE_FROM_SUBJECT.equals(sessConfMap
+                .get(JdbcConnectionParams.AUTH_KERBEROS_AUTH_TYPE));
         transport = isHttpTransportMode() ? createHttpTransport() : createBinaryTransport();
         if (!transport.isOpen()) {
           LOG.info("Will try to open client transport with JDBC Uri: " + jdbcUriString);
@@ -265,8 +269,9 @@ public class HiveConnection implements j
        * In https mode, the entire information is encrypted
        * TODO: Optimize this with a mix of kerberos + using cookie.
        */
-      requestInterceptor = new HttpKerberosRequestInterceptor(
-          sessConfMap.get(JdbcConnectionParams.AUTH_PRINCIPAL), host, getServerHttpUrl(false));
+      requestInterceptor =
+          new HttpKerberosRequestInterceptor(sessConfMap.get(JdbcConnectionParams.AUTH_PRINCIPAL),
+              host, getServerHttpUrl(useSsl), assumeSubject);
     }
     else {
       /**
@@ -351,8 +356,6 @@ public class HiveConnection implements j
           }
           saslProps.put(Sasl.QOP, saslQOP.toString());
           saslProps.put(Sasl.SERVER_AUTH, "true");
-          boolean assumeSubject = JdbcConnectionParams.AUTH_KERBEROS_AUTH_TYPE_FROM_SUBJECT.equals(sessConfMap
-              .get(JdbcConnectionParams.AUTH_KERBEROS_AUTH_TYPE));
           transport = KerberosSaslHelper.getKerberosTransport(
               sessConfMap.get(JdbcConnectionParams.AUTH_PRINCIPAL), host,
               HiveAuthFactory.getSocketTransport(host, port, loginTimeout), saslProps,

Modified: hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpKerberosRequestInterceptor.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpKerberosRequestInterceptor.java?rev=1636701&r1=1636700&r2=1636701&view=diff
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpKerberosRequestInterceptor.java (original)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpKerberosRequestInterceptor.java Tue Nov  4 19:45:34 2014
@@ -39,15 +39,17 @@ public class HttpKerberosRequestIntercep
   String principal;
   String host;
   String serverHttpUrl;
+  boolean assumeSubject;
 
   // A fair reentrant lock
   private static ReentrantLock kerberosLock = new ReentrantLock(true);
 
   public HttpKerberosRequestInterceptor(String principal, String host,
-      String serverHttpUrl) {
+      String serverHttpUrl, boolean assumeSubject) {
     this.principal = principal;
     this.host = host;
     this.serverHttpUrl = serverHttpUrl;
+    this.assumeSubject = assumeSubject;
   }
 
   @Override
@@ -59,7 +61,7 @@ public class HttpKerberosRequestIntercep
       // Locking ensures the tokens are unique in case of concurrent requests
       kerberosLock.lock();
       kerberosAuthHeader = HttpAuthUtils.getKerberosServiceTicket(
-          principal, host, serverHttpUrl);
+          principal, host, serverHttpUrl, assumeSubject);
       // Set the session key token (Base64 encoded) in the headers
       httpRequest.addHeader(HttpAuthUtils.AUTHORIZATION + ": " +
           HttpAuthUtils.NEGOTIATE + " ", kerberosAuthHeader);

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java?rev=1636701&r1=1636700&r2=1636701&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java Tue Nov  4 19:45:34 2014
@@ -18,16 +18,18 @@
 
 package org.apache.hive.service.auth;
 
-import java.io.IOException;
+import java.security.AccessControlContext;
+import java.security.AccessController;
 import java.security.PrivilegedExceptionAction;
 
+import javax.security.auth.Subject;
+
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.http.protocol.BasicHttpContext;
 import org.apache.http.protocol.HttpContext;
 import org.ietf.jgss.GSSContext;
-import org.ietf.jgss.GSSCredential;
 import org.ietf.jgss.GSSManager;
 import org.ietf.jgss.GSSName;
 import org.ietf.jgss.Oid;
@@ -36,60 +38,54 @@ import org.ietf.jgss.Oid;
  * Utility functions for HTTP mode authentication.
  */
 public final class HttpAuthUtils {
-
   public static final String WWW_AUTHENTICATE = "WWW-Authenticate";
   public static final String AUTHORIZATION = "Authorization";
   public static final String BASIC = "Basic";
   public static final String NEGOTIATE = "Negotiate";
-  
-  /**
-   * @return Stringified Base64 encoded kerberosAuthHeader on success
-   */
-  public static String getKerberosServiceTicket(String principal, String host, String serverHttpUrl)
-    throws IOException, InterruptedException {
-    UserGroupInformation clientUGI = getClientUGI("kerberos");
-    String serverPrincipal = getServerPrincipal(principal, host);
-    // Uses the Ticket Granting Ticket in the UserGroupInformation
-    return clientUGI.doAs(
-      new HttpKerberosClientAction(serverPrincipal, clientUGI.getUserName(), serverHttpUrl));
-  }
 
   /**
-   * Get server principal and verify that hostname is present.
-   */
-  private static String getServerPrincipal(String principal, String host) throws IOException {
-    return ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host);
-  }
-
-  /**
-   * JAAS login to setup the client UserGroupInformation.
-   * Sets up the Kerberos Ticket Granting Ticket,
-   * in the client UserGroupInformation object.
-   *
-   * @return Client's UserGroupInformation
+   * @return Stringified Base64 encoded kerberosAuthHeader on success
+   * @throws Exception
    */
-  public static UserGroupInformation getClientUGI(String authType) throws IOException {
-    return ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf(authType);
+  public static String getKerberosServiceTicket(String principal, String host,
+      String serverHttpUrl, boolean assumeSubject) throws Exception {
+    String serverPrincipal =
+        ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host);
+    if (assumeSubject) {
+      // With this option, we're assuming that the external application,
+      // using the JDBC driver has done a JAAS kerberos login already
+      AccessControlContext context = AccessController.getContext();
+      Subject subject = Subject.getSubject(context);
+      if (subject == null) {
+        throw new Exception("The Subject is not set");
+      }
+      return Subject.doAs(subject, new HttpKerberosClientAction(serverPrincipal, serverHttpUrl));
+    } else {
+      // JAAS login from ticket cache to setup the client UserGroupInformation
+      UserGroupInformation clientUGI =
+          ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf("kerberos");
+      return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal, serverHttpUrl));
+    }
   }
 
   private HttpAuthUtils() {
     throw new UnsupportedOperationException("Can't initialize class");
   }
 
+  /**
+   * We'll create an instance of this class within a doAs block so that the client's TGT credentials
+   * can be read from the Subject
+   */
   public static class HttpKerberosClientAction implements PrivilegedExceptionAction<String> {
-
     public static final String HTTP_RESPONSE = "HTTP_RESPONSE";
     public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL";
     private final String serverPrincipal;
-    private final String clientUserName;
     private final String serverHttpUrl;
     private final Base64 base64codec;
     private final HttpContext httpContext;
 
-    public HttpKerberosClientAction(String serverPrincipal, String clientUserName,
-      String serverHttpUrl) {
+    public HttpKerberosClientAction(String serverPrincipal, String serverHttpUrl) {
       this.serverPrincipal = serverPrincipal;
-      this.clientUserName = clientUserName;
       this.serverHttpUrl = serverHttpUrl;
       base64codec = new Base64(0);
       httpContext = new BasicHttpContext();
@@ -102,38 +98,17 @@ public final class HttpAuthUtils {
       Oid mechOid = new Oid("1.2.840.113554.1.2.2");
       // Oid for kerberos principal name
       Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1");
-
       GSSManager manager = GSSManager.getInstance();
-
-      // GSS name for client
-      GSSName clientName = manager.createName(clientUserName, GSSName.NT_USER_NAME);
       // GSS name for server
       GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid);
-
-      // GSS credentials for client
-      GSSCredential clientCreds =
-        manager.createCredential(clientName, GSSCredential.DEFAULT_LIFETIME, mechOid,
-          GSSCredential.INITIATE_ONLY);
-
-      /*
-       * Create a GSSContext for mutual authentication with the
-       * server.
-       *    - serverName is the GSSName that represents the server.
-       *    - krb5Oid is the Oid that represents the mechanism to
-       *      use. The client chooses the mechanism to use.
-       *    - clientCreds are the client credentials
-       */
+      // Create a GSSContext for authentication with the service.
+      // We're passing client credentials as null since we want them to be read from the Subject.
       GSSContext gssContext =
-        manager.createContext(serverName, mechOid, clientCreds, GSSContext.DEFAULT_LIFETIME);
-
-      // Mutual authentication not r
+          manager.createContext(serverName, mechOid, null, GSSContext.DEFAULT_LIFETIME);
       gssContext.requestMutualAuth(false);
-
       // Establish context
       byte[] inToken = new byte[0];
-
       byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length);
-
       gssContext.dispose();
       // Base64 encoded and stringified token for server
       return new String(base64codec.encode(outToken));

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java?rev=1636701&r1=1636700&r2=1636701&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java Tue Nov  4 19:45:34 2014
@@ -208,11 +208,9 @@ public class ThriftHttpServlet extends T
 
         // Create a GSS context
         gssContext = manager.createContext(serverCreds);
-
         // Get service ticket from the authorization header
         String serviceTicketBase64 = getAuthHeader(request, authType);
         byte[] inToken = Base64.decodeBase64(serviceTicketBase64.getBytes());
-
         gssContext.acceptSecContext(inToken, 0, inToken.length);
         // Authenticate or deny based on its context completion
         if (!gssContext.isEstablished()) {