diff --git beeline/src/java/org/apache/hive/beeline/BeeLine.java beeline/src/java/org/apache/hive/beeline/BeeLine.java index c60c11d..b8bd9fb 100644 --- beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -81,14 +81,15 @@ import jline.console.ConsoleReader; import jline.console.history.History; import jline.console.history.FileHistory; - import jline.internal.Log; + import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.ShutdownHookManager; /** * A console SQL shell with command completion. @@ -127,6 +128,7 @@ private ConsoleReader consoleReader; private List batch = null; private final Reflector reflector; + public static final int SHUTDOWN_HOOK_PRIORITY = 1; private History history; @@ -877,7 +879,7 @@ public ConsoleReader getConsoleReader(InputStream inputStream) throws IOExceptio } // add shutdown hook to flush the history to history file - Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { + ShutdownHookManager.get().addShutdownHook(new Thread(new Runnable() { @Override public void run() { History h = consoleReader.getHistory(); @@ -889,7 +891,7 @@ public void run() { } } } - })); + }), SHUTDOWN_HOOK_PRIORITY); consoleReader.addCompleter(new BeeLineCompleter(this)); return consoleReader; diff --git hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java index a001252..2ecfe3d 100644 --- hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java +++ hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java @@ -38,6 +38,7 @@ 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.ShutdownHookManager; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,6 +64,7 @@ private static final AtomicInteger nextId = new AtomicInteger(0); private final ScheduledFuture cleanupHandle; // used to cleanup cache + public static final int SHUTDOWN_HOOK_PRIORITY = 0; // Since HiveMetaStoreClient is not threadsafe, hive clients are not shared across threads. // Thread local variable containing each thread's unique ID, is used as one of the keys for the cache @@ -161,7 +163,8 @@ public void run() { closeAllClientsQuietly(); } }; - Runtime.getRuntime().addShutdownHook(cleanupHiveClientShutdownThread); + ShutdownHookManager.get().addShutdownHook(cleanupHiveClientShutdownThread, + SHUTDOWN_HOOK_PRIORITY); } /** diff --git metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index ab011fc..ab6e937 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -204,6 +204,7 @@ import org.apache.hadoop.hive.thrift.TUGIContainingTransport; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.StringUtils; import org.apache.thrift.TException; import org.apache.thrift.TProcessor; @@ -236,6 +237,7 @@ // Can be used to determine if the calls to metastore api (HMSHandler) are being made with // embedded metastore or a remote one private static boolean isMetaStoreRemote = false; + public static final int SHUTDOWN_HOOK_PRIORITY = 0; /** A fixed date format to be used for hive partition column values. */ public static final ThreadLocal PARTITION_DATE_FORMAT = @@ -5727,7 +5729,7 @@ public static void main(String[] args) throws Throwable { } // Add shutdown hook. - Runtime.getRuntime().addShutdownHook(new Thread() { + ShutdownHookManager.get().addShutdownHook(new Thread() { @Override public void run() { String shutdownMsg = "Shutting down hive metastore."; @@ -5736,7 +5738,7 @@ public void run() { System.err.println(shutdownMsg); } } - }); + }, SHUTDOWN_HOOK_PRIORITY); Lock startLock = new ReentrantLock(); Condition startCondition = startLock.newCondition(); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java index 6a6593c..61812fe 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java @@ -59,6 +59,7 @@ import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.TaskCompletionEvent; import org.apache.hadoop.mapred.TaskReport; +import org.apache.hadoop.util.ShutdownHookManager; import org.apache.log4j.Appender; import org.apache.log4j.FileAppender; import org.apache.log4j.LogManager; @@ -80,6 +81,8 @@ private LogHelper console; private HadoopJobExecHook callBackObj; + public static final int SHUTDOWN_HOOK_PRIORITY = 0; + /** * Update counters relevant to this task. */ @@ -172,12 +175,12 @@ public HadoopJobExecHelper(JobConf job, LogHelper console, * */ static { - Runtime.getRuntime().addShutdownHook(new Thread() { + ShutdownHookManager.get().addShutdownHook(new Thread() { @Override public void run() { killRunningJobs(); } - }); + }, SHUTDOWN_HOOK_PRIORITY); } public static void killRunningJobs() { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java index ad012b6..c690ea6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.exec.spark.session; import com.google.common.base.Preconditions; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; @@ -25,6 +26,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.shims.Utils; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hive.spark.client.SparkClientFactory; import java.io.IOException; @@ -46,11 +48,11 @@ private Set createdSessions; private AtomicBoolean inited = new AtomicBoolean(false); - + public static final int SHUTDOWN_HOOK_PRIORITY = 0; private static SparkSessionManagerImpl instance; static { - Runtime.getRuntime().addShutdownHook(new Thread() { + ShutdownHookManager.get().addShutdownHook(new Thread() { @Override public void run() { try { @@ -61,7 +63,7 @@ public void run() { // ignore } } - }); + }, SHUTDOWN_HOOK_PRIORITY); } public static synchronized SparkSessionManagerImpl getInstance() diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java index 64b1e42..c4b9fa6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; +import org.apache.hadoop.util.ShutdownHookManager; import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.counters.TezCounters; @@ -114,10 +115,11 @@ private final NumberFormat secondsFormat; private final NumberFormat commaFormat; private static final List shutdownList; + public static final int SHUTDOWN_HOOK_PRIORITY = 0; static { shutdownList = Collections.synchronizedList(new LinkedList()); - Runtime.getRuntime().addShutdownHook(new Thread() { + ShutdownHookManager.get().addShutdownHook(new Thread() { @Override public void run() { for (DAGClient c: shutdownList) { @@ -132,7 +134,7 @@ public void run() { // ignore } } - }); + }, SHUTDOWN_HOOK_PRIORITY); } public static void initShutdownHook() { diff --git ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java index 513a2fa..f1c94bf 100644 --- ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java +++ ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.exec.ExplainTask; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent; @@ -46,7 +47,6 @@ public class ATSHook implements ExecuteWithHookContext { private static final Log LOG = LogFactory.getLog(ATSHook.class.getName()); - private static final Object LOCK = new Object(); private static ExecutorService executor; private static TimelineClient timelineClient; private enum EntityTypes { HIVE_QUERY_ID }; @@ -54,31 +54,30 @@ private enum OtherInfoTypes { QUERY, STATUS, TEZ, MAPRED }; private enum PrimaryFilterTypes { user, operationid }; private static final int WAIT_TIME = 3; + public static final int SHUTDOWN_HOOK_PRIORITY = 0; - public ATSHook() { - synchronized(LOCK) { - if (executor == null) { - - executor = Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ATS Logger %d").build()); - - YarnConfiguration yarnConf = new YarnConfiguration(); - timelineClient = TimelineClient.createTimelineClient(); - timelineClient.init(yarnConf); - timelineClient.start(); - - Runtime.getRuntime().addShutdownHook(new Thread() { - @Override - public void run() { - try { - executor.shutdown(); - executor.awaitTermination(WAIT_TIME, TimeUnit.SECONDS); - executor = null; - } catch(InterruptedException ie) { /* ignore */ } - timelineClient.stop(); + static { + if (executor == null) { + + executor = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("ATS Logger %d").build()); + + YarnConfiguration yarnConf = new YarnConfiguration(); + timelineClient = TimelineClient.createTimelineClient(); + timelineClient.init(yarnConf); + timelineClient.start(); + ShutdownHookManager.get().addShutdownHook(new Thread() { + @Override + public void run() { + try { + executor.shutdown(); + executor.awaitTermination(WAIT_TIME, TimeUnit.SECONDS); + executor = null; + } catch (InterruptedException ie) { /* ignore */ } - }); - } + timelineClient.stop(); + } + }, SHUTDOWN_HOOK_PRIORITY); } LOG.info("Created ATS Hook"); diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/CuratorFrameworkSingleton.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/CuratorFrameworkSingleton.java index fbf2a01..a208487 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/CuratorFrameworkSingleton.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/CuratorFrameworkSingleton.java @@ -27,19 +27,22 @@ import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper; +import org.apache.hadoop.util.ShutdownHookManager; public class CuratorFrameworkSingleton { private static HiveConf conf = null; private static CuratorFramework sharedClient = null; static final Log LOG = LogFactory.getLog("CuratorFrameworkSingleton"); + public static final int SHUTDOWN_HOOK_PRIORITY = 0; + static { // Add shutdown hook. - Runtime.getRuntime().addShutdownHook(new Thread() { + ShutdownHookManager.get().addShutdownHook(new Thread() { @Override public void run() { closeAndReleaseInstance(); } - }); + }, SHUTDOWN_HOOK_PRIORITY); } public static synchronized CuratorFramework getInstance(HiveConf hiveConf) { diff --git service/src/java/org/apache/hive/service/server/HiveServer2.java service/src/java/org/apache/hive/service/server/HiveServer2.java index 21025a2..d725670 100644 --- service/src/java/org/apache/hive/service/server/HiveServer2.java +++ service/src/java/org/apache/hive/service/server/HiveServer2.java @@ -46,6 +46,7 @@ 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.ShutdownHookManager; import org.apache.hive.common.util.HiveStringUtils; import org.apache.hive.common.util.HiveVersionInfo; import org.apache.hive.service.CompositeService; @@ -73,6 +74,7 @@ private String znodePath; private CuratorFramework zooKeeperClient; private boolean registeredWithZooKeeper = false; + public static final int SHUTDOWN_HOOK_PRIORITY = 0; public HiveServer2() { super(HiveServer2.class.getSimpleName()); @@ -93,12 +95,12 @@ public synchronized void init(HiveConf hiveConf) { // Add a shutdown hook for catching SIGTERM & SIGINT final HiveServer2 hiveServer2 = this; - Runtime.getRuntime().addShutdownHook(new Thread() { + ShutdownHookManager.get().addShutdownHook(new Thread() { @Override public void run() { hiveServer2.stop(); } - }); + }, SHUTDOWN_HOOK_PRIORITY); } public static boolean isHTTPTransportMode(HiveConf hiveConf) { diff --git testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.java testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.java index 279cd72..a6c04e3 100644 --- testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.java +++ testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.java @@ -28,6 +28,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hive.ptest.api.Status; import org.apache.hive.ptest.api.request.TestListRequest; import org.apache.hive.ptest.api.request.TestLogRequest; @@ -66,6 +67,7 @@ public class ExecutionController { private static final long MAX_READ_SIZE = 1024L * 1024L; private static final String CONF_PROPERTY = "hive.ptest.execution.context.conf"; + public static final int SHUTDOWN_HOOK_PRIORITY = 1; private static final Logger LOG = LoggerFactory .getLogger(ExecutionController.class); @@ -97,7 +99,7 @@ public boolean removeEldestEntry(Map.Entry entry) { mTestExecutor.setName("TestExecutor"); mTestExecutor.setDaemon(true); mTestExecutor.start(); - Runtime.getRuntime().addShutdownHook(new Thread() { + ShutdownHookManager.get().addShutdownHook(new Thread() { @Override public void run() { LOG.info("Shutdown hook called"); @@ -112,7 +114,7 @@ public void run() { LOG.error("Error shutting down ExecutionContextProvider", e); } } - }); + }, SHUTDOWN_HOOK_PRIORITY); } @RequestMapping(value="/testStart", method = RequestMethod.POST)