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/12 11:41:58 UTC

svn commit: r1576685 [6/6] - in /hive/trunk: beeline/ beeline/src/java/org/apache/hive/beeline/ beeline/src/test/org/apache/hive/beeline/ common/src/java/org/apache/hadoop/hive/conf/ conf/ data/files/ itests/hive-unit/src/test/java/org/apache/hive/jdbc...

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java Wed Mar 12 10:41:57 2014
@@ -21,6 +21,8 @@ package org.apache.hive.service.cli;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hive.service.auth.HiveAuthFactory;
+
 
 /**
  * EmbeddedCLIServiceClient.
@@ -188,4 +190,22 @@ public class EmbeddedCLIServiceClient ex
     return cliService.fetchResults(opHandle, orientation, maxRows);
   }
 
+
+  @Override
+  public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+         String owner, String renewer) throws HiveSQLException {
+    return cliService.getDelegationToken(sessionHandle, authFactory, owner, renewer);
+  }
+
+  @Override
+  public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+      String tokenStr) throws HiveSQLException {
+    cliService.cancelDelegationToken(sessionHandle, authFactory, tokenStr);
+  }
+
+  @Override
+  public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+      String tokenStr) throws HiveSQLException {
+    cliService.renewDelegationToken(sessionHandle, authFactory, tokenStr);
+  }
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/ICLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/ICLIService.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/ICLIService.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/ICLIService.java Wed Mar 12 10:41:57 2014
@@ -23,6 +23,8 @@ import java.util.Map;
 
 
 
+import org.apache.hive.service.auth.HiveAuthFactory;
+
 public interface ICLIService {
 
   public abstract SessionHandle openSession(String username, String password,
@@ -91,4 +93,14 @@ public interface ICLIService {
   public abstract RowSet fetchResults(OperationHandle opHandle)
       throws HiveSQLException;
 
+  public abstract String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+      String owner, String renewer) throws HiveSQLException;
+
+  public abstract void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+      String tokenStr) throws HiveSQLException;
+
+  public abstract void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+      String tokenStr) throws HiveSQLException;
+
+
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSession.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSession.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSession.java Wed Mar 12 10:41:57 2014
@@ -24,6 +24,7 @@ import java.util.Map;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.GetInfoType;
 import org.apache.hive.service.cli.GetInfoValue;
@@ -183,4 +184,17 @@ public interface HiveSession {
   public String getUserName();
 
   public void setUserName(String userName);
+
+  public String getIpAddress();
+
+  public void setIpAddress(String ipAddress);
+
+  public String getDelegationToken(HiveAuthFactory authFactory, String owner,
+      String renewer) throws HiveSQLException;
+
+  public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr)
+      throws HiveSQLException;
+
+  public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr)
+      throws HiveSQLException;
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java Wed Mar 12 10:41:57 2014
@@ -35,7 +35,9 @@ import org.apache.hadoop.hive.ql.exec.Fe
 import org.apache.hadoop.hive.ql.exec.ListSinkOperator;
 import org.apache.hadoop.hive.ql.history.HiveHistory;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.common.util.HiveVersionInfo;
+import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.GetInfoType;
 import org.apache.hive.service.cli.GetInfoValue;
@@ -67,6 +69,7 @@ public class HiveSessionImpl implements 
   private final String password;
   private final HiveConf hiveConf;
   private final SessionState sessionState;
+  private String ipAddress;
 
   private static final String FETCH_WORK_SERDE_CLASS =
       "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe";
@@ -79,11 +82,12 @@ public class HiveSessionImpl implements 
   private final Set<OperationHandle> opHandleSet = new HashSet<OperationHandle>();
 
   public HiveSessionImpl(TProtocolVersion protocol, String username, String password,
-      HiveConf serverhiveConf, Map<String, String> sessionConfMap) {
+      HiveConf serverhiveConf, Map<String, String> sessionConfMap, String ipAddress) {
     this.username = username;
     this.password = password;
     this.sessionHandle = new SessionHandle(protocol);
     this.hiveConf = new HiveConf(serverhiveConf);
+    this.ipAddress = ipAddress;
 
     //set conf properties specified by user from client side
     if (sessionConfMap != null) {
@@ -418,6 +422,7 @@ public class HiveSessionImpl implements 
   public String getUserName() {
     return username;
   }
+
   @Override
   public void setUserName(String userName) {
     this.username = userName;
@@ -479,4 +484,42 @@ public class HiveSessionImpl implements 
   protected HiveSession getSession() {
     return this;
   }
+
+  @Override
+  public String getIpAddress() {
+    return ipAddress;
+  }
+
+  @Override
+  public void setIpAddress(String ipAddress) {
+    this.ipAddress = ipAddress;
+  }
+
+  @Override
+  public String getDelegationToken(HiveAuthFactory authFactory, String owner, String renewer)
+      throws HiveSQLException {
+    HiveAuthFactory.verifyProxyAccess(getUsername(), owner, getIpAddress(), getHiveConf());
+    return authFactory.getDelegationToken(owner, renewer);
+  }
+
+  @Override
+  public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr)
+      throws HiveSQLException {
+    HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr),
+        getIpAddress(), getHiveConf());
+    authFactory.cancelDelegationToken(tokenStr);
+  }
+
+  @Override
+  public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr)
+      throws HiveSQLException {
+    HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr),
+        getIpAddress(), getHiveConf());
+    authFactory.renewDelegationToken(tokenStr);
+  }
+
+  // extract the real user from the given token string
+  private String getUserFromToken(HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException {
+    return authFactory.getUserFromToken(tokenStr);
+  }
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java Wed Mar 12 10:41:57 2014
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.metadat
 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.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.thrift.TProtocolVersion;
 
@@ -43,8 +44,9 @@ public class HiveSessionImplwithUGI exte
   private HiveSession proxySession = null;
 
   public HiveSessionImplwithUGI(TProtocolVersion protocol, String username, String password,
-      HiveConf hiveConf, Map<String, String> sessionConf, String delegationToken) throws HiveSQLException {
-    super(protocol, username, password, hiveConf, sessionConf);
+      HiveConf hiveConf, Map<String, String> sessionConf, String ipAddress,
+       String delegationToken) throws HiveSQLException {
+    super(protocol, username, password, hiveConf, sessionConf, ipAddress);
     setSessionUGI(username);
     setDelegationToken(delegationToken);
   }
@@ -148,5 +150,22 @@ public class HiveSessionImplwithUGI exte
     this.proxySession = proxySession;
   }
 
+  @Override
+  public String getDelegationToken(HiveAuthFactory authFactory, String owner,
+      String renewer) throws HiveSQLException {
+    return authFactory.getDelegationToken(owner, renewer);
+  }
+
+  @Override
+  public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr)
+      throws HiveSQLException {
+    authFactory.cancelDelegationToken(tokenStr);
+  }
+
+  @Override
+  public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr)
+      throws HiveSQLException {
+    authFactory.renewDelegationToken(tokenStr);
+  }
 
 }

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=1576685&r1=1576684&r2=1576685&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 Mar 12 10:41:57 2014
@@ -18,6 +18,7 @@
 
 package org.apache.hive.service.cli.session;
 
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -31,6 +32,9 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.hooks.HookUtils;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
@@ -101,17 +105,14 @@ public class SessionManager extends Comp
   public SessionHandle openSession(TProtocolVersion protocol, String username, String password,
       Map<String, String> sessionConf, boolean withImpersonation, String delegationToken)
       throws HiveSQLException {
-    if (username == null) {
-      username = threadLocalUserName.get();
-    }
     HiveSession session;
     if (withImpersonation) {
       HiveSessionImplwithUGI hiveSessionUgi = new HiveSessionImplwithUGI(protocol, username, password,
-        hiveConf, sessionConf, delegationToken);
+        hiveConf, sessionConf, threadLocalIpAddress.get(), delegationToken);
       session = HiveSessionProxy.getProxy(hiveSessionUgi, hiveSessionUgi.getSessionUgi());
       hiveSessionUgi.setProxySession(session);
     } else {
-      session = new HiveSessionImpl(protocol, username, password, hiveConf, sessionConf);
+      session = new HiveSessionImpl(protocol, username, password, hiveConf, sessionConf, threadLocalIpAddress.get());
     }
     session.setSessionManager(this);
     session.setOperationManager(operationManager);
@@ -161,6 +162,10 @@ public class SessionManager extends Comp
     threadLocalIpAddress.remove();
   }
 
+  public static String getIpAddress() {
+    return threadLocalIpAddress.get();
+  }
+
   private static ThreadLocal<String> threadLocalUserName = new ThreadLocal<String>(){
     @Override
     protected synchronized String initialValue() {
@@ -176,6 +181,10 @@ public class SessionManager extends Comp
     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,
@@ -190,3 +199,4 @@ public class SessionManager extends Comp
   }
 
 }
+

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=1576685&r1=1576684&r2=1576685&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 Mar 12 10:41:57 2014
@@ -29,6 +29,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.service.AbstractService;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.CLIService;
@@ -41,6 +43,7 @@ import org.apache.hive.service.cli.Opera
 import org.apache.hive.service.cli.RowSet;
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.TableSchema;
+import org.apache.hive.service.cli.session.SessionManager;
 import org.apache.thrift.TException;
 import org.apache.thrift.server.TServer;
 
@@ -111,6 +114,75 @@ public abstract class ThriftCLIService e
     super.stop();
   }
 
+  @Override
+  public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req)
+      throws TException {
+    TGetDelegationTokenResp resp = new TGetDelegationTokenResp();
+
+    if (hiveAuthFactory == null) {
+      resp.setStatus(unsecureTokenErrorStatus());
+    } else {
+      try {
+        String token = cliService.getDelegationToken(
+            new SessionHandle(req.getSessionHandle()),
+            hiveAuthFactory, req.getOwner(), req.getRenewer());
+        resp.setDelegationToken(token);
+        resp.setStatus(OK_STATUS);
+      } catch (HiveSQLException e) {
+        LOG.error("Error obtaining delegation token", e);
+        TStatus tokenErrorStatus = HiveSQLException.toTStatus(e);
+        tokenErrorStatus.setSqlState("42000");
+        resp.setStatus(tokenErrorStatus);
+      }
+    }
+    return resp;
+  }
+
+  @Override
+  public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req)
+      throws TException {
+    TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp();
+
+    if (hiveAuthFactory == null) {
+      resp.setStatus(unsecureTokenErrorStatus());
+    } else {
+      try {
+        cliService.cancelDelegationToken(new SessionHandle(req.getSessionHandle()),
+            hiveAuthFactory, req.getDelegationToken());
+        resp.setStatus(OK_STATUS);
+      } catch (HiveSQLException e) {
+        LOG.error("Error canceling delegation token", e);
+        resp.setStatus(HiveSQLException.toTStatus(e));
+      }
+    }
+    return resp;
+  }
+
+  @Override
+  public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req)
+      throws TException {
+    TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp();
+    if (hiveAuthFactory == null) {
+      resp.setStatus(unsecureTokenErrorStatus());
+    } else {
+      try {
+        cliService.renewDelegationToken(new SessionHandle(req.getSessionHandle()),
+            hiveAuthFactory, req.getDelegationToken());
+        resp.setStatus(OK_STATUS);
+      } catch (HiveSQLException e) {
+        LOG.error("Error obtaining renewing token", e);
+        resp.setStatus(HiveSQLException.toTStatus(e));
+      }
+    }
+    return resp;
+  }
+
+  private TStatus unsecureTokenErrorStatus() {
+    TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS);
+    errorStatus.setErrorMessage("Delegation token only supported over remote " +
+        "client with kerberos authentication");
+    return errorStatus;
+  }
 
   @Override
   public TOpenSessionResp OpenSession(TOpenSessionReq req) throws TException {
@@ -129,13 +201,25 @@ public abstract class ThriftCLIService e
     return resp;
   }
 
-  private String getUserName(TOpenSessionReq req) {
+  private String getIpAddress() {
+    if (hiveAuthFactory != null) {
+      return hiveAuthFactory.getIpAddress();
+    }
+    return SessionManager.getIpAddress();
+  }
+
+  private String getUserName(TOpenSessionReq req) throws HiveSQLException {
+    String userName;
     if (hiveAuthFactory != null
         && hiveAuthFactory.getRemoteUser() != null) {
-      return hiveAuthFactory.getRemoteUser();
+      userName = hiveAuthFactory.getRemoteUser();
     } else {
-      return req.getUsername();
+      userName = SessionManager.getUserName();
     }
+    if (userName == null) {
+      userName = req.getUsername();
+    }
+    return getProxyUser(userName, req.getConfiguration(), getIpAddress());
   }
 
   SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res)
@@ -145,9 +229,8 @@ public abstract class ThriftCLIService e
     TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, req.getClient_protocol());
 
     SessionHandle sessionHandle;
-    if (cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION)
-        .equals(HiveAuthFactory.AuthTypes.KERBEROS.toString()) &&
-        cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
+    if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) &&
+        (userName != null)) {
       String delegationTokenStr = null;
       try {
         delegationTokenStr = cliService.getDelegationTokenFromMetaStore(userName);
@@ -420,4 +503,36 @@ public abstract class ThriftCLIService e
 
   @Override
   public abstract void run();
+
+  /**
+   * If the proxy user name is provided then check privileges to substitute the user.
+   * @param realUser
+   * @param sessionConf
+   * @param ipAddress
+   * @return
+   * @throws HiveSQLException
+   */
+  private String getProxyUser(String realUser, Map<String, String> sessionConf,
+      String ipAddress) throws HiveSQLException {
+    if (sessionConf == null || !sessionConf.containsKey(HiveAuthFactory.HS2_PROXY_USER)) {
+      return realUser;
+    }
+
+    // Extract the proxy user name and check if we are allowed to do the substitution
+    String proxyUser = sessionConf.get(HiveAuthFactory.HS2_PROXY_USER);
+    if (!hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ALLOW_USER_SUBSTITUTION)) {
+      throw new HiveSQLException("Proxy user substitution is not allowed");
+    }
+
+    // If there's no authentication, then directly substitute the user
+    if (HiveAuthFactory.AuthTypes.NONE.toString().
+        equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) {
+      return proxyUser;
+    }
+
+    // Verify proxy user privilege of the realUser for the proxyUser
+    HiveAuthFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hiveConf);
+    return proxyUser;
+  }
 }
