1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.io.IOException;
23 import java.lang.reflect.Field;
24 import java.util.HashMap;
25 import java.util.Map;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
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;
35
36
37
38
39
40 public class ShutdownHook {
41 private static final Log LOG = LogFactory.getLog(ShutdownHook.class);
42 private static final String CLIENT_FINALIZER_DATA_METHOD = "clientFinalizer";
43
44
45
46
47 public static final String RUN_SHUTDOWN_HOOK = "hbase.shutdown.hook";
48
49
50
51
52
53 public static final String FS_SHUTDOWN_HOOK_WAIT = "hbase.fs.shutdown.hook.wait";
54
55
56
57
58
59
60 private final static Map<Runnable, Integer> fsShutdownHooks = new HashMap<Runnable, Integer>();
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81 public static void install(final Configuration conf, final FileSystem fs,
82 final Stoppable stop, final Thread threadToJoin) {
83 Runnable fsShutdownHook = suppressHdfsShutdownHook(fs);
84 Thread t = new ShutdownHookThread(conf, stop, threadToJoin, fsShutdownHook);
85 ShutdownHookManager.affixShutdownHook(t, 0);
86 LOG.info("Installed shutdown hook thread: " + t.getName());
87 }
88
89
90
91
92 private static class ShutdownHookThread extends Thread {
93 private final Stoppable stop;
94 private final Thread threadToJoin;
95 private final Runnable fsShutdownHook;
96 private final Configuration conf;
97
98 ShutdownHookThread(final Configuration conf, final Stoppable stop,
99 final Thread threadToJoin, final Runnable fsShutdownHook) {
100 super("Shutdownhook:" + threadToJoin.getName());
101 this.stop = stop;
102 this.threadToJoin = threadToJoin;
103 this.conf = conf;
104 this.fsShutdownHook = fsShutdownHook;
105 }
106
107 @Override
108 public void run() {
109 boolean b = this.conf.getBoolean(RUN_SHUTDOWN_HOOK, true);
110 LOG.info("Shutdown hook starting; " + RUN_SHUTDOWN_HOOK + "=" + b +
111 "; fsShutdownHook=" + this.fsShutdownHook);
112 if (b) {
113 this.stop.stop("Shutdown hook");
114 Threads.shutdown(this.threadToJoin);
115 if (this.fsShutdownHook != null) {
116 synchronized (fsShutdownHooks) {
117 int refs = fsShutdownHooks.get(fsShutdownHook);
118 if (refs == 1) {
119 LOG.info("Starting fs shutdown hook thread.");
120 Thread fsShutdownHookThread = (fsShutdownHook instanceof Thread) ?
121 (Thread)fsShutdownHook : new Thread(fsShutdownHook);
122 fsShutdownHookThread.start();
123 Threads.shutdown(fsShutdownHookThread,
124 this.conf.getLong(FS_SHUTDOWN_HOOK_WAIT, 30000));
125 }
126 if (refs > 0) {
127 fsShutdownHooks.put(fsShutdownHook, refs - 1);
128 }
129 }
130 }
131 }
132 LOG.info("Shutdown hook finished.");
133 }
134 }
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153 private static Runnable suppressHdfsShutdownHook(final FileSystem fs) {
154 try {
155
156
157
158
159
160
161
162 Runnable hdfsClientFinalizer = null;
163
164 Class<?> [] classes = FileSystem.class.getDeclaredClasses();
165 Class<?> cache = null;
166 for (Class<?> c: classes) {
167 if (c.getSimpleName().equals("Cache")) {
168 cache = c;
169 break;
170 }
171 }
172 Field field = null;
173 try {
174 field = cache.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
175 } catch (NoSuchFieldException e) {
176
177
178 }
179 if (field != null) {
180 field.setAccessible(true);
181 Field cacheField = FileSystem.class.getDeclaredField("CACHE");
182 cacheField.setAccessible(true);
183 Object cacheInstance = cacheField.get(fs);
184 hdfsClientFinalizer = (Runnable)field.get(cacheInstance);
185 } else {
186
187 field = FileSystem.class.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
188 field.setAccessible(true);
189 hdfsClientFinalizer = (Runnable)field.get(null);
190 }
191 if (hdfsClientFinalizer == null) {
192 throw new RuntimeException("Client finalizer is null, can't suppress!");
193 }
194 synchronized (fsShutdownHooks) {
195 if (!fsShutdownHooks.containsKey(hdfsClientFinalizer) &&
196 !ShutdownHookManager.deleteShutdownHook(hdfsClientFinalizer)) {
197 throw new RuntimeException("Failed suppression of fs shutdown hook: " +
198 hdfsClientFinalizer);
199 }
200 Integer refs = fsShutdownHooks.get(hdfsClientFinalizer);
201 fsShutdownHooks.put(hdfsClientFinalizer, refs == null ? 1 : refs + 1);
202 }
203 return hdfsClientFinalizer;
204 } catch (NoSuchFieldException nsfe) {
205 LOG.fatal("Couldn't find field 'clientFinalizer' in FileSystem!", nsfe);
206 throw new RuntimeException("Failed to suppress HDFS shutdown hook");
207 } catch (IllegalAccessException iae) {
208 LOG.fatal("Couldn't access field 'clientFinalizer' in FileSystem!", iae);
209 throw new RuntimeException("Failed to suppress HDFS shutdown hook");
210 }
211 }
212
213
214 static class DoNothingThread extends Thread {
215 DoNothingThread() {
216 super("donothing");
217 }
218 @Override
219 public void run() {
220 super.run();
221 }
222 }
223
224
225 static class DoNothingStoppable implements Stoppable {
226 @Override
227 public boolean isStopped() {
228
229 return false;
230 }
231
232 @Override
233 public void stop(String why) {
234
235 }
236 }
237
238
239
240
241
242
243
244
245
246
247 public static void main(final String [] args) throws IOException {
248 Configuration conf = HBaseConfiguration.create();
249 String prop = System.getProperty(RUN_SHUTDOWN_HOOK);
250 if (prop != null) {
251 conf.setBoolean(RUN_SHUTDOWN_HOOK, Boolean.parseBoolean(prop));
252 }
253
254 FileSystem fs = FileSystem.get(conf);
255 Thread donothing = new DoNothingThread();
256 donothing.start();
257 ShutdownHook.install(conf, fs, new DoNothingStoppable(), donothing);
258 }
259 }