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 2015/01/15 20:01:59 UTC

svn commit: r1652224 - in /hive/trunk/service/src/java/org/apache/hive/service/cli/session: HiveSession.java HiveSessionBase.java HiveSessionImpl.java HiveSessionImplwithUGI.java SessionManager.java

Author: vgumashta
Date: Thu Jan 15 19:01:59 2015
New Revision: 1652224

URL: http://svn.apache.org/r1652224
Log:
HIVE-9234: HiveServer2 leaks FileSystem objects in FileSystem.CACHE (Vaibhav Gumashta reviewed by Brock Noland, Thejas Nair)

Modified:
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java

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=1652224&r1=1652223&r2=1652224&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 Thu Jan 15 19:01:59 2015
@@ -27,7 +27,7 @@ import org.apache.hive.service.cli.*;
 
 public interface HiveSession extends HiveSessionBase {
 
-  void open();
+  void open(Map<String, String> sessionConfMap) throws Exception;
 
   IMetaStoreClient getMetaStoreClient() throws HiveSQLException;
 

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java?rev=1652224&r1=1652223&r2=1652224&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java Thu Jan 15 19:01:59 2015
@@ -55,12 +55,6 @@ public interface HiveSessionBase {
   void setOperationManager(OperationManager operationManager);
 
   /**
-   * Initialize the session
-   * @param sessionConfMap
-   */
-  void initialize(Map<String, String> sessionConfMap) throws Exception;
-
-  /**
    * Check whether operation logging is enabled and session dir is created successfully
    */
   boolean isOperationLogEnabled();

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=1652224&r1=1652223&r2=1652224&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 Thu Jan 15 19:01:59 2015
@@ -47,7 +47,15 @@ import org.apache.hadoop.hive.ql.session
 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.*;
+import org.apache.hive.service.cli.FetchOrientation;
+import org.apache.hive.service.cli.FetchType;
+import org.apache.hive.service.cli.GetInfoType;
+import org.apache.hive.service.cli.GetInfoValue;
+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.ExecuteStatementOperation;
 import org.apache.hive.service.cli.operation.GetCatalogsOperation;
 import org.apache.hive.service.cli.operation.GetColumnsOperation;
@@ -66,26 +74,20 @@ import org.apache.hive.service.server.Th
  *
  */
 public class HiveSessionImpl implements HiveSession {
-
   private final SessionHandle sessionHandle;
-
   private String username;
   private final String password;
   private HiveConf hiveConf;
-  private final SessionState sessionState;
+  private SessionState sessionState;
   private String ipAddress;
-
   private static final String FETCH_WORK_SERDE_CLASS =
       "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe";
   private static final Log LOG = LogFactory.getLog(HiveSessionImpl.class);
-
-
   private SessionManager sessionManager;
   private OperationManager operationManager;
   private final Set<OperationHandle> opHandleSet = new HashSet<OperationHandle>();
   private boolean isOperationLogEnabled;
   private File sessionLogDir;
-
   private volatile long lastAccessTime;
 
   public HiveSessionImpl(TProtocolVersion protocol, String username, String password,
@@ -106,7 +108,6 @@ public class HiveSessionImpl implements
     } catch (IOException e) {
       LOG.warn("Error setting scheduler queue: " + e, e);
     }
-
     // Set an explicit session name to control the download directory name
     hiveConf.set(ConfVars.HIVESESSIONID.varname,
         sessionHandle.getHandleIdentifier().toString());
@@ -114,37 +115,36 @@ public class HiveSessionImpl implements
     hiveConf.set(ListSinkOperator.OUTPUT_FORMATTER,
         FetchFormatter.ThriftFormatter.class.getName());
     hiveConf.setInt(ListSinkOperator.OUTPUT_PROTOCOL, protocol.getValue());
+  }
 
-    /**
-     * Create a new SessionState object that will be associated with this HiveServer2 session.
-     * When the server executes multiple queries in the same session,
-     * this SessionState object is reused across multiple queries.
-     */
+  @Override
+  /**
+   * Opens a new HiveServer2 session for the client connection.
+   * Creates a new SessionState object that will be associated with this HiveServer2 session.
+   * When the server executes multiple queries in the same session,
+   * this SessionState object is reused across multiple queries.
+   * Note that if doAs is true, this call goes through a proxy object,
+   * which wraps the method logic in a UserGroupInformation#doAs.
+   * That's why it is important to create SessionState here rather than in the constructor.
+   */
+  public void open(Map<String, String> sessionConfMap) throws HiveSQLException {
     sessionState = new SessionState(hiveConf, username);
     sessionState.setUserIpAddress(ipAddress);
     sessionState.setIsHiveServerQuery(true);
-
-    lastAccessTime = System.currentTimeMillis();
     SessionState.start(sessionState);
-  }
-
-  @Override
-  public void initialize(Map<String, String> sessionConfMap) throws Exception {
-    // Process global init file: .hiverc
-    processGlobalInitFile();
     try {
       sessionState.reloadAuxJars();
     } catch (IOException e) {
-      String msg = "fail to load reloadable jar file path" + e;
+      String msg = "Failed to load reloadable jar file path: " + e;
       LOG.error(msg, e);
-      throw new Exception(msg, e);
+      throw new HiveSQLException(msg, e);
     }
-    SessionState.setCurrentSessionState(sessionState);
-
-    // Set conf properties specified by user from client side
+    // Process global init file: .hiverc
+    processGlobalInitFile();
     if (sessionConfMap != null) {
       configureSession(sessionConfMap);
     }
+    lastAccessTime = System.currentTimeMillis();
   }
 
   /**
@@ -199,12 +199,16 @@ public class HiveSessionImpl implements
     }
   }
 
-  private void configureSession(Map<String, String> sessionConfMap) throws Exception {
+  private void configureSession(Map<String, String> sessionConfMap) throws HiveSQLException {
     SessionState.setCurrentSessionState(sessionState);
     for (Map.Entry<String, String> entry : sessionConfMap.entrySet()) {
       String key = entry.getKey();
       if (key.startsWith("set:")) {
-        SetProcessor.setVariable(key.substring(4), entry.getValue());
+        try {
+          SetProcessor.setVariable(key.substring(4), entry.getValue());
+        } catch (Exception e) {
+          throw new HiveSQLException(e);
+        }
       } else if (key.startsWith("use:")) {
         SessionState.get().setCurrentDatabase(entry.getValue());
       } else {
@@ -217,7 +221,6 @@ public class HiveSessionImpl implements
   public void setOperationLogSessionDir(File operationLogRootDir) {
     sessionLogDir = new File(operationLogRootDir, sessionHandle.getHandleIdentifier().toString());
     isOperationLogEnabled = true;
-
     if (!sessionLogDir.exists()) {
       if (!sessionLogDir.mkdir()) {
         LOG.warn("Unable to create operation log session directory: " +
@@ -225,7 +228,6 @@ public class HiveSessionImpl implements
         isOperationLogEnabled = false;
       }
     }
-
     if (isOperationLogEnabled) {
       LOG.info("Operation log session directory is created: " + sessionLogDir.getAbsolutePath());
     }
@@ -265,19 +267,8 @@ public class HiveSessionImpl implements
     this.operationManager = operationManager;
   }
 
-  @Override
-  /**
-   * Opens a new HiveServer2 session for the client connection.
-   * Note that if doAs is true, this call goes through a proxy object,
-   * which wraps the method logic in a UserGroupInformation#doAs.
-   * That is why it is important to call SessionState#start here rather than the constructor.
-   */
-  public void open() {
-    SessionState.start(sessionState);
-  }
-
   protected synchronized void acquire(boolean userAccess) {
-    // Need to make sure that the this HiveServer2's session's session state is
+    // Need to make sure that the this HiveServer2's session's SessionState is
     // stored in the thread local for the handler thread.
     SessionState.setCurrentSessionState(sessionState);
     if (userAccess) {
@@ -556,7 +547,6 @@ public class HiveSessionImpl implements
       opHandleSet.clear();
       // Cleanup session log directory.
       cleanupSessionLogDir();
-
       HiveHistory hiveHist = sessionState.getHiveHistory();
       if (null != hiveHist) {
         hiveHist.closeStream();

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=1652224&r1=1652223&r2=1652224&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 Thu Jan 15 19:01:59 2015
@@ -97,20 +97,25 @@ public class HiveSessionImplwithUGI exte
   }
 
   /**
-   * close the file systems for the session
-   * cancel the session's delegation token and close the metastore connection
+   * Close the file systems for the session and remove it from the FileSystem cache.
+   * Cancel the session's delegation token and close the metastore connection
    */
   @Override
   public void close() throws HiveSQLException {
     try {
-    acquire(true);
-    FileSystem.closeAllForUGI(sessionUgi);
-    cancelDelegationToken();
-    } catch (IOException ioe) {
-      LOG.error("Could not clean up file-system handles for UGI: " + sessionUgi, ioe);
+      acquire(true);
+      cancelDelegationToken();
     } finally {
-      release(true);
-      super.close();
+      try {
+        super.close();
+      } finally {
+        try {
+          FileSystem.closeAllForUGI(sessionUgi);
+        } catch (IOException ioe) {
+          throw new HiveSQLException("Could not clean up file-system handles for UGI: "
+              + sessionUgi, ioe);
+        }
+      }
     }
   }
 

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=1652224&r1=1652223&r2=1652224&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 Thu Jan 15 19:01:59 2015
@@ -267,22 +267,19 @@ public class SessionManager extends Comp
     session.setSessionManager(this);
     session.setOperationManager(operationManager);
     try {
-      session.initialize(sessionConf);
-      if (isOperationLogEnabled) {
-        session.setOperationLogSessionDir(operationLogRootDir);
-      }
-      session.open();
+      session.open(sessionConf);
     } catch (Exception e) {
-      throw new HiveSQLException("Failed to open new session", e);
+      throw new HiveSQLException("Failed to open new session: " + e, e);
+    }
+    if (isOperationLogEnabled) {
+      session.setOperationLogSessionDir(operationLogRootDir);
     }
     try {
       executeSessionHooks(session);
     } catch (Exception e) {
       throw new HiveSQLException("Failed to execute session hooks", e);
     }
-
     handleToSession.put(session.getSessionHandle(), session);
-
     return session.getSessionHandle();
   }