+

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java Wed Mar 12 10:41:57 2014
@@ -21,6 +21,7 @@ package org.apache.hive.service.cli.thri
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.CLIServiceClient;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.GetInfoType;
@@ -33,6 +34,7 @@ import org.apache.hive.service.cli.RowSe
 import org.apache.hive.service.cli.RowSetFactory;
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.TableSchema;
+import org.apache.thrift.TException;
 
 /**
  * ThriftCLIServiceClient.
@@ -404,4 +406,48 @@ public class ThriftCLIServiceClient exte
     // TODO: set the correct default fetch size
     return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, 10000);
   }
+
+  @Override
+  public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+      String owner, String renewer) throws HiveSQLException {
+    TGetDelegationTokenReq req = new TGetDelegationTokenReq(
+        sessionHandle.toTSessionHandle(), owner, renewer);
+    try {
+      TGetDelegationTokenResp tokenResp = cliService.GetDelegationToken(req);
+      checkStatus(tokenResp.getStatus());
+      return tokenResp.getDelegationToken();
+    } catch (Exception e) {
+      throw new HiveSQLException(e);
+    }
+  }
+
+  @Override
+  public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+      String tokenStr) throws HiveSQLException {
+    TCancelDelegationTokenReq cancelReq = new TCancelDelegationTokenReq(
+          sessionHandle.toTSessionHandle(), tokenStr);
+    try {
+      TCancelDelegationTokenResp cancelResp =
+        cliService.CancelDelegationToken(cancelReq);
+      checkStatus(cancelResp.getStatus());
+      return;
+    } catch (TException e) {
+      throw new HiveSQLException(e);
+    }
+  }
+
+  @Override
+  public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory,
+      String tokenStr) throws HiveSQLException {
+    TRenewDelegationTokenReq cancelReq = new TRenewDelegationTokenReq(
+        sessionHandle.toTSessionHandle(), tokenStr);
+    try {
+      TRenewDelegationTokenResp renewResp =
+        cliService.RenewDelegationToken(cancelReq);
+      checkStatus(renewResp.getStatus());
+      return;
+    } catch (Exception e) {
+      throw new HiveSQLException(e);
+    }
+  }
 }

Modified: hive/trunk/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java (original)
+++ hive/trunk/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java Wed Mar 12 10:41:57 2014
@@ -45,6 +45,6 @@ public class TestPlainSaslHelper extends
     tcliService.init(hconf);
     TProcessorFactory procFactory = PlainSaslHelper.getPlainProcessorFactory(tcliService);
     assertEquals("doAs enabled processor for unsecure mode",
-        procFactory.getProcessor(null).getClass(), TUGIContainingProcessor.class);
+        procFactory.getProcessor(null).getClass(), TSetIpAddressProcessor.class);
   }
 }

Modified: hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java (original)
+++ hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java Wed Mar 12 10:41:57 2014
@@ -19,12 +19,16 @@
 package org.apache.hive.service.cli.session;
 
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import junit.framework.Assert;
 import junit.framework.TestCase;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.thrift.EmbeddedThriftBinaryCLIService;
@@ -34,7 +38,7 @@ import org.junit.Test;
 
 public class TestSessionHooks extends TestCase {
 
-  public static final String SESSION_USER_NAME = "user1";
+  private static String sessionUserName = "user1";
   private EmbeddedThriftBinaryCLIService service;
   private ThriftCLIServiceClient client;
 
@@ -44,7 +48,7 @@ public class TestSessionHooks extends Te
 
     @Override
     public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException {
-      Assert.assertEquals(sessionHookContext.getSessionUser(), SESSION_USER_NAME);
+      Assert.assertEquals(sessionHookContext.getSessionUser(), sessionUserName);
       String sessionHook = sessionHookContext.getSessionConf().
           getVar(ConfVars.HIVE_SERVER2_SESSION_HOOK);
       Assert.assertTrue(sessionHook.contains(this.getClass().getName()));
@@ -56,18 +60,36 @@ public class TestSessionHooks extends Te
   @Before
   public void setUp() throws Exception {
     super.setUp();
+    SessionHookTest.runCount.set(0);
     System.setProperty(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname,
         TestSessionHooks.SessionHookTest.class.getName());
     service = new EmbeddedThriftBinaryCLIService();
+    service.init(new HiveConf());
     client = new ThriftCLIServiceClient(service);
   }
 
   @Test
   public void testSessionHook () throws Exception {
     // create session, test if the hook got fired by checking the expected property
-    SessionHandle sessionHandle = client.openSession(SESSION_USER_NAME, "foobar",
+    SessionHandle sessionHandle = client.openSession(sessionUserName, "foobar",
           Collections.<String, String>emptyMap());
     Assert.assertEquals(1, SessionHookTest.runCount.get());
     client.closeSession(sessionHandle);
   }
+
+  /***
+   * Create session with proxy user property. Verify the effective session user
+   * @throws Exception
+   */
+  @Test
+  public void testProxyUser() throws Exception {
+    String connectingUser = "user1";
+    String proxyUser = System.getProperty("user.name");
+    Map<String, String>sessConf = new HashMap<String,String>();
+    sessConf.put(HiveAuthFactory.HS2_PROXY_USER, proxyUser);
+    sessionUserName = proxyUser;
+    SessionHandle sessionHandle = client.openSession(connectingUser, "foobar", sessConf);
+    Assert.assertEquals(1, SessionHookTest.runCount.get());
+    client.closeSession(sessionHandle);
+  }
 }

