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 2015/10/16 22:36:21 UTC

hive git commit: Revert "HIVE-11768 : java.io.DeleteOnExitHook leaks memory on long running Hive Server2 Instances (Navis Ryu via Thejas Nair)"

Repository: hive
Updated Branches:
  refs/heads/branch-1 1fb681931 -> 476bd9803


Revert "HIVE-11768 : java.io.DeleteOnExitHook leaks memory on long running Hive Server2 Instances (Navis Ryu via Thejas Nair)"

This reverts commit 1fb6819311966b374f4336d97f0067f8a981945a.


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/476bd980
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/476bd980
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/476bd980

Branch: refs/heads/branch-1
Commit: 476bd98036bca03de153bb709c989919ac92482b
Parents: 1fb6819
Author: Thejas Nair <th...@hortonworks.com>
Authored: Fri Oct 16 13:36:11 2015 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Fri Oct 16 13:36:11 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/FileUtils.java    | 38 --------------
 .../hive/common/util/ShutdownHookManager.java   | 52 +-------------------
 .../common/util/TestShutdownHookManager.java    | 22 ++-------
 .../hadoop/hive/ql/session/SessionState.java    | 43 ++++++++--------
 .../cli/operation/HiveCommandOperation.java     |  4 +-
 .../service/cli/operation/SQLOperation.java     |  5 +-
 6 files changed, 33 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/476bd980/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
index d781f08..7e4f386 100644
--- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.common;
 
