1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.util;
20
21 import com.google.common.base.Preconditions;
22 import java.io.InterruptedIOException;
23 import java.io.IOException;
24 import java.lang.reflect.Constructor;
25 import java.lang.reflect.InvocationTargetException;
26 import java.util.List;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.HConstants;
31 import org.apache.hadoop.hbase.classification.InterfaceAudience;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.CoordinatedStateManager;
34 import org.apache.hadoop.hbase.master.HMaster;
35 import org.apache.hadoop.hbase.regionserver.HRegionServer;
36
37
38
39
40 @InterfaceAudience.Private
41 public class JVMClusterUtil {
42 private static final Log LOG = LogFactory.getLog(JVMClusterUtil.class);
43
44
45
46
47 public static class RegionServerThread extends Thread {
48 private final HRegionServer regionServer;
49
50 public RegionServerThread(final HRegionServer r, final int index) {
51 super(r, "RS:" + index + ";" + r.getServerName().toShortString());
52 this.regionServer = r;
53 }
54
55
56 public HRegionServer getRegionServer() {
57 return this.regionServer;
58 }
59
60
61
62
63
64 public void waitForServerOnline() {
65
66
67
68
69 regionServer.waitForServerOnline();
70 }
71 }
72
73
74
75
76
77
78
79
80
81
82
83 public static JVMClusterUtil.RegionServerThread createRegionServerThread(
84 final Configuration c, CoordinatedStateManager cp, final Class<? extends HRegionServer> hrsc,
85 final int index)
86 throws IOException {
87 HRegionServer server;
88 try {
89
90 Constructor<? extends HRegionServer> ctor = hrsc.getConstructor(Configuration.class,
91 CoordinatedStateManager.class);
92 ctor.setAccessible(true);
93 server = ctor.newInstance(c, cp);
94 } catch (InvocationTargetException ite) {
95 Throwable target = ite.getTargetException();
96 throw new RuntimeException("Failed construction of RegionServer: " +
97 hrsc.toString() + ((target.getCause() != null)?
98 target.getCause().getMessage(): ""), target);
99 } catch (Exception e) {
100 throw new IOException(e);
101 }
102 return new JVMClusterUtil.RegionServerThread(server, index);
103 }
104
105
106
107
108
109 public static class MasterThread extends Thread {
110 private final HMaster master;
111
112 public MasterThread(final HMaster m, final int index) {
113 super(m, "M:" + index + ";" + m.getServerName().toShortString());
114 this.master = m;
115 }
116
117
118 public HMaster getMaster() {
119 return this.master;
120 }
121 }
122
123
124
125
126
127
128
129
130
131
132
133 public static JVMClusterUtil.MasterThread createMasterThread(
134 final Configuration c, CoordinatedStateManager cp, final Class<? extends HMaster> hmc,
135 final int index)
136 throws IOException {
137 HMaster server;
138 try {
139 server = hmc.getConstructor(Configuration.class, CoordinatedStateManager.class).
140 newInstance(c, cp);
141 } catch (InvocationTargetException ite) {
142 Throwable target = ite.getTargetException();
143 throw new RuntimeException("Failed construction of Master: " +
144 hmc.toString() + ((target.getCause() != null)?
145 target.getCause().getMessage(): ""), target);
146 } catch (Exception e) {
147 throw new IOException(e);
148 }
149
150
151
152 c.set(HConstants.MASTER_ADDRS_KEY,
153 Preconditions.checkNotNull(server.getServerName().getAddress()).toString());
154 return new JVMClusterUtil.MasterThread(server, index);
155 }
156
157 private static JVMClusterUtil.MasterThread findActiveMaster(
158 List<JVMClusterUtil.MasterThread> masters) {
159 for (JVMClusterUtil.MasterThread t : masters) {
160 if (t.master.isActiveMaster()) {
161 return t;
162 }
163 }
164
165 return null;
166 }
167
168
169
170
171
172
173
174
175 public static String startup(final List<JVMClusterUtil.MasterThread> masters,
176 final List<JVMClusterUtil.RegionServerThread> regionservers) throws IOException {
177
178 Configuration configuration = null;
179
180 if (masters == null || masters.isEmpty()) {
181 return null;
182 }
183
184 for (JVMClusterUtil.MasterThread t : masters) {
185 configuration = t.getMaster().getConfiguration();
186 t.start();
187 }
188
189
190
191
192 long startTime = System.currentTimeMillis();
193 while (findActiveMaster(masters) == null) {
194 try {
195 Thread.sleep(100);
196 } catch (InterruptedException e) {
197 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
198 }
199 int startTimeout = configuration != null ? Integer.parseInt(
200 configuration.get("hbase.master.start.timeout.localHBaseCluster", "30000")) : 30000;
201 if (System.currentTimeMillis() > startTime + startTimeout) {
202 throw new RuntimeException(String.format("Master not active after %s seconds", startTimeout));
203 }
204 }
205
206 if (regionservers != null) {
207 for (JVMClusterUtil.RegionServerThread t: regionservers) {
208 t.start();
209 }
210 }
211
212
213
214 startTime = System.currentTimeMillis();
215 final int maxwait = 200000;
216 while (true) {
217 JVMClusterUtil.MasterThread t = findActiveMaster(masters);
218 if (t != null && t.master.isInitialized()) {
219 return t.master.getServerName().toString();
220 }
221
222 if (System.currentTimeMillis() > startTime + 10000) {
223 try {
224 Thread.sleep(1000);
225 } catch (InterruptedException e) {
226 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
227 }
228 }
229 if (System.currentTimeMillis() > startTime + maxwait) {
230 String msg = "Master not initialized after " + maxwait + "ms seconds";
231 Threads.printThreadInfo(System.out,
232 "Thread dump because: " + msg);
233 throw new RuntimeException(msg);
234 }
235 try {
236 Thread.sleep(100);
237 } catch (InterruptedException e) {
238 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
239 }
240 }
241 }
242
243
244
245
246
247 public static void shutdown(final List<MasterThread> masters,
248 final List<RegionServerThread> regionservers) {
249 LOG.debug("Shutting down HBase Cluster");
250 if (masters != null) {
251
252 JVMClusterUtil.MasterThread activeMaster = null;
253 for (JVMClusterUtil.MasterThread t : masters) {
254 if (!t.master.isActiveMaster()) {
255 try {
256 t.master.stopMaster();
257 } catch (IOException e) {
258 LOG.error("Exception occurred while stopping master", e);
259 }
260 } else {
261 activeMaster = t;
262 }
263 }
264
265 if (activeMaster != null) {
266 try {
267 activeMaster.master.shutdown();
268 } catch (IOException e) {
269 LOG.error("Exception occurred in HMaster.shutdown()", e);
270 }
271 }
272
273 }
274 boolean wasInterrupted = false;
275 final long maxTime = System.currentTimeMillis() + 30 * 1000;
276 if (regionservers != null) {
277
278 for (RegionServerThread t : regionservers) {
279 t.getRegionServer().stop("Shutdown requested");
280 }
281 for (RegionServerThread t : regionservers) {
282 long now = System.currentTimeMillis();
283 if (t.isAlive() && !wasInterrupted && now < maxTime) {
284 try {
285 t.join(maxTime - now);
286 } catch (InterruptedException e) {
287 LOG.info("Got InterruptedException on shutdown - " +
288 "not waiting anymore on region server ends", e);
289 wasInterrupted = true;
290 }
291 }
292 }
293
294
295 for (int i = 0; i < 100; ++i) {
296 boolean atLeastOneLiveServer = false;
297 for (RegionServerThread t : regionservers) {
298 if (t.isAlive()) {
299 atLeastOneLiveServer = true;
300 try {
301 LOG.warn("RegionServerThreads remaining, give one more chance before interrupting");
302 t.join(1000);
303 } catch (InterruptedException e) {
304 wasInterrupted = true;
305 }
306 }
307 }
308 if (!atLeastOneLiveServer) break;
309 for (RegionServerThread t : regionservers) {
310 if (t.isAlive()) {
311 LOG.warn("RegionServerThreads taking too long to stop, interrupting");
312 t.interrupt();
313 }
314 }
315 }
316 }
317
318 if (masters != null) {
319 for (JVMClusterUtil.MasterThread t : masters) {
320 while (t.master.isAlive() && !wasInterrupted) {
321 try {
322
323
324
325 Threads.threadDumpingIsAlive(t.master.getThread());
326 } catch(InterruptedException e) {
327 LOG.info("Got InterruptedException on shutdown - " +
328 "not waiting anymore on master ends", e);
329 wasInterrupted = true;
330 }
331 }
332 }
333 }
334 LOG.info("Shutdown of " +
335 ((masters != null) ? masters.size() : "0") + " master(s) and " +
336 ((regionservers != null) ? regionservers.size() : "0") +
337 " regionserver(s) " + (wasInterrupted ? "interrupted" : "complete"));
338
339 if (wasInterrupted){
340 Thread.currentThread().interrupt();
341 }
342 }
343 }