You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2014/03/17 13:56:11 UTC

svn commit: r1578348 [1/2] - in /hive/trunk: ./ itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/ jdbc/src/java/org/apache/hive/jdbc/ service/src/java/org/apache/hive/service/auth/ service/src/java/org/apache/hive/service/cli/ service/...

Author: thejas
Date: Mon Mar 17 12:56:10 2014
New Revision: 1578348

URL: http://svn.apache.org/r1578348
Log:
HIVE-4764 : Support Kerberos HTTP authentication for HiveServer2 running in http mode (Vaibhav Gumashta via Thejas Nair)

Added:
    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/auth/HttpAuthenticationException.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java
Modified:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java
    hive/trunk/pom.xml
    hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
    hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java
    hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
    hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java?rev=1578348&r1=1578347&r2=1578348&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java Mon Mar 17 12:56:10 2014
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.jdbc.HttpBasicAuthInterceptor;
 import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
-import org.apache.hive.service.server.HiveServer2;
 import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.thrift.transport.THttpClient;
 import org.apache.thrift.transport.TTransport;
@@ -170,40 +169,6 @@ public class TestThriftHttpCLIService ex
     }
   }
 
-
-  private void testWithAuthMode(AuthTypes authType) throws Exception {
-    // Stop and restart HiveServer2 in given incorrect auth mode
-    stopHiveServer2();
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, authType.toString());
-    hiveServer2 = new HiveServer2();
-    // HiveServer2 in Http mode will not start using KERBEROS/LDAP/CUSTOM auth types
-    startHiveServer2WithConf(hiveConf);
-
-    // This will throw an expected exception since Http server is not running
-    testOpenSessionExpectedException();
-
-    // Stop and restart back with the original config
-    stopHiveServer2();
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NOSASL.toString());
-    hiveServer2 = new HiveServer2();
-    startHiveServer2WithConf(hiveConf);
-  }
-
-  @Test
-  public void testKerberosMode()  throws Exception {
-    testWithAuthMode(AuthTypes.KERBEROS);
-  }
-
-  @Test
-  public void testLDAPMode()  throws Exception {
-    testWithAuthMode(AuthTypes.LDAP);
-  }
-
-  @Test
-  public void testCustomMode()  throws Exception {
-    testWithAuthMode(AuthTypes.CUSTOM);
-  }
-
   private static TTransport createHttpTransport() throws Exception {
     DefaultHttpClient httpClient = new DefaultHttpClient();
     String httpUrl = transportMode + "://" + host + ":" + port +

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=1578348&r1=1578347&r2=1578348&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 Mon Mar 17 12:56:10 2014
@@ -47,7 +47,6 @@ import java.util.Properties;
 import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 
-import javax.net.ssl.SSLContext;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 
@@ -74,10 +73,9 @@ import org.apache.hive.service.cli.thrif
 import org.apache.hive.service.cli.thrift.TRenewDelegationTokenResp;
 import org.apache.hive.service.cli.thrift.TSessionHandle;
 import org.apache.http.HttpRequestInterceptor;
-import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
-import org.apache.http.conn.ssl.SSLContexts;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
+import org.apache.http.conn.scheme.Scheme;
+import org.apache.http.conn.ssl.SSLSocketFactory;
+import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.transport.THttpClient;
@@ -197,6 +195,7 @@ public class HiveConnection implements j
   }
 
   private void openTransport() throws SQLException {
+    // TODO: Refactor transport creation to a factory, it's getting uber messy here
     transport = isHttpTransportMode() ? createHttpTransport() : createBinaryTransport();
     try {
       if (!transport.isOpen()) {
@@ -208,8 +207,10 @@ public class HiveConnection implements j
     }
   }
 
-  private TTransport createHttpTransport() throws SQLException {
-    CloseableHttpClient httpClient;
+  private String getServerHttpUrl(boolean useSsl) {
+    // Create the http/https url
+    // JDBC driver will set up an https url if ssl is enabled, otherwise http
+    String schemeName = useSsl ? "https" : "http";
     // http path should begin with "/"
     String httpPath;
     httpPath = hiveConfMap.get(
@@ -217,21 +218,28 @@ public class HiveConnection implements j
     if(httpPath == null) {
       httpPath = "/";
     }
-    if(!httpPath.startsWith("/")) {
+    else if(!httpPath.startsWith("/")) {
       httpPath = "/" + httpPath;
     }
+    return schemeName +  "://" + host + ":" + port + httpPath;
+  }
+
+  private TTransport createHttpTransport() throws SQLException {
+    DefaultHttpClient httpClient;
 
-    boolean useSsl = "true".equalsIgnoreCase(sessConfMap.get(HIVE_USE_SSL));
+    boolean useSsl = isSslConnection();
 
     // Create an http client from the configs
-    httpClient = getHttpClient(useSsl);
+    try {
+      httpClient = getHttpClient(useSsl);
+    } catch (Exception e) {
+      String msg =  "Could not create http connection to " +
+          jdbcURI + ". " + e.getMessage();
+      throw new SQLException(msg, " 08S01", e);
+    }
 
-    // Create the http/https url
-    // JDBC driver will set up an https url if ssl is enabled, otherwise http
-    String schemeName = useSsl ? "https" : "http";
-    String httpUrl = schemeName +  "://" + host + ":" + port + httpPath;
     try {
-      transport = new THttpClient(httpUrl, httpClient);
+      transport = new THttpClient(getServerHttpUrl(useSsl), httpClient);
     }
     catch (TTransportException e) {
       String msg =  "Could not create http connection to " +
@@ -241,29 +249,53 @@ public class HiveConnection implements j
     return transport;
   }
 
-  private CloseableHttpClient getHttpClient(Boolean useSsl) throws SQLException {
-    // Add an interceptor to pass username/password in the header
-    // for basic preemtive http authentication at the server
-    // In https mode, the entire information is encrypted
-    HttpRequestInterceptor authInterceptor = new HttpBasicAuthInterceptor(
-        getUserName(), getPasswd());
-    if (useSsl) {
-      String sslTrustStorePath = sessConfMap.get(HIVE_SSL_TRUST_STORE);
-      String sslTrustStorePassword = sessConfMap.get(
-          HIVE_SSL_TRUST_STORE_PASSWORD);
-      KeyStore sslTrustStore;
-      SSLContext sslContext;
-      if (sslTrustStorePath == null || sslTrustStorePath.isEmpty()) {
-        // Create a default client context based on standard JSSE trust material
-        sslContext = SSLContexts.createDefault();
-      } else {
-        // Pick trust store config from the given path
+  private DefaultHttpClient getHttpClient(Boolean useSsl) throws SQLException {
+    DefaultHttpClient httpClient = new DefaultHttpClient();
+    // Request interceptor for any request pre-processing logic
+    HttpRequestInterceptor requestInterceptor;
+    // If Kerberos
+    if (isKerberosAuthMode()) {
+      if (useSsl) {
+        String msg = "SSL encryption is currently not supported with " +
+            "kerberos authentication";
+        throw new SQLException(msg, " 08S01");
+      }
+      /**
+       * Add an interceptor which sets the appropriate header in the request.
+       * It does the kerberos authentication and get the final service ticket,
+       * for sending to the server before every request.
+       */
+      requestInterceptor = new HttpKerberosRequestInterceptor(
+          sessConfMap.get(HIVE_AUTH_PRINCIPAL), host, getServerHttpUrl(false));
+    }
+    else {
+      /**
+       * Add an interceptor to pass username/password in the header.
+       * In https mode, the entire information is encrypted
+       */
+      requestInterceptor = new HttpBasicAuthInterceptor(getUserName(), getPassword());
+      // Configure httpClient for SSL
+      if (useSsl) {
+        String sslTrustStorePath = sessConfMap.get(HIVE_SSL_TRUST_STORE);
+        String sslTrustStorePassword = sessConfMap.get(
+            HIVE_SSL_TRUST_STORE_PASSWORD);
+        KeyStore sslTrustStore;
+        SSLSocketFactory socketFactory;
         try {
-          sslTrustStore = KeyStore.getInstance(HIVE_SSL_TRUST_STORE_TYPE);
-          sslTrustStore.load(new FileInputStream(sslTrustStorePath),
-              sslTrustStorePassword.toCharArray());
-          sslContext = SSLContexts.custom().loadTrustMaterial(
-              sslTrustStore).build();
+          if (sslTrustStorePath == null || sslTrustStorePath.isEmpty()) {
+            // Create a default socket factory based on standard JSSE trust material
+            socketFactory = SSLSocketFactory.getSocketFactory();
+          }
+          else {
+            // Pick trust store config from the given path
+            sslTrustStore = KeyStore.getInstance(HIVE_SSL_TRUST_STORE_TYPE);
+            sslTrustStore.load(new FileInputStream(sslTrustStorePath),
+                sslTrustStorePassword.toCharArray());
+            socketFactory = new SSLSocketFactory(sslTrustStore);
+          }
+          socketFactory.setHostnameVerifier(SSLSocketFactory.ALLOW_ALL_HOSTNAME_VERIFIER);
+          Scheme sslScheme = new Scheme("https", 443, socketFactory);
+          httpClient.getConnectionManager().getSchemeRegistry().register(sslScheme);
         }
         catch (Exception e) {
           String msg =  "Could not create an https connection to " +
@@ -271,13 +303,9 @@ public class HiveConnection implements j
           throw new SQLException(msg, " 08S01", e);
         }
       }
-      return HttpClients.custom().setHostnameVerifier(SSLConnectionSocketFactory.ALLOW_ALL_HOSTNAME_VERIFIER).setSslcontext(
-          sslContext).addInterceptorFirst(authInterceptor).build();
-    }
-    else {
-      // Create a plain http client
-      return HttpClients.custom().addInterceptorFirst(authInterceptor).build();
     }
+    httpClient.addRequestInterceptor(requestInterceptor);
+    return httpClient;
   }
 
   /**
@@ -319,19 +347,12 @@ public class HiveConnection implements j
           String tokenStr = getClientDelegationToken(sessConfMap);
           if (tokenStr != null) {
             transport = KerberosSaslHelper.getTokenTransport(tokenStr,
-                  host, HiveAuthFactory.getSocketTransport(host, port, loginTimeout), saslProps);
+                host, HiveAuthFactory.getSocketTransport(host, port, loginTimeout), saslProps);
           } else {
             // we are using PLAIN Sasl connection with user/password
-            String userName = sessConfMap.get(HIVE_AUTH_USER);
-            if ((userName == null) || userName.isEmpty()) {
-              userName = HIVE_ANONYMOUS_USER;
-            }
-            String passwd = sessConfMap.get(HIVE_AUTH_PASSWD);
-            if ((passwd == null) || passwd.isEmpty()) {
-              passwd = HIVE_ANONYMOUS_PASSWD;
-            }
-            String useSslStr = sessConfMap.get(HIVE_USE_SSL);
-            if ("true".equalsIgnoreCase(useSslStr)) {
+            String userName = getUserName();
+            String passwd = getPassword();
+            if (isSslConnection()) {
               // get SSL socket
               String sslTrustStore = sessConfMap.get(HIVE_SSL_TRUST_STORE);
               String sslTrustStorePassword = sessConfMap.get(HIVE_SSL_TRUST_STORE_PASSWORD);
@@ -339,14 +360,14 @@ public class HiveConnection implements j
                 transport = HiveAuthFactory.getSSLSocket(host, port, loginTimeout);
               } else {
                 transport = HiveAuthFactory.getSSLSocket(host, port, loginTimeout,
-                  sslTrustStore, sslTrustStorePassword);
+                    sslTrustStore, sslTrustStorePassword);
               }
             } else {
               // get non-SSL socket transport
               transport = HiveAuthFactory.getSocketTransport(host, port, loginTimeout);
             }
-          // Overlay the SASL transport on top of the base socket transport (SSL or non-SSL)
-          transport = PlainSaslHelper.getPlainTransport(userName, passwd, transport);
+            // Overlay the SASL transport on top of the base socket transport (SSL or non-SSL)
+            transport = PlainSaslHelper.getPlainTransport(userName, passwd, transport);
           }
         }
       } else {
@@ -363,16 +384,6 @@ public class HiveConnection implements j
     return transport;
   }
 
-
-  private boolean isHttpTransportMode() {
-    String transportMode =
-        hiveConfMap.get(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname);
-    if(transportMode != null && (transportMode.equalsIgnoreCase("http"))) {
-      return true;
-    }
-    return false;
-  }
-
   // Lookup the delegation token. First in the connection URL, then Configuration
   private String getClientDelegationToken(Map<String, String> jdbcConnConf)
       throws SQLException {
@@ -453,10 +464,28 @@ public class HiveConnection implements j
   /**
    * @return password from sessConfMap
    */
-  private String getPasswd() {
+  private String getPassword() {
     return getSessionValue(HIVE_AUTH_PASSWD, HIVE_ANONYMOUS_PASSWD);
   }
 
+  private boolean isSslConnection() {
+    return "true".equalsIgnoreCase(sessConfMap.get(HIVE_USE_SSL));
+  }
+
+  private boolean isKerberosAuthMode() {
+    return !HIVE_AUTH_SIMPLE.equals(sessConfMap.get(HIVE_AUTH_TYPE))
+        && sessConfMap.containsKey(HIVE_AUTH_PRINCIPAL);
+  }
+
+  private boolean isHttpTransportMode() {
+    String transportMode =
+        hiveConfMap.get(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname);
+    if(transportMode != null && (transportMode.equalsIgnoreCase("http"))) {
+      return true;
+    }
+    return false;
+  }
+
   /**
    * Lookup varName in sessConfMap, if its null or empty return the default
    * value varDefault
@@ -495,7 +524,7 @@ public class HiveConnection implements j
       return tokenResp.getDelegationToken();
     } catch (TException e) {
       throw new SQLException("Could not retrieve token: " +
-            e.getMessage(), " 08S01", e);
+          e.getMessage(), " 08S01", e);
     }
   }
 
@@ -503,12 +532,12 @@ public class HiveConnection implements j
     TCancelDelegationTokenReq cancelReq = new TCancelDelegationTokenReq(sessHandle, tokenStr);
     try {
       TCancelDelegationTokenResp cancelResp =
-              client.CancelDelegationToken(cancelReq);
+          client.CancelDelegationToken(cancelReq);
       Utils.verifySuccess(cancelResp.getStatus());
       return;
     } catch (TException e) {
       throw new SQLException("Could not cancel token: " +
-            e.getMessage(), " 08S01", e);
+          e.getMessage(), " 08S01", e);
     }
   }
 
@@ -516,12 +545,12 @@ public class HiveConnection implements j
     TRenewDelegationTokenReq cancelReq = new TRenewDelegationTokenReq(sessHandle, tokenStr);
     try {
       TRenewDelegationTokenResp renewResp =
-              client.RenewDelegationToken(cancelReq);
+          client.RenewDelegationToken(cancelReq);
       Utils.verifySuccess(renewResp.getStatus());
       return;
     } catch (TException e) {
       throw new SQLException("Could not renew token: " +
-            e.getMessage(), " 08S01", e);
+          e.getMessage(), " 08S01", e);
     }
   }
 

Modified: hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java?rev=1578348&r1=1578347&r2=1578348&view=diff
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java (original)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java Mon Mar 17 12:56:10 2014
@@ -25,6 +25,7 @@ import org.apache.http.HttpException;
 import org.apache.http.HttpRequest;
 import org.apache.http.HttpRequestInterceptor;
 import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.impl.auth.AuthSchemeBase;
 import org.apache.http.impl.auth.BasicScheme;
 import org.apache.http.protocol.HttpContext;
 
@@ -34,20 +35,22 @@ import org.apache.http.protocol.HttpCont
  *
  */
 public class HttpBasicAuthInterceptor implements HttpRequestInterceptor {
+  UsernamePasswordCredentials credentials;
+  AuthSchemeBase authScheme;
 
-  Header basicAuthHeader;
-  public HttpBasicAuthInterceptor(String username, String password){
+  public HttpBasicAuthInterceptor(String username, String password) {
     if(username != null){
-      UsernamePasswordCredentials creds = new UsernamePasswordCredentials(username, password);
-      basicAuthHeader = BasicScheme.authenticate(creds, "UTF-8", false);
+      credentials = new UsernamePasswordCredentials(username, password);
     }
+    authScheme = new BasicScheme();
   }
 
   @Override
-  public void process(HttpRequest httpRequest, HttpContext httpContext) throws HttpException, IOException {
-    if(basicAuthHeader != null){
-      httpRequest.addHeader(basicAuthHeader);
-    }
+  public void process(HttpRequest httpRequest, HttpContext httpContext)
+      throws HttpException, IOException {
+    Header basicAuthHeader = authScheme.authenticate(
+        credentials, httpRequest, httpContext);
+    httpRequest.addHeader(basicAuthHeader);
   }
 
 }

Added: 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=1578348&view=auto
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpKerberosRequestInterceptor.java (added)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HttpKerberosRequestInterceptor.java Mon Mar 17 12:56:10 2014
@@ -0,0 +1,65 @@
+/**
+* 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.jdbc;
+
+import java.io.IOException;
+
+import org.apache.hive.service.auth.HttpAuthUtils;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpRequestInterceptor;
+import org.apache.http.protocol.HttpContext;
+
+/**
+*
+* Authentication interceptor which adds Base64 encoded payload,
+* containing the username and kerberos service ticket,
+* to the outgoing http request header.
+*
+*/
+public class HttpKerberosRequestInterceptor implements HttpRequestInterceptor {
+
+String principal;
+String host;
+String serverHttpUrl;
+
+public HttpKerberosRequestInterceptor(String principal, String host,
+    String serverHttpUrl) {
+  this.principal = principal;
+  this.host = host;
+  this.serverHttpUrl = serverHttpUrl;
+}
+
+@Override
+public void process(HttpRequest httpRequest, HttpContext httpContext)
+    throws HttpException, IOException {
+  String kerberosAuthHeader;
+  try {
+    // Generate the service ticket for sending to the server.
+    kerberosAuthHeader = HttpAuthUtils.getKerberosServiceTicket(
+        principal, host, serverHttpUrl);
+    // Set the session key token (Base64 encoded) in the headers
+    httpRequest.addHeader(HttpAuthUtils.AUTHORIZATION + ": " +
+        HttpAuthUtils.NEGOTIATE + " ", kerberosAuthHeader);
+  } catch (Exception e) {
+    throw new HttpException(e.getMessage(), e);
+  }
+}
+
+}

Modified: hive/trunk/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/pom.xml?rev=1578348&r1=1578347&r2=1578348&view=diff
==============================================================================
--- hive/trunk/pom.xml (original)
+++ hive/trunk/pom.xml Mon Mar 17 12:56:10 2014
@@ -107,8 +107,8 @@
     <hbase.hadoop1.version>0.96.0-hadoop1</hbase.hadoop1.version>
     <hbase.hadoop2.version>0.96.0-hadoop2</hbase.hadoop2.version>
     <!-- httpcomponents are not always in version sync -->
-    <httpcomponents.client.version>4.3.2</httpcomponents.client.version>
-    <httpcomponents.core.version>4.3.1</httpcomponents.core.version>
+    <httpcomponents.client.version>4.2.5</httpcomponents.client.version>
+    <httpcomponents.core.version>4.2.5</httpcomponents.core.version>
     <jackson.version>1.9.2</jackson.version>
     <javaewah.version>0.3.2</javaewah.version>
     <javolution.version>5.5.1</javolution.version>

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java?rev=1578348&r1=1578347&r2=1578348&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java Mon Mar 17 12:56:10 2014
@@ -32,9 +32,9 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.thrift.TProcessorFactory;
 import org.apache.thrift.transport.TSSLTransportFactory;
 import org.apache.thrift.transport.TServerSocket;
@@ -55,7 +55,7 @@ public class HiveAuthFactory {
     KERBEROS("KERBEROS"),
     CUSTOM("CUSTOM");
 
-    private String authType; // Auth type for SASL
+    private String authType;
 
     AuthTypes(String authType) {
       this.authType = authType;
@@ -69,6 +69,7 @@ public class HiveAuthFactory {
 
   private HadoopThriftAuthBridge.Server saslServer = null;
   private String authTypeStr;
+  private String transportMode;
   private final HiveConf conf;
 
   public static final String HS2_PROXY_USER = "hive.server2.proxy.user";
@@ -76,40 +77,48 @@ public class HiveAuthFactory {
 
   public HiveAuthFactory() throws TTransportException {
     conf = new HiveConf();
-
+    transportMode = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE);
     authTypeStr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION);
-    if (authTypeStr == null) {
-      authTypeStr = AuthTypes.NONE.getAuthName();
+
+    // In http mode we use NOSASL as the default auth type
+    if (transportMode.equalsIgnoreCase("http")) {
+      if (authTypeStr == null) {
+        authTypeStr = AuthTypes.NOSASL.getAuthName();
+      }
     }
-    if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())
-        && ShimLoader.getHadoopShims().isSecureShimImpl()) {
-      saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(
-        conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB),
-        conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL)
-        );
-      // start delegation token manager
-      try {
-        saslServer.startDelegationTokenSecretManager(conf, null);
-      } catch (IOException e) {
-        throw new TTransportException("Failed to start token manager", e);
+    else {
+      if (authTypeStr == null) {
+        authTypeStr = AuthTypes.NONE.getAuthName();
+      }
+      if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())
+          && ShimLoader.getHadoopShims().isSecureShimImpl()) {
+        saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(
+            conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB),
+            conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL)
+            );
+        // start delegation token manager
+        try {
+          saslServer.startDelegationTokenSecretManager(conf, null);
+        } catch (IOException e) {
+          throw new TTransportException("Failed to start token manager", e);
+        }
       }
-
     }
   }
 
   public Map<String, String> getSaslProperties() {
     Map<String, String> saslProps = new HashMap<String, String>();
     SaslQOP saslQOP =
-            SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP));
+        SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP));
     // hadoop.rpc.protection being set to a higher level than hive.server2.thrift.rpc.protection
     // does not make sense in most situations. Log warning message in such cases.
     Map<String, String> hadoopSaslProps =  ShimLoader.getHadoopThriftAuthBridge().
-            getHadoopSaslProperties(conf);
+        getHadoopSaslProperties(conf);
     SaslQOP hadoopSaslQOP = SaslQOP.fromString(hadoopSaslProps.get(Sasl.QOP));
     if(hadoopSaslQOP.ordinal() > saslQOP.ordinal()) {
       LOG.warn(MessageFormat.format("\"hadoop.rpc.protection\" is set to higher security level " +
-              "{0} then {1} which is set to {2}", hadoopSaslQOP.toString(),
-              ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP.varname, saslQOP.toString()));
+          "{0} then {1} which is set to {2}", hadoopSaslQOP.toString(),
+          ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP.varname, saslQOP.toString()));
     }
     saslProps.put(Sasl.QOP, saslQOP.toString());
     saslProps.put(Sasl.SERVER_AUTH, "true");
@@ -142,10 +151,15 @@ public class HiveAuthFactory {
 
   public TProcessorFactory getAuthProcFactory(ThriftCLIService service)
       throws LoginException {
-    if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
-      return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service);
-    } else {
-      return PlainSaslHelper.getPlainProcessorFactory(service);
+    if (transportMode.equalsIgnoreCase("http")) {
+      return HttpAuthUtils.getAuthProcFactory(service);
+    }
+    else {
+      if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
+        return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service);
+      } else {
+        return PlainSaslHelper.getPlainProcessorFactory(service);
+      }
     }
   }
 
@@ -161,14 +175,11 @@ public class HiveAuthFactory {
     return saslServer != null ? saslServer.getRemoteAddress().toString() : null;
   }
 
-  /* perform kerberos login using the hadoop shim API if the configuration is available */
+  // 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);
     String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB);
