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/14 19:07:26 UTC

svn commit: r1577633 - in /hive/branches/branch-0.13/service/src/java/org/apache/hive/service: auth/ cli/session/ cli/thrift/

Author: thejas
Date: Fri Mar 14 18:07:25 2014
New Revision: 1577633

URL: http://svn.apache.org/r1577633
Log:
HIVE-6312 : doAs with plain sasl auth should be session aware (Navis via Thejas Nair)

Added:
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
Removed:
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/TUGIContainingProcessor.java
Modified:
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionProxy.java
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java

Modified: hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java?rev=1577633&r1=1577632&r2=1577633&view=diff
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (original)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java Fri Mar 14 18:07:25 2014
@@ -18,10 +18,7 @@
 
 package org.apache.hive.service.auth;
 
-import java.net.Socket;
-
 import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Processor;
-import org.apache.hive.service.cli.session.SessionManager;
 import org.apache.hive.service.cli.thrift.TCLIService;
 import org.apache.hive.service.cli.thrift.TCLIService.Iface;
 import org.apache.thrift.TException;
@@ -54,14 +51,19 @@ public class TSetIpAddressProcessor<I ex
   public boolean process(final TProtocol in, final TProtocol out) throws TException {
     setIpAddress(in);
     setUserName(in);
-    return super.process(in, out);
+    try {
+      return super.process(in, out);
+    } finally {
+      threadLocalUserName.remove();
+      threadLocalIpAddress.remove();
+    }
   }
 
   private void setUserName(final TProtocol in) {
     TTransport transport = in.getTransport();
     if (transport instanceof TSaslServerTransport) {
       String userName = ((TSaslServerTransport)transport).getSaslServer().getAuthorizationID();
-      SessionManager.setUserName(userName);
+      threadLocalUserName.set(userName);
     }
   }
 
@@ -69,16 +71,12 @@ public class TSetIpAddressProcessor<I ex
     TTransport transport = in.getTransport();
     TSocket tSocket = getUnderlyingSocketFromTransport(transport);
     if (tSocket != null) {
-     setIpAddress(tSocket.getSocket());
+      threadLocalIpAddress.set(tSocket.getSocket().getInetAddress().toString());
     } else {
       LOGGER.warn("Unknown Transport, cannot determine ipAddress");
     }
   }
 
-  private void setIpAddress(Socket socket) {
-    SessionManager.setIpAddress(socket.getInetAddress().toString());
-  }
-
   private TSocket getUnderlyingSocketFromTransport(TTransport transport) {
     while (transport != null) {
       if (transport instanceof TSaslServerTransport) {
@@ -93,4 +91,26 @@ public class TSetIpAddressProcessor<I ex
     }
     return null;
   }
+
+  private static ThreadLocal<String> threadLocalIpAddress = new ThreadLocal<String>() {
+    @Override
+    protected synchronized String initialValue() {
+      return null;
+    }
+  };
+
+  private static ThreadLocal<String> threadLocalUserName = new ThreadLocal<String>(){
+    @Override
+    protected synchronized String initialValue() {
+      return null;
+    }
+  };
+
+  public static String getUserIpAddress() {
+    return threadLocalIpAddress.get();
+  }
+
+  public static String getUserName() {
+    return threadLocalUserName.get();
+  }
 }
\ No newline at end of file

Modified: hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSession.java?rev=1577633&r1=1577632&r2=1577633&view=diff
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSession.java (original)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSession.java Fri Mar 14 18:07:25 2014
@@ -21,9 +21,7 @@ package org.apache.hive.service.cli.sess
 import java.util.List;
 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;
@@ -31,39 +29,11 @@ import org.apache.hive.service.cli.GetIn
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationHandle;
 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.operation.OperationManager;
-import org.apache.hive.service.cli.thrift.TProtocolVersion;
 
