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/10/01 21:55:26 UTC

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

Author: vgumashta
Date: Wed Oct  1 19:55:26 2014
New Revision: 1628821

URL: http://svn.apache.org/r1628821
Log:
HIVE-8299: HiveServer2 in http-kerberos & doAs=true is failing with org.apache.hadoop.security.AccessControlException (Vaibhav Gumashta reviewed by Thejas Nair)

Removed:
    hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java
Modified:
    hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.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

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=1628821&r1=1628820&r2=1628821&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 Wed Oct  1 19:55:26 2014
@@ -136,15 +136,17 @@ public class HiveAuthFactory {
     return transportFactory;
   }
 
+  /**
+   * Returns the thrift processor factory for HiveServer2 running in binary mode
+   * @param service
+   * @return
+   * @throws LoginException
+   */
   public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException {
-    if ("http".equalsIgnoreCase(transportMode)) {
-      return HttpAuthUtils.getAuthProcFactory(service);
+    if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
+      return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service);
     } else {
-      if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
-        return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service);
-      } else {
-        return PlainSaslHelper.getPlainProcessorFactory(service);
-      }
+      return PlainSaslHelper.getPlainProcessorFactory(service);
     }
   }
 

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=1628821&r1=1628820&r2=1628821&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 Wed Oct  1 19:55:26 2014
@@ -22,17 +22,10 @@ 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.GSSManager;
@@ -48,11 +41,7 @@ public final class HttpAuthUtils {
   public static final String AUTHORIZATION = "Authorization";
   public static final String BASIC = "Basic";
   public static final String NEGOTIATE = "Negotiate";
-
-  public static TProcessorFactory getAuthProcFactory(ThriftCLIService service) {
-    return new HttpCLIServiceProcessorFactory(service);
-  }
-
+  
   /**
    * @return Stringified Base64 encoded kerberosAuthHeader on success
    */
@@ -87,26 +76,6 @@ public final class HttpAuthUtils {
     throw new UnsupportedOperationException("Can't initialize class");
   }
 
-  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;
-      hiveConf = service.getHiveConf();
-      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 class HttpKerberosClientAction implements PrivilegedExceptionAction<String> {
 
     public static final String HTTP_RESPONSE = "HTTP_RESPONSE";

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java?rev=1628821&r1=1628820&r2=1628821&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java Wed Oct  1 19:55:26 2014
@@ -18,7 +18,6 @@
 
 package org.apache.hive.service.auth;
 
-import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Processor;
 import org.apache.hive.service.cli.thrift.TCLIService;
 import org.apache.hive.service.cli.thrift.TCLIService.Iface;
 import org.apache.thrift.TException;
@@ -43,7 +42,7 @@ import org.slf4j.LoggerFactory;
  */
 public class TSetIpAddressProcessor<I extends Iface> extends TCLIService.Processor<Iface> {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+  private static final Logger LOGGER = LoggerFactory.getLogger(TSetIpAddressProcessor.class.getName());
 
   public TSetIpAddressProcessor(Iface iface) {
     super(iface);

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=1628821&r1=1628820&r2=1628821&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 Wed Oct  1 19:55:26 2014
@@ -229,6 +229,23 @@ public class SessionManager extends Comp
     return openSession(protocol, username, password, ipAddress, sessionConf, false, null);
   }
 
+  /**
+   * Opens a new session and creates a session handle.
+   * The username passed to this method is the effective username.
+   * If withImpersonation is true (==doAs true) we wrap all the calls in HiveSession 
+   * within a UGI.doAs, where UGI corresponds to the effective user.
+   * @see org.apache.hive.service.cli.thrift.ThriftCLIService#getUserName() 
+   * 
+   * @param protocol
+   * @param username
+   * @param password
+   * @param ipAddress
+   * @param sessionConf
+   * @param withImpersonation
+   * @param delegationToken
+   * @return
+   * @throws HiveSQLException
+   */
   public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress,
       Map<String, String> sessionConf, boolean withImpersonation, String delegationToken)
           throws HiveSQLException {

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=1628821&r1=1628820&r2=1628821&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 Wed Oct  1 19:55:26 2014
@@ -262,6 +262,16 @@ public abstract class ThriftCLIService e
     return clientIpAddress;
   }
 
+  /**
+   * Returns the effective username.
+   * 1. If hive.server2.allow.user.substitution = false: the username of the connecting user 
+   * 2. If hive.server2.allow.user.substitution = true: the username of the end user,
+   * that the connecting user is trying to proxy for. 
+   * This includes a check whether the connecting user is allowed to proxy for the end user.
+   * @param req
+   * @return
+   * @throws HiveSQLException
+   */
   private String getUserName(TOpenSessionReq req) throws HiveSQLException {
     String userName = null;
     // Kerberos

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=1628821&r1=1628820&r2=1628821&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 Wed Oct  1 19:55:26 2014
@@ -31,6 +31,7 @@ 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.hive.service.cli.thrift.TCLIService.Iface;
 import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup;
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.TProcessorFactory;
@@ -102,8 +103,7 @@ public class ThriftHttpCLIService extend
 
       // Thrift configs
       hiveAuthFactory = new HiveAuthFactory(hiveConf);
-      TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this);
-      TProcessor processor = processorFactory.getProcessor(null);
+      TProcessor processor = new TCLIService.Processor<Iface>(this);
       TProtocolFactory protocolFactory = new TBinaryProtocol.Factory();
       // Set during the init phase of HiveServer2 if auth mode is kerberos
       // UGI for the hive/_HOST (kerberos) principal