-    if (principal.isEmpty() && keyTabFile.isEmpty()) {
-      // no security configuration available
-      return;
-    } else if (!principal.isEmpty() && !keyTabFile.isEmpty()) {
+    if (!principal.isEmpty() && !keyTabFile.isEmpty()) {
       ShimLoader.getHadoopShims().loginUserFromKeytab(principal, keyTabFile);
     } else {
       throw new IOException ("HiveServer2 kerberos principal or keytab is not correctly configured");
@@ -289,7 +300,7 @@ public class HiveAuthFactory {
       }
       if (!proxyUser.equalsIgnoreCase(realUser)) {
         ShimLoader.getHadoopShims().
-          authorizeProxyAccess(proxyUser, sessionUgi, ipAddress, hiveConf);
+        authorizeProxyAccess(proxyUser, sessionUgi, ipAddress, hiveConf);
       }
     } catch (IOException e) {
       throw new HiveSQLException("Failed to validate proxy privilage of " + realUser +

Added: 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=1578348&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java Mon Mar 17 12:56:10 2014
@@ -0,0 +1,191 @@
+/**
+ * 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 java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.service.cli.thrift.TCLIService;
+import org.apache.hive.service.cli.thrift.TCLIService.Iface;
+import org.apache.hive.service.cli.thrift.ThriftCLIService;
+import org.apache.http.protocol.BasicHttpContext;
+import org.apache.http.protocol.HttpContext;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.TProcessorFactory;
+import org.apache.thrift.transport.TTransport;
+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;
+
+/**
+ *
+ * Utility functions for http mode authentication
+ *
+ */
+public 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";
+
+  public static class HttpCLIServiceProcessorFactory extends TProcessorFactory {
+    private final ThriftCLIService service;
+    private final HiveConf hiveConf;
+    private final boolean isDoAsEnabled;
+
+    public HttpCLIServiceProcessorFactory(ThriftCLIService service) {
+      super(null);
+      this.service = service;
+      this.hiveConf = service.getHiveConf();
+      this.isDoAsEnabled = hiveConf.getBoolVar(
+          HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS);
+    }
+
+    @Override
+    public TProcessor getProcessor(TTransport trans) {
+      TProcessor baseProcessor = new TCLIService.Processor<Iface>(service);
+      return isDoAsEnabled ? new HttpCLIServiceUGIProcessor(baseProcessor) :
+        baseProcessor;
+    }
+  }
+
+  public static TProcessorFactory getAuthProcFactory(ThriftCLIService service) {
+    return new HttpCLIServiceProcessorFactory(service);
+  }
+
+  /**
+   *
+   * @return Stringified Base64 encoded kerberosAuthHeader on success
+   * @throws GSSException
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public static String getKerberosServiceTicket(String principal,
+      String host, String serverHttpUrl)
+          throws GSSException, 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.getShortUserName(), serverHttpUrl));
+  }
+
+  /**
+   * Get server pricipal and verify that hostname is present
+   * @return
+   * @throws IOException
+   */
+  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
+   * @throws IOException
+   */
+  public static UserGroupInformation getClientUGI(String authType)
+      throws IOException {
+    return ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf(authType);
+  }
+
+  /**
+   *
+   * HttpKerberosClientAction
+   *
+   */
+  public static class HttpKerberosClientAction implements
+  PrivilegedExceptionAction<String> {
+    String serverPrincipal;
+    String clientUserName;
+    String serverHttpUrl;
+    private final Base64 base64codec;
+    public static final String HTTP_RESPONSE = "HTTP_RESPONSE";
+    public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL";
+    private final HttpContext httpContext;
+
+    public HttpKerberosClientAction(String serverPrincipal,
+        String clientUserName, String serverHttpUrl) {
+      this.serverPrincipal = serverPrincipal;
+      this.clientUserName = clientUserName;
+      this.serverHttpUrl = serverHttpUrl;
+      this.base64codec = new Base64(0);
+      this.httpContext = new BasicHttpContext();
+      httpContext.setAttribute(SERVER_HTTP_URL, serverHttpUrl);
+    }
+
+    @Override
+    public String run() throws Exception {
+      // This Oid for Kerberos GSS-API mechanism.
+      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
+       */
+      GSSContext gssContext = manager.createContext(serverName,
+          mechOid, clientCreds, GSSContext.DEFAULT_LIFETIME);
+
+      // Mutual authentication not r
+      gssContext.requestMutualAuth(false);
+
+      // Estabilish context
+      byte[] inToken = new byte[0];
+      byte[] outToken;
+
+      outToken = gssContext.initSecContext(inToken, 0, inToken.length);
+
+      gssContext.dispose();
+      // Base64 encoded and stringified token for server
+      String authHeaderBase64String = new String(base64codec.encode(outToken));
+      return authHeaderBase64String;
+    }
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java?rev=1578348&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java Mon Mar 17 12:56:10 2014
@@ -0,0 +1,42 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+
+package org.apache.hive.service.auth;
+
+public class HttpAuthenticationException extends Exception{
+  static final long serialVersionUID = 0;
+
+  /**
+   * @param cause original exception.
+   */
+  public HttpAuthenticationException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * @param msg exception message.
+   */
+  public HttpAuthenticationException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * @param msg exception message.
+   * @param cause original exception.
+   */
+  public HttpAuthenticationException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java?rev=1578348&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java Mon Mar 17 12:56:10 2014
@@ -0,0 +1,83 @@
+/**
+ * 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 java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hive.shims.HadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.service.cli.session.SessionManager;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.protocol.TProtocol;
+
+/**
+ *
+ * Wraps the underlying thrift processor's process call,
+ * to assume the client user's UGI/Subject for the doAs calls.
+ * Gets the client's username from a threadlocal in SessionManager which is
+ * set in the ThriftHttpServlet, and constructs a client UGI object from that.
+ *
+ */
+
+public class HttpCLIServiceUGIProcessor implements TProcessor {
+
+  private final TProcessor underlyingProcessor;
+  private final HadoopShims shim;
+
+  public HttpCLIServiceUGIProcessor(TProcessor underlyingProcessor) {
+    this.underlyingProcessor = underlyingProcessor;
+    this.shim = ShimLoader.getHadoopShims();
+  }
+
+  @Override
+  public boolean process(final TProtocol in, final TProtocol out) throws TException {
+    /**
+     * Build the client UGI from threadlocal username [SessionManager.getUserName()].
+     * The threadlocal username is set in the ThriftHttpServlet.
+     */
+    UserGroupInformation clientUgi = null;
+    try {
+      clientUgi = shim.createRemoteUser(SessionManager.getUserName(), new ArrayList<String>());
+      return shim.doAs(clientUgi, new PrivilegedExceptionAction<Boolean>() {
+        @Override
+        public Boolean run() {
+          try {
+            return underlyingProcessor.process(in, out);
+          } catch (TException te) {
+            throw new RuntimeException(te);
+          }
+        }
+      });
+    }
+    catch (RuntimeException rte) {
+      if (rte.getCause() instanceof TException) {
+        throw (TException)rte.getCause();
+      }
+      throw rte;
+    } catch (InterruptedException ie) {
+      throw new RuntimeException(ie); // unexpected!
+    } catch (IOException ioe) {
+      throw new RuntimeException(ioe); // unexpected!
+    }
+  }
+}

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java?rev=1578348&r1=1578347&r2=1578348&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java Mon Mar 17 12:56:10 2014
@@ -34,11 +34,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.ServiceException;
 import org.apache.hive.service.auth.HiveAuthFactory;
@@ -64,8 +66,7 @@ public class CLIService extends Composit
   private HiveConf hiveConf;
   private SessionManager sessionManager;
   private IMetaStoreClient metastoreClient;
-  private String serverUserName = null;
-
+  private UserGroupInformation serviceUGI;
 
   public CLIService() {
     super("CLIService");
@@ -74,21 +75,29 @@ public class CLIService extends Composit
   @Override
   public synchronized void init(HiveConf hiveConf) {
     this.hiveConf = hiveConf;
-
     sessionManager = new SessionManager();
     addService(sessionManager);
-    try {
-      HiveAuthFactory.loginFromKeytab(hiveConf);
-      serverUserName = ShimLoader.getHadoopShims().
-          getShortUserName(ShimLoader.getHadoopShims().getUGIForConf(hiveConf));
-    } catch (IOException e) {
-      throw new ServiceException("Unable to login to kerberos with given principal/keytab", e);
-    } catch (LoginException e) {
-      throw new ServiceException("Unable to login to kerberos with given principal/keytab", e);
+    /**
+     * If auth mode is Kerberos, do a kerberos login for the service from the keytab
+     */
+    if (hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION).equalsIgnoreCase(
+        HiveAuthFactory.AuthTypes.KERBEROS.toString())) {
+      try {
+        HiveAuthFactory.loginFromKeytab(hiveConf);
+        this.serviceUGI = ShimLoader.getHadoopShims().getUGIForConf(hiveConf);
+      } catch (IOException e) {
+        throw new ServiceException("Unable to login to kerberos with given principal/keytab", e);
+      } catch (LoginException e) {
+        throw new ServiceException("Unable to login to kerberos with given principal/keytab", e);
+      }
     }
     super.init(hiveConf);
   }
 
+  public UserGroupInformation getServiceUGI() {
+    return this.serviceUGI;
+  }
+
   @Override
   public synchronized void start() {
     super.start();
@@ -441,7 +450,7 @@ public class CLIService extends Composit
   public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
       String tokenStr) throws HiveSQLException {
     sessionManager.getSession(sessionHandle).
-        cancelDelegationToken(authFactory, tokenStr);
+    cancelDelegationToken(authFactory, tokenStr);
     LOG.info(sessionHandle  + ": cancelDelegationToken()");
   }
 

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java?rev=1578348&r1=1578347&r2=1578348&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java Mon Mar 17 12:56:10 2014
@@ -101,7 +101,7 @@ public class SessionManager extends Comp
 
   public SessionHandle openSession(TProtocolVersion protocol, String username, String password,
       Map<String, String> sessionConf, boolean withImpersonation, String delegationToken)
-      throws HiveSQLException {
+          throws HiveSQLException {
     HiveSession session;
     if (withImpersonation) {
       HiveSessionImplwithUGI hiveSessionUgi = new HiveSessionImplwithUGI(protocol, username, password,
@@ -145,6 +145,44 @@ public class SessionManager extends Comp
     return operationManager;
   }
 
+  private static ThreadLocal<String> threadLocalIpAddress = new ThreadLocal<String>() {
+    @Override
+    protected synchronized String initialValue() {
+      return null;
+    }
+  };
+
+  public static void setIpAddress(String ipAddress) {
+    threadLocalIpAddress.set(ipAddress);
+  }
+
+  public static void clearIpAddress() {
+    threadLocalIpAddress.remove();
+  }
+
+  public static String getIpAddress() {
+    return threadLocalIpAddress.get();
+  }
+
+  private static ThreadLocal<String> threadLocalUserName = new ThreadLocal<String>(){
+    @Override
+    protected synchronized String initialValue() {
+      return null;
+    }
+  };
+
+  public static void setUserName(String userName) {
+    threadLocalUserName.set(userName);
+  }
+
+  public static void clearUserName() {
+    threadLocalUserName.remove();
+  }
+
+  public static String getUserName() {
+    return threadLocalUserName.get();
+  }
+
   // execute session hooks
   private void executeSessionHooks(HiveSession session) throws Exception {
     List<HiveSessionHook> sessionHooks = HookUtils.getHooks(hiveConf,

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java?rev=1578348&r1=1578347&r2=1578348&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java Mon Mar 17 12:56:10 2014
@@ -220,12 +220,20 @@ public abstract class ThriftCLIService e
     return getProxyUser(userName, req.getConfiguration(), getIpAddress());
   }
 
+  /**
+   * Create a session handle
+   * @param req
+   * @param res
+   * @return
+   * @throws HiveSQLException
+   * @throws LoginException
+   * @throws IOException
+   */
   SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res)
       throws HiveSQLException, LoginException, IOException {
-
     String userName = getUserName(req);
-    TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, req.getClient_protocol());
-
+    TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION,
+        req.getClient_protocol());
     SessionHandle sessionHandle;
     if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) &&
         (userName != null)) {

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java?rev=1578348&r1=1578347&r2=1578348&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java Mon Mar 17 12:56:10 2014
@@ -20,9 +20,13 @@ package org.apache.hive.service.cli.thri
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
+import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
 import org.apache.hive.service.cli.CLIService;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.TProcessorFactory;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocolFactory;
 import org.apache.thrift.server.TServlet;
@@ -59,32 +63,20 @@ public class ThriftHttpCLIService extend
       minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MIN_WORKER_THREADS);
       maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_WORKER_THREADS);
 
-      String httpPath =  hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH);
-      // The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on.
-      // httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*"
-      if(httpPath == null || httpPath.equals("")) {
-        httpPath = "/*";
-      }
-      else {
-        if(!httpPath.startsWith("/")) {
-          httpPath = "/" + httpPath;
-        }
-        if(httpPath.endsWith("/")) {
-          httpPath = httpPath + "*";
-        }
-        if(!httpPath.endsWith("/*")) {
-          httpPath = httpPath + "/*";
-        }
-      }
+      String httpPath =  getHttpPath(hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH));
 
       httpServer = new org.eclipse.jetty.server.Server();
       QueuedThreadPool threadPool = new QueuedThreadPool();
       threadPool.setMinThreads(minWorkerThreads);
       threadPool.setMaxThreads(maxWorkerThreads);
       httpServer.setThreadPool(threadPool);
-      SelectChannelConnector connector;
-      Boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL);
+
+      SelectChannelConnector connector = new SelectChannelConnector();;
+      boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL);
       String schemeName = useSsl ? "https" : "http";
+      String authType = hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION);
+      // Set during the init phase of HiveServer2 if auth mode is kerberos
+      UserGroupInformation serviceUGI = cliService.getServiceUGI();
 
       if (useSsl) {
         String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim();
@@ -97,8 +89,6 @@ public class ThriftHttpCLIService extend
         sslContextFactory.setKeyStorePath(keyStorePath);
         sslContextFactory.setKeyStorePassword(keyStorePassword);
         connector = new SslSelectChannelConnector(sslContextFactory);
-      } else {
-        connector = new SelectChannelConnector();
       }
 
       connector.setPort(portNum);