Modified: hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java (original)
+++ hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java Wed Mar 12 10:41:57 2014
@@ -556,6 +556,12 @@ public class Hadoop20Shims implements Ha
   }
 
   @Override
+  public String addServiceToToken(String tokenStr, String tokenService) throws IOException {
+    throw new UnsupportedOperationException("Tokens are not supported in current hadoop version");
+  }
+
+
+  @Override
   public <T> T doAs(UserGroupInformation ugi, PrivilegedExceptionAction<T> pvea) throws
     IOException, InterruptedException {
     try {
@@ -637,6 +643,11 @@ public class Hadoop20Shims implements Ha
   }
 
   @Override
+  public void authorizeProxyAccess(String proxyUser, UserGroupInformation realUserUgi,
+      String ipAddress, Configuration conf) throws IOException {
+    // This hadoop version doesn't have proxy verification
+  }
+
   public boolean isSecurityEnabled() {
     return false;
   }

Modified: hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java (original)
+++ hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java Wed Mar 12 10:41:57 2014
@@ -64,6 +64,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenSelector;
@@ -459,12 +460,39 @@ public abstract class HadoopShimsSecure 
     return token != null ? token.encodeToUrlString() : null;
   }
 
+  /**
+   * Create a delegation token object for the given token string and service.
+   * Add the token to given UGI
+   */
   @Override
   public void setTokenStr(UserGroupInformation ugi, String tokenStr, String tokenService) throws IOException {
+    Token<DelegationTokenIdentifier> delegationToken = createToken(tokenStr, tokenService);
+    ugi.addToken(delegationToken);
+  }
+
+  /**
+   * Add a given service to delegation token string.
+   */
+  @Override
+  public String addServiceToToken(String tokenStr, String tokenService)
+  throws IOException {
+    Token<DelegationTokenIdentifier> delegationToken = createToken(tokenStr, tokenService);
+    return delegationToken.encodeToUrlString();
+  }
+
+  /**
+   * Create a new token using the given string and service
+   * @param tokenStr
+   * @param tokenService
+   * @return
+   * @throws IOException
+   */
+  private Token<DelegationTokenIdentifier> createToken(String tokenStr, String tokenService)
+      throws IOException {
     Token<DelegationTokenIdentifier> delegationToken = new Token<DelegationTokenIdentifier>();
     delegationToken.decodeFromUrlString(tokenStr);
     delegationToken.setService(new Text(tokenService));
-    ugi.addToken(delegationToken);
+    return delegationToken;
   }
 
   @Override