-public interface HiveSession {
+public interface HiveSession extends HiveSessionBase {
 
-  TProtocolVersion getProtocolVersion();
-
-  /**
-   * Set the session manager for the session
-   * @param sessionManager
-   */
-  public void setSessionManager(SessionManager sessionManager);
-
-  /**
-   * Get the session manager for the session
-   */
-  public SessionManager getSessionManager();
-
-  /**
-   * Set operation manager for the session
-   * @param operationManager
-   */
-  public void setOperationManager(OperationManager operationManager);
-
-  public SessionHandle getSessionHandle();
-
-  public String getUsername();
-
-  public String getPassword();
-
-  public HiveConf getHiveConf();
+  public void open();
 
   public IMetaStoreClient getMetaStoreClient() throws HiveSQLException;
 
@@ -179,16 +149,6 @@ public interface HiveSession {
 
   public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException;
 
-  public SessionState getSessionState();
-
-  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;
 

Added: hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java?rev=1577633&view=auto
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java (added)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java Fri Mar 14 18:07:25 2014
@@ -0,0 +1,69 @@
+/**
+ * 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.cli.session;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hive.service.cli.SessionHandle;
+import org.apache.hive.service.cli.operation.OperationManager;
+import org.apache.hive.service.cli.thrift.TProtocolVersion;
+
+/**
+ * Methods that don't need to be executed under a doAs
+ * context are here. Rest of them in HiveSession interface
+ */
+public interface HiveSessionBase {
+
+  TProtocolVersion getProtocolVersion();
+
+  /**
+   * Set the session manager for the session
+   * @param sessionManager
+   */
+  public void setSessionManager(SessionManager sessionManager);
+
+  /**
+   * Get the session manager for the session
+   */
+  public SessionManager getSessionManager();
+
+  /**
+   * Set operation manager for the session
+   * @param operationManager
+   */
+  public void setOperationManager(OperationManager operationManager);
+
+  public SessionHandle getSessionHandle();
+
+  public String getUsername();
+
+  public String getPassword();
+
+  public HiveConf getHiveConf();
+
+  public SessionState getSessionState();
+
+  public String getUserName();
+
+  public void setUserName(String userName);
+
+  public String getIpAddress();
+
+  public void setIpAddress(String ipAddress);
+}

Modified: hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java?rev=1577633&r1=1577632&r2=1577633&view=diff
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (original)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java Fri Mar 14 18:07:25 2014
@@ -131,6 +131,11 @@ public class HiveSessionImpl implements 
     this.operationManager = operationManager;
   }
 
+  @Override
+  public void open() {
+    SessionState.start(sessionState);
+  }
+
   protected synchronized void acquire() throws HiveSQLException {
     // need to make sure that the this connections session state is
     // stored in the thread local for sessions.

Modified: hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionProxy.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionProxy.java?rev=1577633&r1=1577632&r2=1577633&view=diff
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionProxy.java (original)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionProxy.java Fri Mar 14 18:07:25 2014
@@ -54,23 +54,14 @@ public class HiveSessionProxy implements
   public Object invoke(Object arg0, final Method method, final Object[] args)
       throws Throwable {
     try {
+      if (method.getDeclaringClass() == HiveSessionBase.class) {
+        return invoke(method, args);
+      }
       return ShimLoader.getHadoopShims().doAs(ugi,
         new PrivilegedExceptionAction<Object> () {
           @Override
           public Object run() throws HiveSQLException {
-            try {
-              return method.invoke(base, args);
-            } catch (InvocationTargetException e) {
-              if (e.getCause() instanceof HiveSQLException) {
-                throw (HiveSQLException)e.getCause();
-              } else {
-                throw new RuntimeException(e.getCause());
-              }
-            } catch (IllegalArgumentException e) {
-              throw new RuntimeException(e);
-            } catch (IllegalAccessException e) {
-              throw new RuntimeException(e);
-            }
+            return invoke(method, args);
           }
         });
     } catch (UndeclaredThrowableException e) {
@@ -83,5 +74,19 @@ public class HiveSessionProxy implements
     }
   }
 
+  private Object invoke(final Method method, final Object[] args) throws HiveSQLException {
+    try {
+      return method.invoke(base, args);
+    } catch (InvocationTargetException e) {
+      if (e.getCause() instanceof HiveSQLException) {
+        throw (HiveSQLException)e.getCause();
+      }
+      throw new RuntimeException(e.getCause());
+    } catch (IllegalArgumentException e) {
+      throw new RuntimeException(e);
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException(e);
+    }
+  }
 }
 

Modified: hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/SessionManager.java?rev=1577633&r1=1577632&r2=1577633&view=diff
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/SessionManager.java (original)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/SessionManager.java Fri Mar 14 18:07:25 2014
@@ -18,7 +18,6 @@
 
 package org.apache.hive.service.cli.session;
 
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -32,10 +31,8 @@ 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.auth.TSetIpAddressProcessor;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.operation.OperationManager;
@@ -108,15 +105,16 @@ public class SessionManager extends Comp
     HiveSession session;
     if (withImpersonation) {
       HiveSessionImplwithUGI hiveSessionUgi = new HiveSessionImplwithUGI(protocol, username, password,
-        hiveConf, sessionConf, threadLocalIpAddress.get(), delegationToken);
+        hiveConf, sessionConf, TSetIpAddressProcessor.getUserIpAddress(), delegationToken);
       session = HiveSessionProxy.getProxy(hiveSessionUgi, hiveSessionUgi.getSessionUgi());
       hiveSessionUgi.setProxySession(session);
     } else {
-      session = new HiveSessionImpl(protocol, username, password, hiveConf, sessionConf, threadLocalIpAddress.get());
+      session = new HiveSessionImpl(protocol, username, password, hiveConf, sessionConf,
+          TSetIpAddressProcessor.getUserIpAddress());
     }
     session.setSessionManager(this);
     session.setOperationManager(operationManager);
-
+    session.open();
     handleToSession.put(session.getSessionHandle(), session);
 
     try {
@@ -147,44 +145,6 @@ 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);
-  }
-
-  private 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);
-  }
-
-  private 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/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java?rev=1577633&r1=1577632&r2=1577633&view=diff
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (original)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java Fri Mar 14 18:07:25 2014
@@ -29,10 +29,9 @@ 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.auth.TSetIpAddressProcessor;
 import org.apache.hive.service.cli.CLIService;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.GetInfoType;