-import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
@@ -45,7 +44,6 @@ import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
-import org.apache.hive.common.util.ShutdownHookManager;
 
 
 /**
@@ -768,40 +766,4 @@ public final class FileUtils {
       return null;
     }
   }
-
-  public static void deleteDirectory(File directory) throws IOException {
-    org.apache.commons.io.FileUtils.deleteDirectory(directory);
-  }
-
-  /**
-   * create temporary file and register it to delete-on-exit hook.
-   * File.deleteOnExit is not used for possible memory leakage.
-   */
-  public static File createTempFile(String lScratchDir, String prefix, String suffix) throws IOException {
-    File tmpDir = lScratchDir == null ? null : new File(lScratchDir);
-    if (tmpDir != null && !tmpDir.exists() && !tmpDir.mkdirs()) {
-      // Do another exists to check to handle possible race condition
-      // Another thread might have created the dir, if that is why
-      // mkdirs returned false, that is fine
-      if (!tmpDir.exists()) {
-        throw new RuntimeException("Unable to create temp directory "
-            + lScratchDir);
-      }
-    }
-    File tmpFile = File.createTempFile(prefix, suffix, tmpDir);
-    ShutdownHookManager.deleteOnExit(tmpFile);
-    return tmpFile;
-  }
-
-  /**
-   * delete a temporary file and remove it from delete-on-exit hook.
-   */
-  public static boolean deleteTmpFile(File tempFile) {
-    if (tempFile != null) {
-      tempFile.delete();
-      ShutdownHookManager.cancelDeleteOnExit(tempFile);
-      return true;
-    }
-    return false;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/476bd980/common/src/java/org/apache/hive/common/util/ShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/ShutdownHookManager.java b/common/src/java/org/apache/hive/common/util/ShutdownHookManager.java
index 0392eb5..fd2f20a 100644
--- a/common/src/java/org/apache/hive/common/util/ShutdownHookManager.java
+++ b/common/src/java/org/apache/hive/common/util/ShutdownHookManager.java
@@ -18,9 +18,6 @@
 
 package org.apache.hive.common.util;
 
-import com.google.common.annotations.VisibleForTesting;
-
-import java.io.File;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -47,18 +44,15 @@ public class ShutdownHookManager {
 
   private static final ShutdownHookManager MGR = new ShutdownHookManager();
 
-  private static final DeleteOnExitHook DELETE_ON_EXIT_HOOK = new DeleteOnExitHook();
-
   private static final Log LOG = LogFactory.getLog(ShutdownHookManager.class);
 
   static {
-    MGR.addShutdownHookInternal(DELETE_ON_EXIT_HOOK, -1);
     Runtime.getRuntime().addShutdownHook(
       new Thread() {
         @Override
         public void run() {
           MGR.shutdownInProgress.set(true);
-          for (Runnable hook : getShutdownHooksInOrder()) {
+          for (Runnable hook: MGR.getShutdownHooksInOrder()) {
             try {
               hook.run();
             } catch (Throwable ex) {
@@ -121,7 +115,7 @@ public class ShutdownHookManager {
     return MGR.getShutdownHooksInOrderInternal();
   }
 
-  private List<Runnable> getShutdownHooksInOrderInternal() {
+  List<Runnable> getShutdownHooksInOrderInternal() {
     List<HookEntry> list;
     synchronized (MGR.hooks) {
       list = new ArrayList<HookEntry>(MGR.hooks);
@@ -151,9 +145,6 @@ public class ShutdownHookManager {
    * @param priority priority of the shutdownHook.
    */
   public static void addShutdownHook(Runnable shutdownHook, int priority) {
-    if (priority < 0) {
-      throw new IllegalArgumentException("Priority should be greater than or equal to zero");
-    }
     MGR.addShutdownHookInternal(shutdownHook, priority);
   }
 
@@ -211,43 +202,4 @@ public class ShutdownHookManager {
   private boolean isShutdownInProgressInternal() {
     return shutdownInProgress.get();
   }
-
-  /**
-   * register file to delete-on-exit hook
-   *
-   * @see {@link org.apache.hadoop.hive.common.FileUtils#createTempFile}
-   */
-  public static void deleteOnExit(File file) {
-    if (isShutdownInProgress()) {
-      throw new IllegalStateException("Shutdown in progress, cannot add a deleteOnExit");
-    }
-    DELETE_ON_EXIT_HOOK.deleteTargets.add(file);
-  }
-
-  /**
-   * deregister file from delete-on-exit hook
-   */
-  public static void cancelDeleteOnExit(File file) {
-    if (isShutdownInProgress()) {
-      throw new IllegalStateException("Shutdown in progress, cannot cancel a deleteOnExit");
-    }
-    DELETE_ON_EXIT_HOOK.deleteTargets.remove(file);
-  }
-
-  @VisibleForTesting
-  static boolean isRegisteredToDeleteOnExit(File file) {
-    return DELETE_ON_EXIT_HOOK.deleteTargets.contains(file);
-  }
-
-  private static class DeleteOnExitHook implements Runnable {
-    private final Set<File> deleteTargets = Collections.synchronizedSet(new HashSet<File>());
-
-    @Override
-    public void run() {
-      for (File deleteTarget : deleteTargets) {
-        deleteTarget.delete();
-      }
-      deleteTargets.clear();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/476bd980/common/src/test/org/apache/hive/common/util/TestShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hive/common/util/TestShutdownHookManager.java b/common/src/test/org/apache/hive/common/util/TestShutdownHookManager.java
index 66f6073..fa30f15 100644
--- a/common/src/test/org/apache/hive/common/util/TestShutdownHookManager.java
+++ b/common/src/test/org/apache/hive/common/util/TestShutdownHookManager.java
@@ -21,11 +21,6 @@ package org.apache.hive.common.util;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.hadoop.hive.common.FileUtils;
-
 /**
  * TestShutdownHookManager.
  *
@@ -35,7 +30,7 @@ public class TestShutdownHookManager {
 
   @Test
   public void shutdownHookManager() {
-    Assert.assertEquals(1, ShutdownHookManager.getShutdownHooksInOrder().size());
+    Assert.assertEquals(0, ShutdownHookManager.getShutdownHooksInOrder().size());
     Runnable hook1 = new Runnable() {
       @Override
       public void run() {
@@ -49,30 +44,23 @@ public class TestShutdownHookManager {
 
     ShutdownHookManager.addShutdownHook(hook1, 0);
     Assert.assertTrue(ShutdownHookManager.hasShutdownHook(hook1));
-    Assert.assertEquals(2, ShutdownHookManager.getShutdownHooksInOrder().size());
+    Assert.assertEquals(1, ShutdownHookManager.getShutdownHooksInOrder().size());
     Assert.assertEquals(hook1, ShutdownHookManager.getShutdownHooksInOrder().get(0));
     ShutdownHookManager.removeShutdownHook(hook1);
     Assert.assertFalse(ShutdownHookManager.hasShutdownHook(hook1));
 
     ShutdownHookManager.addShutdownHook(hook1, 0);
     Assert.assertTrue(ShutdownHookManager.hasShutdownHook(hook1));
-    Assert.assertEquals(2, ShutdownHookManager.getShutdownHooksInOrder().size());
+    Assert.assertEquals(1, ShutdownHookManager.getShutdownHooksInOrder().size());
     Assert.assertTrue(ShutdownHookManager.hasShutdownHook(hook1));
-    Assert.assertEquals(2, ShutdownHookManager.getShutdownHooksInOrder().size());
+    Assert.assertEquals(1, ShutdownHookManager.getShutdownHooksInOrder().size());
 
     ShutdownHookManager.addShutdownHook(hook2, 1);
     Assert.assertTrue(ShutdownHookManager.hasShutdownHook(hook1));
     Assert.assertTrue(ShutdownHookManager.hasShutdownHook(hook2));
-    Assert.assertEquals(3, ShutdownHookManager.getShutdownHooksInOrder().size());
+    Assert.assertEquals(2, ShutdownHookManager.getShutdownHooksInOrder().size());
     Assert.assertEquals(hook2, ShutdownHookManager.getShutdownHooksInOrder().get(0));
     Assert.assertEquals(hook1, ShutdownHookManager.getShutdownHooksInOrder().get(1));
-  }
 
-  @Test
-  public void deleteOnExit() throws IOException {
-    File file = FileUtils.createTempFile(null, "tmp", null);
-    Assert.assertTrue(ShutdownHookManager.isRegisteredToDeleteOnExit(file));
-    FileUtils.deleteTmpFile(file);
-    Assert.assertFalse(ShutdownHookManager.isRegisteredToDeleteOnExit(file));
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/476bd980/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 554c8da..ee0f763 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -23,6 +23,8 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URLClassLoader;
@@ -41,6 +43,7 @@ import java.util.UUID;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,7 +52,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -284,22 +286,6 @@ public class SessionState {
     tmpOutputFile = f;
   }
 
-  public File getTmpErrOutputFile() {
-    return tmpErrOutputFile;
-  }
-
-  public void setTmpErrOutputFile(File tmpErrOutputFile) {
-    this.tmpErrOutputFile = tmpErrOutputFile;
-  }
-
-  public void deleteTmpOutputFile() {
-    FileUtils.deleteTmpFile(tmpOutputFile);
-  }
-
-  public void deleteTmpErrOutputFile() {
-    FileUtils.deleteTmpFile(tmpErrOutputFile);
-  }
-
   public boolean getIsSilent() {
     if(conf != null) {
       return conf.getBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT);
@@ -713,8 +699,6 @@ public class SessionState {
     if (localSessionPath != null) {
       FileSystem.getLocal(conf).delete(localSessionPath, true);
     }
-    deleteTmpOutputFile();
-    deleteTmpErrOutputFile();
   }
 
   /**
@@ -822,10 +806,25 @@ public class SessionState {
    * @throws IOException
    */
   private static File createTempFile(HiveConf conf) throws IOException {
-    String lScratchDir = HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR);
-    String sessionID = conf.getVar(HiveConf.ConfVars.HIVESESSIONID);
+    String lScratchDir =
+        HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR);
 
-    return FileUtils.createTempFile(lScratchDir, sessionID, ".pipeout");
+    File tmpDir = new File(lScratchDir);
+    String sessionID = conf.getVar(HiveConf.ConfVars.HIVESESSIONID);
+    if (!tmpDir.exists()) {
+      if (!tmpDir.mkdirs()) {
+        //Do another exists to check to handle possible race condition
+        // Another thread might have created the dir, if that is why
+        // mkdirs returned false, that is fine
+        if(!tmpDir.exists()){
+          throw new RuntimeException("Unable to create log directory "
+              + lScratchDir);
+        }
+      }
+    }
+    File tmpFile = File.createTempFile(sessionID, ".pipeout", tmpDir);
+    tmpFile.deleteOnExit();
+    return tmpFile;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/476bd980/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
index b438270..bcc66cf 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
@@ -200,8 +200,8 @@ public class HiveCommandOperation extends ExecuteStatementOperation {
   private void cleanTmpFile() {
     resetResultReader();
     SessionState sessionState = getParentSession().getSessionState();
-    sessionState.deleteTmpOutputFile();
-    sessionState.deleteTmpErrOutputFile();
+    File tmp = sessionState.getTmpOutputFile();
+    tmp.delete();
   }
 
   private void resetResultReader() {

http://git-wip-us.apache.org/repos/asf/hive/blob/476bd980/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 5c2f13d..cc9df76 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -287,8 +287,9 @@ public class SQLOperation extends ExecuteStatementOperation {
     driver = null;
 
     SessionState ss = SessionState.get();
-    ss.deleteTmpOutputFile();
-    ss.deleteTmpErrOutputFile();
+    if (ss.getTmpOutputFile() != null) {
+      ss.getTmpOutputFile().delete();
+    }
   }
 
   @Override