@@ -498,6 +526,13 @@ public abstract class HadoopShimsSecure 
   }
 
   @Override
+  public void authorizeProxyAccess(String proxyUser, UserGroupInformation realUserUgi,
+      String ipAddress,  Configuration conf) throws IOException {
+    ProxyUsers.authorize(UserGroupInformation.createProxyUser(proxyUser, realUserUgi),
+        ipAddress, conf);
+  }
+
+  @Override
   public boolean isSecurityEnabled() {
     return UserGroupInformation.isSecurityEnabled();
   }

Modified: hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/DelegationTokenSecretManager.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/DelegationTokenSecretManager.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/DelegationTokenSecretManager.java (original)
+++ hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/DelegationTokenSecretManager.java Wed Mar 12 10:41:57 2014
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.thrift;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
 import java.io.IOException;
 
 import org.apache.hadoop.io.Text;
@@ -83,5 +85,16 @@ public class DelegationTokenSecretManage
         ident, this);
     return t.encodeToUrlString();
   }
+
+  public String getUserFromToken(String tokenStr) throws IOException {
+    Token<DelegationTokenIdentifier> delegationToken = new Token<DelegationTokenIdentifier>();
+    delegationToken.decodeFromUrlString(tokenStr);
+
+    ByteArrayInputStream buf = new ByteArrayInputStream(delegationToken.getIdentifier());
+    DataInputStream in = new DataInputStream(buf);
+    DelegationTokenIdentifier id = createIdentifier();
+    id.readFields(in);
+    return id.getUser().getShortUserName();
+  }
 }
 