@@ -106,14 +96,18 @@ public class ThriftHttpCLIService extend
       connector.setReuseAddress(!Shell.WINDOWS);
       httpServer.addConnector(connector);
 
-      TCLIService.Processor<ThriftCLIService> processor =
-          new TCLIService.Processor<ThriftCLIService>(new EmbeddedThriftBinaryCLIService());
+      hiveAuthFactory = new HiveAuthFactory();
+      TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this);
+      TProcessor processor = processorFactory.getProcessor(null);
 
       TProtocolFactory protocolFactory = new TBinaryProtocol.Factory();
-      TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory);
+      TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory,
+          authType, serviceUGI);
 
-      final ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+      final ServletContextHandler context = new ServletContextHandler(
+          ServletContextHandler.SESSIONS);
       context.setContextPath("/");
+
       httpServer.setHandler(context);
       context.addServlet(new ServletHolder(thriftHttpServlet), httpPath);
 
@@ -130,39 +124,53 @@ public class ThriftHttpCLIService extend
   }
 
   /**
+   * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on.
+   * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*"
+   * @param httpPath
+   * @return
+   */
+  private String getHttpPath(String httpPath) {
+    if(httpPath == null || httpPath.equals("")) {
+      httpPath = "/*";
+    }
+    else {
+      if(!httpPath.startsWith("/")) {
+        httpPath = "/" + httpPath;
+      }
+      if(httpPath.endsWith("/")) {
+        httpPath = httpPath + "*";
+      }
+      if(!httpPath.endsWith("/*")) {
+        httpPath = httpPath + "/*";
+      }
+    }
+    return httpPath;
+  }
+
+  /**
    * Verify that this configuration is supported by transportMode of HTTP
    * @param hiveConf
    */
   private static void verifyHttpConfiguration(HiveConf hiveConf) {
     String authType = hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION);
 
