3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
11 * http://www.apache.org/licenses/LICENSE-2.0
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
19 package org
.apache
.hadoop
.hbase
.regionserver
;
21 import java
.io
.IOException
;
22 import java
.lang
.reflect
.Field
;
23 import java
.util
.HashMap
;
26 import org
.apache
.commons
.logging
.Log
;
27 import org
.apache
.commons
.logging
.LogFactory
;
28 import org
.apache
.hadoop
.hbase
.classification
.InterfaceAudience
;
29 import org
.apache
.hadoop
.conf
.Configuration
;
30 import org
.apache
.hadoop
.fs
.FileSystem
;
31 import org
.apache
.hadoop
.hbase
.HBaseConfiguration
;
32 import org
.apache
.hadoop
.hbase
.Stoppable
;
33 import org
.apache
.hadoop
.hbase
.util
.ShutdownHookManager
;
34 import org
.apache
.hadoop
.hbase
.util
.Threads
;
37 * Manage regionserver shutdown hooks.
38 * @see #install(Configuration, FileSystem, Stoppable, Thread)
40 @InterfaceAudience.Private
41 public class ShutdownHook
{
42 private static final Log LOG
= LogFactory
.getLog(ShutdownHook
.class);
43 private static final String CLIENT_FINALIZER_DATA_METHOD
= "clientFinalizer";
46 * Key for boolean configuration whose default is true.
48 public static final String RUN_SHUTDOWN_HOOK
= "hbase.shutdown.hook";
51 * Key for a long configuration on how much time to wait on the fs shutdown
52 * hook. Default is 30 seconds.
54 public static final String FS_SHUTDOWN_HOOK_WAIT
= "hbase.fs.shutdown.hook.wait";
57 * A place for keeping track of all the filesystem shutdown hooks that need
58 * to be executed after the last regionserver referring to a given filesystem
59 * stops. We keep track of the # of regionserver references in values of the map.
61 private final static Map
<Runnable
, Integer
> fsShutdownHooks
= new HashMap
<Runnable
, Integer
>();
64 * Install a shutdown hook that calls stop on the passed Stoppable
65 * and then thread joins against the passed <code>threadToJoin</code>.
66 * When this thread completes, it then runs the hdfs thread (This install
67 * removes the hdfs shutdown hook keeping a handle on it to run it after
68 * <code>threadToJoin</code> has stopped).
70 * <p>To suppress all shutdown hook handling -- both the running of the
71 * regionserver hook and of the hdfs hook code -- set
72 * {@link ShutdownHook#RUN_SHUTDOWN_HOOK} in {@link Configuration} to
74 * This configuration value is checked when the hook code runs.
76 * @param fs Instance of Filesystem used by the RegionServer
77 * @param stop Installed shutdown hook will call stop against this passed
78 * <code>Stoppable</code> instance.
79 * @param threadToJoin After calling stop on <code>stop</code> will then
82 public static void install(final Configuration conf
, final FileSystem fs
,
83 final Stoppable stop
, final Thread threadToJoin
) {
84 Runnable fsShutdownHook
= suppressHdfsShutdownHook(fs
);
85 Thread t
= new ShutdownHookThread(conf
, stop
, threadToJoin
, fsShutdownHook
);
86 ShutdownHookManager
.affixShutdownHook(t
, 0);
87 LOG
.debug("Installed shutdown hook thread: " + t
.getName());
91 * Thread run by shutdown hook.
93 private static class ShutdownHookThread
extends Thread
{
94 private final Stoppable stop
;
95 private final Thread threadToJoin
;
96 private final Runnable fsShutdownHook
;
97 private final Configuration conf
;
99 ShutdownHookThread(final Configuration conf
, final Stoppable stop
,
100 final Thread threadToJoin
, final Runnable fsShutdownHook
) {
101 super("Shutdownhook:" + threadToJoin
.getName());
103 this.threadToJoin
= threadToJoin
;
105 this.fsShutdownHook
= fsShutdownHook
;
110 boolean b
= this.conf
.getBoolean(RUN_SHUTDOWN_HOOK
, true);
111 LOG
.info("Shutdown hook starting; " + RUN_SHUTDOWN_HOOK
+ "=" + b
+
112 "; fsShutdownHook=" + this.fsShutdownHook
);
114 this.stop
.stop("Shutdown hook");
115 Threads
.shutdown(this.threadToJoin
);
116 if (this.fsShutdownHook
!= null) {
117 synchronized (fsShutdownHooks
) {
118 int refs
= fsShutdownHooks
.get(fsShutdownHook
);
120 LOG
.info("Starting fs shutdown hook thread.");
121 Thread fsShutdownHookThread
= (fsShutdownHook
instanceof Thread
) ?
122 (Thread
)fsShutdownHook
: new Thread(fsShutdownHook
);
123 fsShutdownHookThread
.start();
124 Threads
.shutdown(fsShutdownHookThread
,
125 this.conf
.getLong(FS_SHUTDOWN_HOOK_WAIT
, 30000));
128 fsShutdownHooks
.put(fsShutdownHook
, refs
- 1);
133 LOG
.info("Shutdown hook finished.");
138 * So, HDFS keeps a static map of all FS instances. In order to make sure
139 * things are cleaned up on our way out, it also creates a shutdown hook
140 * so that all filesystems can be closed when the process is terminated; it
141 * calls FileSystem.closeAll. This inconveniently runs concurrently with our
142 * own shutdown handler, and therefore causes all the filesystems to be closed
143 * before the server can do all its necessary cleanup.
145 * <p>The dirty reflection in this method sneaks into the FileSystem class
146 * and grabs the shutdown hook, removes it from the list of active shutdown
147 * hooks, and returns the hook for the caller to run at its convenience.
149 * <p>This seems quite fragile and susceptible to breaking if Hadoop changes
150 * anything about the way this cleanup is managed. Keep an eye on things.
151 * @return The fs shutdown hook
152 * @throws RuntimeException if we fail to find or grap the shutdown hook.
154 private static Runnable
suppressHdfsShutdownHook(final FileSystem fs
) {
156 // This introspection has been updated to work for hadoop 0.20, 0.21 and for
157 // cloudera 0.20. 0.21 and cloudera 0.20 both have hadoop-4829. With the
158 // latter in place, things are a little messy in that there are now two
159 // instances of the data member clientFinalizer; an uninstalled one in
160 // FileSystem and one in the innner class named Cache that actually gets
161 // registered as a shutdown hook. If the latter is present, then we are
162 // on 0.21 or cloudera patched 0.20.
163 Runnable hdfsClientFinalizer
= null;
164 // Look into the FileSystem#Cache class for clientFinalizer
165 Class
<?
> [] classes
= FileSystem
.class.getDeclaredClasses();
166 Class
<?
> cache
= null;
167 for (Class
<?
> c
: classes
) {
168 if (c
.getSimpleName().equals("Cache")) {
175 throw new RuntimeException(
176 "This should not happen. Could not find the cache class in FileSystem.");
181 field
= cache
.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD
);
182 } catch (NoSuchFieldException e
) {
183 // We can get here if the Cache class does not have a clientFinalizer
184 // instance: i.e. we're running on straight 0.20 w/o hadoop-4829.
187 field
.setAccessible(true);
188 Field cacheField
= FileSystem
.class.getDeclaredField("CACHE");
189 cacheField
.setAccessible(true);
190 Object cacheInstance
= cacheField
.get(fs
);
191 hdfsClientFinalizer
= (Runnable
)field
.get(cacheInstance
);
193 // Then we didnt' find clientFinalizer in Cache. Presume clean 0.20 hadoop.
194 field
= FileSystem
.class.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD
);
195 field
.setAccessible(true);
196 hdfsClientFinalizer
= (Runnable
)field
.get(null);
198 if (hdfsClientFinalizer
== null) {
199 throw new RuntimeException("Client finalizer is null, can't suppress!");
201 synchronized (fsShutdownHooks
) {
202 if (!fsShutdownHooks
.containsKey(hdfsClientFinalizer
) &&
203 !ShutdownHookManager
.deleteShutdownHook(hdfsClientFinalizer
)) {
204 throw new RuntimeException("Failed suppression of fs shutdown hook: " +
205 hdfsClientFinalizer
);
207 Integer refs
= fsShutdownHooks
.get(hdfsClientFinalizer
);
208 fsShutdownHooks
.put(hdfsClientFinalizer
, refs
== null ?
1 : refs
+ 1);
210 return hdfsClientFinalizer
;
211 } catch (NoSuchFieldException nsfe
) {
212 LOG
.fatal("Couldn't find field 'clientFinalizer' in FileSystem!", nsfe
);
213 throw new RuntimeException("Failed to suppress HDFS shutdown hook");
214 } catch (IllegalAccessException iae
) {
215 LOG
.fatal("Couldn't access field 'clientFinalizer' in FileSystem!", iae
);
216 throw new RuntimeException("Failed to suppress HDFS shutdown hook");
220 // Thread that does nothing. Used in below main testing.
221 static class DoNothingThread
extends Thread
{
231 // Stoppable with nothing to stop. Used below in main testing.
232 static class DoNothingStoppable
implements Stoppable
{
234 public boolean isStopped() {
235 // TODO Auto-generated method stub
240 public void stop(String why
) {
241 // TODO Auto-generated method stub
246 * Main to test basic functionality. Run with clean hadoop 0.20 and hadoop
247 * 0.21 and cloudera patched hadoop to make sure our shutdown hook handling
248 * works for all compbinations.
249 * Pass '-Dhbase.shutdown.hook=false' to test turning off the running of
252 * @throws IOException
254 public static void main(final String
[] args
) throws IOException
{
255 Configuration conf
= HBaseConfiguration
.create();
256 String prop
= System
.getProperty(RUN_SHUTDOWN_HOOK
);
258 conf
.setBoolean(RUN_SHUTDOWN_HOOK
, Boolean
.parseBoolean(prop
));
260 // Instantiate a FileSystem. This will register the fs shutdown hook.
261 FileSystem fs
= FileSystem
.get(conf
);
262 Thread donothing
= new DoNothingThread();
264 ShutdownHook
.install(conf
, fs
, new DoNothingStoppable(), donothing
);