Modified: hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java (original)
+++ hive/trunk/shims/common-secure/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java Wed Mar 12 10:41:57 2014
@@ -43,6 +43,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport;
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
@@ -401,6 +402,13 @@ import org.apache.thrift.transport.TTran
      }
 
      @Override
+     public String getDelegationTokenWithService(String owner, String renewer, String service)
+         throws IOException, InterruptedException {
+       String token = getDelegationToken(owner, renewer);
+       return ShimLoader.getHadoopShims().addServiceToToken(token, service);
+     }
+
+     @Override
      public long renewDelegationToken(String tokenStrForm) throws IOException {
        if (!authenticationMethod.get().equals(AuthenticationMethod.KERBEROS)) {
          throw new AuthorizationException(
@@ -412,6 +420,11 @@ import org.apache.thrift.transport.TTran
      }
 
      @Override
+     public String getUserFromToken(String tokenStr) throws IOException {
+       return secretManager.getUserFromToken(tokenStr);
+     }
+
+     @Override
      public void cancelDelegationToken(String tokenStrForm) throws IOException {
        secretManager.cancelDelegationToken(tokenStrForm);
      }

Modified: hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java (original)
+++ hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java Wed Mar 12 10:41:57 2014
@@ -230,7 +230,7 @@ public interface HadoopShims {
    * @return the string form of the token found
    * @throws IOException
    */
-  String getTokenStrForm(String tokenSignature) throws IOException;
+  public String getTokenStrForm(String tokenSignature) throws IOException;
 
   /**
    * Add a delegation token to the given ugi
@@ -239,9 +239,18 @@ public interface HadoopShims {
    * @param tokenService
    * @throws IOException
    */
-  void setTokenStr(UserGroupInformation ugi, String tokenStr, String tokenService)
+  public void setTokenStr(UserGroupInformation ugi, String tokenStr, String tokenService)
     throws IOException;
 
+  /**
+   * Add given service to the string format token
+   * @param tokenStr
+   * @param tokenService
+   * @return
+   * @throws IOException
+   */
+  public String addServiceToToken(String tokenStr, String tokenService)
+    throws IOException;
 
   enum JobTrackerState { INITIALIZING, RUNNING };
 
@@ -347,7 +356,14 @@ public interface HadoopShims {
    * @param userName
    * @return
    */
-  UserGroupInformation createProxyUser(String userName) throws IOException;
+  public UserGroupInformation createProxyUser(String userName) throws IOException;
+
+  /**
+   * Verify proxy access to given UGI for given user
+   * @param ugi
+   */
+  public void authorizeProxyAccess(String proxyUser, UserGroupInformation realUserUgi,
+      String ipAddress, Configuration conf) throws IOException;
 
   /**
    * The method sets to set the partition file has a different signature between

Modified: hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java?rev=1576685&r1=1576684&r2=1576685&view=diff
==============================================================================
--- hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java (original)
+++ hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java Wed Mar 12 10:41:57 2014
@@ -91,11 +91,14 @@ import org.apache.thrift.transport.TTran
      public abstract InetAddress getRemoteAddress();
      public abstract void startDelegationTokenSecretManager(Configuration conf,
        Object hmsHandler) throws IOException;
-     public abstract String getRemoteUser();
      public abstract String getDelegationToken(String owner, String renewer)
-     throws IOException, InterruptedException;
+       throws IOException, InterruptedException;
+     public abstract String getDelegationTokenWithService(String owner, String renewer, String service)
+       throws IOException, InterruptedException;
+     public abstract String getRemoteUser();
      public abstract long renewDelegationToken(String tokenStrForm) throws IOException;
      public abstract void cancelDelegationToken(String tokenStrForm) throws IOException;
+     public abstract String getUserFromToken(String tokenStr) throws IOException;
    }
  }