HBASE-4209 The HBase hbase-daemon.sh SIGKILLs master when stopping it
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1177849 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f40479ab63
commit
c80eaecce1
|
@ -325,6 +325,8 @@ Release 0.92.0 - Unreleased
|
|||
RegionServerStoppedException (Ming Ma)
|
||||
HBASE-4476 Compactions must fail if column tracker gets columns out of order
|
||||
(Mikhail Bautin)
|
||||
HBASE-4209 The HBase hbase-daemon.sh SIGKILLs master when stopping it
|
||||
(Roman Shaposhnik)
|
||||
|
||||
TESTS
|
||||
HBASE-4450 test for number of blocks read: to serve as baseline for expected
|
||||
|
|
|
@ -412,7 +412,7 @@ public class LocalHBaseCluster {
|
|||
/**
|
||||
* Start the cluster.
|
||||
*/
|
||||
public void startup() {
|
||||
public void startup() throws IOException {
|
||||
JVMClusterUtil.startup(this.masterThreads, this.regionThreads);
|
||||
}
|
||||
|
||||
|
|
|
@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -34,7 +36,7 @@ import org.apache.hadoop.hbase.util.Threads;
|
|||
* Manage regionserver shutdown hooks.
|
||||
* @see #install(Configuration, FileSystem, Stoppable, Thread)
|
||||
*/
|
||||
class ShutdownHook {
|
||||
public class ShutdownHook {
|
||||
private static final Log LOG = LogFactory.getLog(ShutdownHook.class);
|
||||
private static final String CLIENT_FINALIZER_DATA_METHOD = "clientFinalizer";
|
||||
|
||||
|
@ -49,6 +51,13 @@ class ShutdownHook {
|
|||
*/
|
||||
public static final String FS_SHUTDOWN_HOOK_WAIT = "hbase.fs.shutdown.hook.wait";
|
||||
|
||||
/**
|
||||
* A place for keeping track of all the filesystem shutdown hooks that need
|
||||
* to be executed after the last regionserver referring to a given filesystem
|
||||
* stops. We keep track of the # of regionserver references in values of the map.
|
||||
*/
|
||||
private final static Map<Thread, Integer> fsShutdownHooks = new HashMap<Thread, Integer>();
|
||||
|
||||
/**
|
||||
* Install a shutdown hook that calls stop on the passed Stoppable
|
||||
* and then thread joins against the passed <code>threadToJoin</code>.
|
||||
|
@ -68,7 +77,7 @@ class ShutdownHook {
|
|||
* @param threadToJoin After calling stop on <code>stop</code> will then
|
||||
* join this thread.
|
||||
*/
|
||||
static void install(final Configuration conf, final FileSystem fs,
|
||||
public static void install(final Configuration conf, final FileSystem fs,
|
||||
final Stoppable stop, final Thread threadToJoin) {
|
||||
Thread fsShutdownHook = suppressHdfsShutdownHook(fs);
|
||||
Thread t = new ShutdownHookThread(conf, stop, threadToJoin, fsShutdownHook);
|
||||
|
@ -103,11 +112,19 @@ class ShutdownHook {
|
|||
this.stop.stop("Shutdown hook");
|
||||
Threads.shutdown(this.threadToJoin);
|
||||
if (this.fsShutdownHook != null) {
|
||||
synchronized (fsShutdownHooks) {
|
||||
int refs = fsShutdownHooks.get(fsShutdownHook);
|
||||
if (refs == 1) {
|
||||
LOG.info("Starting fs shutdown hook thread.");
|
||||
this.fsShutdownHook.start();
|
||||
Threads.shutdown(this.fsShutdownHook,
|
||||
this.conf.getLong(FS_SHUTDOWN_HOOK_WAIT, 30000));
|
||||
}
|
||||
if (refs > 0) {
|
||||
fsShutdownHooks.put(fsShutdownHook, refs - 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG.info("Shutdown hook finished.");
|
||||
}
|
||||
|
@ -171,10 +188,15 @@ class ShutdownHook {
|
|||
if (hdfsClientFinalizer == null) {
|
||||
throw new RuntimeException("Client finalizer is null, can't suppress!");
|
||||
}
|
||||
if (!Runtime.getRuntime().removeShutdownHook(hdfsClientFinalizer)) {
|
||||
if (!fsShutdownHooks.containsKey(hdfsClientFinalizer) &&
|
||||
!Runtime.getRuntime().removeShutdownHook(hdfsClientFinalizer)) {
|
||||
throw new RuntimeException("Failed suppression of fs shutdown hook: " +
|
||||
hdfsClientFinalizer);
|
||||
}
|
||||
synchronized (fsShutdownHooks) {
|
||||
Integer refs = fsShutdownHooks.get(hdfsClientFinalizer);
|
||||
fsShutdownHooks.put(hdfsClientFinalizer, refs == null ? 1 : refs + 1);
|
||||
}
|
||||
return hdfsClientFinalizer;
|
||||
} catch (NoSuchFieldException nsfe) {
|
||||
LOG.fatal("Couldn't find field 'clientFinalizer' in FileSystem!", nsfe);
|
||||
|
|
|
@ -26,8 +26,10 @@ import java.util.List;
|
|||
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.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.ShutdownHook;
|
||||
|
||||
/**
|
||||
* Utility used running a cluster all in the one JVM.
|
||||
|
@ -171,7 +173,7 @@ public class JVMClusterUtil {
|
|||
* @return Address to use contacting primary master.
|
||||
*/
|
||||
public static String startup(final List<JVMClusterUtil.MasterThread> masters,
|
||||
final List<JVMClusterUtil.RegionServerThread> regionservers) {
|
||||
final List<JVMClusterUtil.RegionServerThread> regionservers) throws IOException {
|
||||
if (masters != null) {
|
||||
for (JVMClusterUtil.MasterThread t : masters) {
|
||||
t.start();
|
||||
|
@ -179,6 +181,9 @@ public class JVMClusterUtil {
|
|||
}
|
||||
if (regionservers != null) {
|
||||
for (JVMClusterUtil.RegionServerThread t: regionservers) {
|
||||
HRegionServer hrs = t.getRegionServer();
|
||||
ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs
|
||||
.getConfiguration()), hrs, t);
|
||||
t.start();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue