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
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
34
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
42
43 public static final String RUN_SHUTDOWN_HOOK = "hbase.shutdown.hook";
44
45
46
47
48
49 public static final String FS_SHUTDOWN_HOOK_WAIT = "hbase.fs.shutdown.hook.wait";
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
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
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
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132 private static Thread suppressHdfsShutdownHook(final FileSystem fs) {
133 try {
134
135
136
137
138
139
140
141 Thread hdfsClientFinalizer = null;
142
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
156
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
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
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
199 static class DoNothingStoppable implements Stoppable {
200 @Override
201 public void stop() {
202
203 }
204 }
205
206
207
208
209
210
211
212
213
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
222 FileSystem fs = FileSystem.get(conf);
223 Thread donothing = new DoNothingThread();
224 donothing.start();
225 ShutdownHook.install(conf, fs, new DoNothingStoppable(), donothing);
226 }
227 }