-    // error out if KERBEROS or LDAP mode is being used, it is not supported
-    if(authType.equalsIgnoreCase(AuthTypes.KERBEROS.toString()) ||
-        authType.equalsIgnoreCase(AuthTypes.LDAP.toString()) ||
-        authType.equalsIgnoreCase(AuthTypes.CUSTOM.toString())) {
+    // Error out if KERBEROS auth mode is being used and use SSL is also set to true
+    if(authType.equalsIgnoreCase(AuthTypes.KERBEROS.toString()) &&
+        hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL)) {
       String msg = ConfVars.HIVE_SERVER2_AUTHENTICATION + " setting of " +
-          authType + " is currently not supported with " +
-          ConfVars.HIVE_SERVER2_TRANSPORT_MODE + " setting of http";
+          authType + " is not supported with " +
+          ConfVars.HIVE_SERVER2_USE_SSL + " set to true";
       LOG.fatal(msg);
       throw new RuntimeException(msg);
     }
 
-    // Throw exception here
+    // Warn that SASL is not used in http mode
     if(authType.equalsIgnoreCase(AuthTypes.NONE.toString())) {
       // NONE in case of thrift mode uses SASL
       LOG.warn(ConfVars.HIVE_SERVER2_AUTHENTICATION + " setting to " +
-          authType + ". SASL is not supported with http transportMode," +
+          authType + ". SASL is not supported with http transport mode," +
           " so using equivalent of " + AuthTypes.NOSASL);
     }
