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