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