-
-    // doAs is currently not supported with http
-    if(hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
-      String msg = ConfVars.HIVE_SERVER2_ENABLE_DOAS + " setting of " +
-          "true is currently not supported with " +
-          ConfVars.HIVE_SERVER2_TRANSPORT_MODE + " setting of http";
-      LOG.fatal(msg);
-      throw new RuntimeException(msg);
-    }
   }
 
 }
\ No newline at end of file

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=1578348&r1=1578347&r2=1578348&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 Mon Mar 17 12:56:10 2014
@@ -19,6 +19,7 @@
 package org.apache.hive.service.cli.thrift;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
@@ -28,61 +29,252 @@ import org.apache.commons.codec.binary.B
 import org.apache.commons.codec.binary.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.service.auth.AuthenticationProviderFactory;
+import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods;
+import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.auth.HttpAuthUtils;
+import org.apache.hive.service.auth.HttpAuthenticationException;
+import org.apache.hive.service.auth.PasswdAuthenticationProvider;
+import org.apache.hive.service.cli.session.SessionManager;
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.protocol.TProtocolFactory;
 import org.apache.thrift.server.TServlet;
-
+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;
+
+/**
+ *
+ * ThriftHttpServlet
+ *
+ */
 public class ThriftHttpServlet extends TServlet {
 
   private static final long serialVersionUID = 1L;
   public static final Log LOG = LogFactory.getLog(ThriftHttpServlet.class.getName());
+  private final String authType;
+  private final UserGroupInformation serviceUGI;
 
-  public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory) {
+  public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory,
+      String authType, UserGroupInformation serviceUGI) {
     super(processor, protocolFactory);
+    this.authType = authType;
+    this.serviceUGI = serviceUGI;
   }
 
   @Override
   protected void doPost(HttpServletRequest request, HttpServletResponse response)
       throws ServletException, IOException {
-    logRequestHeader(request);
-    super.doPost(request, response);
-  }
+    String clientUserName;
+    try {
+      // For a kerberos setup
+      if(isKerberosAuthMode(authType)) {
+        clientUserName = doKerberosAuth(request, serviceUGI);
+      }
+      else {
+        clientUserName = doPasswdAuth(request, authType);
+      }
 
-  protected void logRequestHeader(HttpServletRequest request) {
-    String authHeaderBase64 = request.getHeader("Authorization");
-    if(authHeaderBase64 == null) {
-      LOG.warn("ThriftHttpServlet: no HTTP Authorization header");
+      LOG.info("Client username: " + clientUserName);
+      
+      // Set the thread local username to be used for doAs if true
+      SessionManager.setUserName(clientUserName);
+      super.doPost(request, response);
     }
-    else {
-      if(!authHeaderBase64.startsWith("Basic")) {
-        LOG.warn("ThriftHttpServlet: HTTP Authorization header exists but is not Basic.");
+    catch (HttpAuthenticationException e) {
+      // Send a 403 to the client
+      LOG.error("Error: ", e);
+      response.setContentType("application/x-thrift");
+      response.setStatus(HttpServletResponse.SC_FORBIDDEN);
+      // Send the response back to the client
+      response.getWriter().println("Authentication Error: " + e.getMessage());
+    }
+    finally {
+      // Clear the thread local username since we set it in each http request
+      SessionManager.clearUserName();
+    }
+  }
+
+  /**
+   * Do the LDAP/PAM authentication
+   * @param request
+   * @param authType
+   * @throws HttpAuthenticationException
+   */
+  private String doPasswdAuth(HttpServletRequest request, String authType)
+      throws HttpAuthenticationException {
+    String userName = getUsername(request, authType);
+    // No-op when authType is NOSASL
+    if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) {
+      try {
+        AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType);
+        PasswdAuthenticationProvider provider =
+            AuthenticationProviderFactory.getAuthenticationProvider(authMethod);
+        provider.Authenticate(userName, getPassword(request, authType));
+
+      } catch (Exception e) {
+        throw new HttpAuthenticationException(e);
       }
-      else if(LOG.isDebugEnabled()) {
-        String authHeaderBase64_Payload = authHeaderBase64.substring("Basic ".length());
-        String authHeaderString = StringUtils.newStringUtf8(
-            Base64.decodeBase64(authHeaderBase64_Payload.getBytes()));
-        String[] creds = authHeaderString.split(":");
-        String username = null;
-        String password = null;
+    }
+    return userName;
+  }
 
-        if(creds.length >= 1) {
-          username = creds[0];
-        }
-        if(creds.length >= 2) {
-          password = creds[1];
-        }
-        if(password == null || password.equals("null") || password.equals("")) {
-          password = "<no password>";
+  /**
+   * Do the GSS-API kerberos authentication.
+   * We already have a logged in subject in the form of serviceUGI,
+   * which GSS-API will extract information from.
+   * @param request
+   * @return
+   * @throws HttpAuthenticationException
+   */
+  private String doKerberosAuth(HttpServletRequest request, 
+      UserGroupInformation serviceUGI) throws HttpAuthenticationException {
+    try {
+      return serviceUGI.doAs(new HttpKerberosServerAction(request, serviceUGI));
+    } catch (Exception e) {
+      throw new HttpAuthenticationException(e);
+    }
+  }
+
+  class HttpKerberosServerAction implements PrivilegedExceptionAction<String> {
+    HttpServletRequest request;
+    UserGroupInformation serviceUGI;
+    
+    HttpKerberosServerAction(HttpServletRequest request, 
+        UserGroupInformation serviceUGI) {
+      this.request = request;
+      this.serviceUGI = serviceUGI;
+    }
+
+    @Override
+    public String run() throws HttpAuthenticationException {
+   // Get own Kerberos credentials for accepting connection
+      GSSManager manager = GSSManager.getInstance();
+      GSSContext gssContext = null;
+      String serverPrincipal = getPrincipalWithoutRealm(
+          serviceUGI.getUserName());
+      try {
+        // This Oid for Kerberos GSS-API mechanism.
+        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");
+
+        // GSS name for server
+        GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid);
+
+        // GSS credentials for server
+        GSSCredential serverCreds = manager.createCredential(serverName,
+            GSSCredential.DEFAULT_LIFETIME,  mechOid, GSSCredential.ACCEPT_ONLY);
+
+        // 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()) {
+          throw new HttpAuthenticationException("Kerberos authentication failed: " +
+              "unable to establish context with the service ticket " +
+              "provided by the client.");
         }
         else {
-          // don't log the actual password.
-          password = "******";
+          return getPrincipalWithoutRealm(gssContext.getSrcName().toString());
+        }
+      }
+      catch (GSSException e) {
+        throw new HttpAuthenticationException("Kerberos authentication failed: ", e);
+      }
+      finally {
+        if (gssContext != null) {
+          try {
+            gssContext.dispose();
+          } catch (GSSException e) {
+            // No-op
+          }
         }
-        LOG.debug("HttpServlet:  HTTP Authorization header:: username=" + username +
-            " password=" + password);
       }
     }
+
+    private String getPrincipalWithoutRealm(String fullPrincipal) {
+      String names[] = fullPrincipal.split("[@]");
+      return names[0];
+    }
+  }
+
+  private String getUsername(HttpServletRequest request, String authType)
+      throws HttpAuthenticationException {
+    String creds[] = getAuthHeaderTokens(request, authType);
+    // Username must be present
+    if (creds[0] == null || creds[0].isEmpty()) {
+      throw new HttpAuthenticationException("Authorization header received " +
+          "from the client does not contain username.");
+    }
+    return creds[0];
+  }
+
+  private String getPassword(HttpServletRequest request, String authType)
+      throws HttpAuthenticationException {
+    String creds[] = getAuthHeaderTokens(request, authType);
+    // Password must be present
+    if (creds[1] == null || creds[1].isEmpty()) {
+      throw new HttpAuthenticationException("Authorization header received " +
+          "from the client does not contain username.");
+    }
+    return creds[1];
+  }
+
+  private String[] getAuthHeaderTokens(HttpServletRequest request,
+      String authType) throws HttpAuthenticationException {
+    String authHeaderBase64 = getAuthHeader(request, authType);
+    String authHeaderString = StringUtils.newStringUtf8(
+        Base64.decodeBase64(authHeaderBase64.getBytes()));
+    String[] creds = authHeaderString.split(":");
+    return creds;
   }
 
+  /**
+   * Returns the base64 encoded auth header payload
+   * @param request
+   * @param authType
+   * @return
+   * @throws HttpAuthenticationException
+   */
+  private String getAuthHeader(HttpServletRequest request, String authType)
+      throws HttpAuthenticationException {
+    String authHeader = request.getHeader(HttpAuthUtils.AUTHORIZATION);
+    // Each http request must have an Authorization header
+    if (authHeader == null || authHeader.isEmpty()) {
+      throw new HttpAuthenticationException("Authorization header received " +
+          "from the client is empty.");
+    }
+
+    String authHeaderBase64String;
+    int beginIndex;
+    if (isKerberosAuthMode(authType)) {
+      beginIndex = (HttpAuthUtils.NEGOTIATE + " ").length();
+    }
+    else {
+      beginIndex = (HttpAuthUtils.BASIC + " ").length();
+    }
+    authHeaderBase64String = authHeader.substring(beginIndex);
+    // Authorization header must have a payload
+    if (authHeaderBase64String == null || authHeaderBase64String.isEmpty()) {
+      throw new HttpAuthenticationException("Authorization header received " +
+          "from the client does not contain any data.");
+    }
+    return authHeaderBase64String;
+  }
+
+  private boolean isKerberosAuthMode(String authType) {
+    return authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString());
+  }
 }
 
+