View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.util;
20  
21  import java.io.InterruptedIOException;
22  import java.io.IOException;
23  import java.io.PrintWriter;
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.classification.InterfaceAudience;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.hbase.CoordinatedStateManager;
33  import org.apache.hadoop.hbase.master.HMaster;
34  import org.apache.hadoop.hbase.regionserver.HRegionServer;
35  import org.apache.hadoop.util.ReflectionUtils;
36  
37  /**
38   * Utility used running a cluster all in the one JVM.
39   */
40  @InterfaceAudience.Private
41  public class JVMClusterUtil {
42    private static final Log LOG = LogFactory.getLog(JVMClusterUtil.class);
43  
44    /**
45     * Datastructure to hold RegionServer Thread and RegionServer instance
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      /** @return the region server */
56      public HRegionServer getRegionServer() {
57        return this.regionServer;
58      }
59  
60      /**
61       * Block until the region server has come online, indicating it is ready
62       * to be used.
63       */
64      public void waitForServerOnline() {
65        // The server is marked online after the init method completes inside of
66        // the HRS#run method.  HRS#init can fail for whatever region.  In those
67        // cases, we'll jump out of the run without setting online flag.  Check
68        // stopRequested so we don't wait here a flag that will never be flipped.
69        regionServer.waitForServerOnline();
70      }
71    }
72  
73    /**
74     * Creates a {@link RegionServerThread}.
75     * Call 'start' on the returned thread to make it run.
76     * @param c Configuration to use.
77     * @param cp consensus provider to use
78     * @param hrsc Class to create.
79     * @param index Used distinguishing the object returned.
80     * @throws IOException
81     * @return Region server added.
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       IOException ioe = new IOException();
101       ioe.initCause(e);
102       throw ioe;
103     }
104     return new JVMClusterUtil.RegionServerThread(server, index);
105   }
106 
107 
108   /**
109    * Datastructure to hold Master Thread and Master instance
110    */
111   public static class MasterThread extends Thread {
112     private final HMaster master;
113 
114     public MasterThread(final HMaster m, final int index) {
115       super(m, "M:" + index + ";" + m.getServerName().toShortString());
116       this.master = m;
117     }
118 
119     /** @return the master */
120     public HMaster getMaster() {
121       return this.master;
122     }
123   }
124 
125   /**
126    * Creates a {@link MasterThread}.
127    * Call 'start' on the returned thread to make it run.
128    * @param c Configuration to use.
129    * @param cp consensus provider to use
130    * @param hmc Class to create.
131    * @param index Used distinguishing the object returned.
132    * @throws IOException
133    * @return Master added.
134    */
135   public static JVMClusterUtil.MasterThread createMasterThread(
136       final Configuration c, CoordinatedStateManager cp, final Class<? extends HMaster> hmc,
137       final int index)
138   throws IOException {
139     HMaster server;
140     try {
141       server = hmc.getConstructor(Configuration.class, CoordinatedStateManager.class).
142         newInstance(c, cp);
143     } catch (InvocationTargetException ite) {
144       Throwable target = ite.getTargetException();
145       throw new RuntimeException("Failed construction of Master: " +
146         hmc.toString() + ((target.getCause() != null)?
147           target.getCause().getMessage(): ""), target);
148     } catch (Exception e) {
149       IOException ioe = new IOException();
150       ioe.initCause(e);
151       throw ioe;
152     }
153     return new JVMClusterUtil.MasterThread(server, index);
154   }
155 
156   private static JVMClusterUtil.MasterThread findActiveMaster(
157     List<JVMClusterUtil.MasterThread> masters) {
158     for (JVMClusterUtil.MasterThread t : masters) {
159       if (t.master.isActiveMaster()) {
160         return t;
161       }
162     }
163 
164     return null;
165   }
166 
167   /**
168    * Start the cluster.  Waits until there is a primary master initialized
169    * and returns its address.
170    * @param masters
171    * @param regionservers
172    * @return Address to use contacting primary master.
173    */
174   public static String startup(final List<JVMClusterUtil.MasterThread> masters,
175       final List<JVMClusterUtil.RegionServerThread> regionservers) throws IOException {
176 
177     if (masters == null || masters.isEmpty()) {
178       return null;
179     }
180 
181     for (JVMClusterUtil.MasterThread t : masters) {
182       t.start();
183     }
184 
185     // Wait for an active master
186     //  having an active master before starting the region threads allows
187     //  then to succeed on their connection to master
188     long startTime = System.currentTimeMillis();
189     while (findActiveMaster(masters) == null) {
190       try {
191         Thread.sleep(100);
192       } catch (InterruptedException e) {
193         throw (InterruptedIOException)new InterruptedIOException().initCause(e);
194       }
195       if (System.currentTimeMillis() > startTime + 30000) {
196         throw new RuntimeException("Master not active after 30 seconds");
197       }
198     }
199 
200     if (regionservers != null) {
201       for (JVMClusterUtil.RegionServerThread t: regionservers) {
202         t.start();
203       }
204     }
205 
206     // Wait for an active master to be initialized (implies being master)
207     //  with this, when we return the cluster is complete
208     startTime = System.currentTimeMillis();
209     final int maxwait = 200000;
210     while (true) {
211       JVMClusterUtil.MasterThread t = findActiveMaster(masters);
212       if (t != null && t.master.isInitialized()) {
213         return t.master.getServerName().toString();
214       }
215       // REMOVE
216       if (System.currentTimeMillis() > startTime + 10000) {
217         try {
218           Thread.sleep(1000);
219         } catch (InterruptedException e) {
220           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
221         }
222       }
223       if (System.currentTimeMillis() > startTime + maxwait) {
224         String msg = "Master not initialized after " + maxwait + "ms seconds";
225         ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
226           "Thread dump because: " + msg);
227         throw new RuntimeException(msg);
228       }
229       try {
230         Thread.sleep(100);
231       } catch (InterruptedException e) {
232         throw (InterruptedIOException)new InterruptedIOException().initCause(e);
233       }
234     }
235   }
236 
237   /**
238    * @param masters
239    * @param regionservers
240    */
241   public static void shutdown(final List<MasterThread> masters,
242       final List<RegionServerThread> regionservers) {
243     LOG.debug("Shutting down HBase Cluster");
244     if (masters != null) {
245       // Do backups first.
246       JVMClusterUtil.MasterThread activeMaster = null;
247       for (JVMClusterUtil.MasterThread t : masters) {
248         if (!t.master.isActiveMaster()) {
249           t.master.stopMaster();
250         } else {
251           activeMaster = t;
252         }
253       }
254       // Do active after.
255       if (activeMaster != null)
256         activeMaster.master.shutdown();
257 
258     }
259     boolean wasInterrupted = false;
260     final long maxTime = System.currentTimeMillis() + 30 * 1000;
261     if (regionservers != null) {
262       // first try nicely.
263       for (RegionServerThread t : regionservers) {
264         t.getRegionServer().stop("Shutdown requested");
265       }
266       for (RegionServerThread t : regionservers) {
267         long now = System.currentTimeMillis();
268         if (t.isAlive() && !wasInterrupted && now < maxTime) {
269           try {
270             t.join(maxTime - now);
271           } catch (InterruptedException e) {
272             LOG.info("Got InterruptedException on shutdown - " +
273                 "not waiting anymore on region server ends", e);
274             wasInterrupted = true; // someone wants us to speed up.
275           }
276         }
277       }
278 
279       // Let's try to interrupt the remaining threads if any.
280       for (int i = 0; i < 100; ++i) {
281         boolean atLeastOneLiveServer = false;
282         for (RegionServerThread t : regionservers) {
283           if (t.isAlive()) {
284             atLeastOneLiveServer = true;
285             try {
286               LOG.warn("RegionServerThreads remaining, give one more chance before interrupting");
287               t.join(1000);
288             } catch (InterruptedException e) {
289               wasInterrupted = true;
290             }
291           }
292         }
293         if (!atLeastOneLiveServer) break;
294         for (RegionServerThread t : regionservers) {
295           if (t.isAlive()) {
296             LOG.warn("RegionServerThreads taking too long to stop, interrupting");
297             t.interrupt();
298           }
299         }
300       }
301     }
302 
303     if (masters != null) {
304       for (JVMClusterUtil.MasterThread t : masters) {
305         while (t.master.isAlive() && !wasInterrupted) {
306           try {
307             // The below has been replaced to debug sometime hangs on end of
308             // tests.
309             // this.master.join():
310             Threads.threadDumpingIsAlive(t.master.getThread());
311           } catch(InterruptedException e) {
312             LOG.info("Got InterruptedException on shutdown - " +
313                 "not waiting anymore on master ends", e);
314             wasInterrupted = true;
315           }
316         }
317       }
318     }
319     LOG.info("Shutdown of " +
320       ((masters != null) ? masters.size() : "0") + " master(s) and " +
321       ((regionservers != null) ? regionservers.size() : "0") +
322       " regionserver(s) " + (wasInterrupted ? "interrupted" : "complete"));
323 
324     if (wasInterrupted){
325       Thread.currentThread().interrupt();
326     }
327   }
328 }