1 /**
2 * Copyright 2010 The Apache Software Foundation
3 *
4 * Licensed to the Apache Software Foundation (ASF) under one
5 * or more contributor license agreements. See the NOTICE file
6 * distributed with this work for additional information
7 * regarding copyright ownership. The ASF licenses this file
8 * to you under the Apache License, Version 2.0 (the
9 * "License"); you may not use this file except in compliance
10 * with the License. You may obtain a copy of the License at
11 *
12 * http://www.apache.org/licenses/LICENSE-2.0
13 *
14 * Unless required by applicable law or agreed to in writing, software
15 * distributed under the License is distributed on an "AS IS" BASIS,
16 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17 * See the License for the specific language governing permissions and
18 * limitations under the License.
19 */
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.io.IOException;
23 import java.lang.reflect.Field;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.FileSystem;
29 import org.apache.hadoop.hbase.HBaseConfiguration;
30 import org.apache.hadoop.hbase.util.Threads;
31
32 /**
33 * Manage regionserver shutdown hooks.
34 * @see #install(Configuration, FileSystem, Stoppable, Thread)
35 */
36 class ShutdownHook {
37 private static final Log LOG = LogFactory.getLog(ShutdownHook.class);
38 private static final String CLIENT_FINALIZER_DATA_METHOD = "clientFinalizer";
39
40 /**
41 * Key for boolean configuration whose default is true.
42 */
43 public static final String RUN_SHUTDOWN_HOOK = "hbase.shutdown.hook";
44
45 /**
46 * Key for a long configuration on how much time to wait on the fs shutdown
47 * hook. Default is 30 seconds.
48 */
49 public static final String FS_SHUTDOWN_HOOK_WAIT = "hbase.fs.shutdown.hook.wait";
50
51 /**
52 * Install a shutdown hook that calls stop on the passed Stoppable
53 * and then thread joins against the passed <code>threadToJoin</code>.
54 * When this thread completes, it then runs the hdfs thread (This install
55 * removes the hdfs shutdown hook keeping a handle on it to run it after
56 * <code>threadToJoin</code> has stopped).
57 *
58 * <p>To suppress all shutdown hook handling -- both the running of the
59 * regionserver hook and of the hdfs hook code -- set
60 * {@link ShutdownHook#RUN_SHUTDOWN_HOOK} in {@link Configuration} to
61 * <code>false</code>.
62 * This configuration value is checked when the hook code runs.
63 * @param conf
64 * @param fs Instance of Filesystem used by the RegionServer
65 * @param stop Installed shutdown hook will call stop against this passed
66 * <code>Stoppable</code> instance.
67 * @param threadToJoin After calling stop on <code>stop</code> will then
68 * join this thread.
69 */
70 static void install(final Configuration conf, final FileSystem fs,
71 final Stoppable stop, final Thread threadToJoin) {
72 Thread fsShutdownHook = suppressHdfsShutdownHook(fs);
73 Thread t = new ShutdownHookThread(conf, stop, threadToJoin, fsShutdownHook);
74 Runtime.getRuntime().addShutdownHook(t);
75 LOG.info("Installed shutdown hook thread: " + t.getName());
76 }
77
78 /*
79 * Thread run by shutdown hook.
80 */
81 private static class ShutdownHookThread extends Thread {
82 private final Stoppable stop;
83 private final Thread threadToJoin;
84 private final Thread fsShutdownHook;
85 private final Configuration conf;
86
87 ShutdownHookThread(final Configuration conf, final Stoppable stop,
88 final Thread threadToJoin, final Thread fsShutdownHook) {
89 super("Shutdownhook:" + threadToJoin.getName());
90 this.stop = stop;
91 this.threadToJoin = threadToJoin;
92 this.conf = conf;
93 this.fsShutdownHook = fsShutdownHook;
94 }
95
96 @Override
97 public void run() {
98 boolean b = this.conf.getBoolean(RUN_SHUTDOWN_HOOK, true);
99 LOG.info("Shutdown hook starting; " + RUN_SHUTDOWN_HOOK + "=" + b +
100 "; fsShutdownHook=" + this.fsShutdownHook);
101 if (b) {
102 this.stop.stop();
103 Threads.shutdown(this.threadToJoin);
104 if (this.fsShutdownHook != null) {
105 LOG.info("Starting fs shutdown hook thread.");
106 this.fsShutdownHook.start();
107 Threads.shutdown(this.fsShutdownHook,
108 this.conf.getLong(FS_SHUTDOWN_HOOK_WAIT, 30000));
109 }
110 }
111 LOG.info("Shutdown hook finished.");
112 }
113 }
114
115 /*
116 * So, HDFS keeps a static map of all FS instances. In order to make sure
117 * things are cleaned up on our way out, it also creates a shutdown hook
118 * so that all filesystems can be closed when the process is terminated; it
119 * calls FileSystem.closeAll. This inconveniently runs concurrently with our
120 * own shutdown handler, and therefore causes all the filesystems to be closed
121 * before the server can do all its necessary cleanup.
122 *
123 * <p>The dirty reflection in this method sneaks into the FileSystem class
124 * and grabs the shutdown hook, removes it from the list of active shutdown
125 * hooks, and returns the hook for the caller to run at its convenience.
126 *
127 * <p>This seems quite fragile and susceptible to breaking if Hadoop changes
128 * anything about the way this cleanup is managed. Keep an eye on things.
129 * @return The fs shutdown hook
130 * @throws RuntimeException if we fail to find or grap the shutdown hook.
131 */
132 private static Thread suppressHdfsShutdownHook(final FileSystem fs) {
133 try {
134 // This introspection has been updated to work for hadoop 0.20, 0.21 and for
135 // cloudera 0.20. 0.21 and cloudera 0.20 both have hadoop-4829. With the
136 // latter in place, things are a little messy in that there are now two
137 // instances of the data member clientFinalizer; an uninstalled one in
138 // FileSystem and one in the innner class named Cache that actually gets
139 // registered as a shutdown hook. If the latter is present, then we are
140 // on 0.21 or cloudera patched 0.20.
141 Thread hdfsClientFinalizer = null;
142 // Look into the FileSystem#Cache class for clientFinalizer
143 Class<?> [] classes = FileSystem.class.getDeclaredClasses();
144 Class<?> cache = null;
145 for (Class<?> c: classes) {
146 if (c.getSimpleName().equals("Cache")) {
147 cache = c;
148 break;
149 }
150 }
151 Field field = null;
152 try {
153 field = cache.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
154 } catch (NoSuchFieldException e) {
155 // We can get here if the Cache class does not have a clientFinalizer
156 // instance: i.e. we're running on straight 0.20 w/o hadoop-4829.
157 }
158 if (field != null) {
159 field.setAccessible(true);
160 Field cacheField = FileSystem.class.getDeclaredField("CACHE");
161 cacheField.setAccessible(true);
162 Object cacheInstance = cacheField.get(fs);
163 hdfsClientFinalizer = (Thread)field.get(cacheInstance);
164 } else {
165 // Then we didnt' find clientFinalizer in Cache. Presume clean 0.20 hadoop.
166 field = FileSystem.class.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
167 field.setAccessible(true);
168 hdfsClientFinalizer = (Thread)field.get(null);
169 }
170 if (hdfsClientFinalizer == null) {
171 throw new RuntimeException("Client finalizer is null, can't suppress!");
172 }
173 if (!Runtime.getRuntime().removeShutdownHook(hdfsClientFinalizer)) {
174 throw new RuntimeException("Failed suppression of fs shutdown hook: " +
175 hdfsClientFinalizer);
176 }
177 return hdfsClientFinalizer;
178 } catch (NoSuchFieldException nsfe) {
179 LOG.fatal("Couldn't find field 'clientFinalizer' in FileSystem!", nsfe);
180 throw new RuntimeException("Failed to suppress HDFS shutdown hook");
181 } catch (IllegalAccessException iae) {
182 LOG.fatal("Couldn't access field 'clientFinalizer' in FileSystem!", iae);
183 throw new RuntimeException("Failed to suppress HDFS shutdown hook");
184 }
185 }
186
187 // Thread that does nothing. Used in below main testing.
188 static class DoNothingThread extends Thread {
189 DoNothingThread() {
190 super("donothing");
191 }
192 @Override
193 public void run() {
194 super.run();
195 }
196 }
197
198 // Stoppable with nothing to stop. Used below in main testing.
199 static class DoNothingStoppable implements Stoppable {
200 @Override
201 public void stop() {
202 // Nothing to do.
203 }
204 }
205
206 /**
207 * Main to test basic functionality. Run with clean hadoop 0.20 and hadoop
208 * 0.21 and cloudera patched hadoop to make sure our shutdown hook handling
209 * works for all compbinations.
210 * Pass '-Dhbase.shutdown.hook=false' to test turning off the running of
211 * shutdown hooks.
212 * @param args
213 * @throws IOException
214 */
215 public static void main(final String [] args) throws IOException {
216 Configuration conf = HBaseConfiguration.create();
217 String prop = System.getProperty(RUN_SHUTDOWN_HOOK);
218 if (prop != null) {
219 conf.setBoolean(RUN_SHUTDOWN_HOOK, Boolean.parseBoolean(prop));
220 }
221 // Instantiate a FileSystem. This will register the fs shutdown hook.
222 FileSystem fs = FileSystem.get(conf);
223 Thread donothing = new DoNothingThread();
224 donothing.start();
225 ShutdownHook.install(conf, fs, new DoNothingStoppable(), donothing);
226 }
227 }