@@ -43,7 +42,6 @@ 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;
 
@@ -205,16 +203,16 @@ public abstract class ThriftCLIService e
     if (hiveAuthFactory != null) {
       return hiveAuthFactory.getIpAddress();
     }
-    return SessionManager.getIpAddress();
+    return TSetIpAddressProcessor.getUserIpAddress();
   }
 
   private String getUserName(TOpenSessionReq req) throws HiveSQLException {
-    String userName;
-    if (hiveAuthFactory != null
-        && hiveAuthFactory.getRemoteUser() != null) {
-      userName = hiveAuthFactory.getRemoteUser();
-    } else {
-      userName = SessionManager.getUserName();
+    String userName = null;
+    if (hiveAuthFactory != null) {
+      userName = hiveAuthFactory.getRemoteUser(); // kerberos
+    }
+    if (userName == null) {
+      userName = TSetIpAddressProcessor.getUserName();  // except kerberos, nosasl
     }
     if (userName == null) {
       userName = req.getUsername();
@@ -231,12 +229,7 @@ public abstract class ThriftCLIService e
     SessionHandle sessionHandle;
     if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) &&
         (userName != null)) {
-      String delegationTokenStr = null;
-      try {
-        delegationTokenStr = cliService.getDelegationTokenFromMetaStore(userName);
-      } catch (UnsupportedOperationException e) {
-        // The delegation token is not applicable in the given deployment mode
-      }
+      String delegationTokenStr = getDelegationToken(userName);
       sessionHandle = cliService.openSessionWithImpersonation(protocol, userName,
           req.getPassword(), req.getConfiguration(), delegationTokenStr);
     } else {
@@ -247,6 +240,21 @@ public abstract class ThriftCLIService e
     return sessionHandle;
   }
 
+
+  private String getDelegationToken(String userName)
+      throws HiveSQLException, LoginException, IOException {
+    if (userName == null || !cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION)
+        .equals(HiveAuthFactory.AuthTypes.KERBEROS.toString())) {
+      return null;
+    }
+    try {
+      return cliService.getDelegationTokenFromMetaStore(userName);
+    } catch (UnsupportedOperationException e) {
+      // The delegation token is not applicable in the given deployment mode
+    }
+    return null;
+  }
+
   private TProtocolVersion getMinVersion(TProtocolVersion... versions) {
     TProtocolVersion[] values = TProtocolVersion.values();
     int current = values[values.length - 1].getValue();