1   /**
2    * Copyright 2008 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase;
21  
22  import java.io.IOException;
23  import java.security.PrivilegedAction;
24  import java.util.ArrayList;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.concurrent.ConcurrentHashMap;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.fs.FileSystem;
33  import org.apache.hadoop.hbase.client.HConnectionManager;
34  import org.apache.hadoop.hbase.master.HMaster;
35  import org.apache.hadoop.hbase.regionserver.HRegion;
36  import org.apache.hadoop.hbase.regionserver.HRegionServer;
37  import org.apache.hadoop.hbase.util.Bytes;
38  import org.apache.hadoop.hbase.util.JVMClusterUtil;
39  import org.apache.hadoop.hbase.util.Threads;
40  import org.apache.hadoop.hdfs.DistributedFileSystem;
41  import org.apache.hadoop.io.MapWritable;
42  import org.apache.hadoop.security.UserGroupInformation;
43  
44  /**
45   * This class creates a single process HBase cluster.
46   * each server.  The master uses the 'default' FileSystem.  The RegionServers,
47   * if we are running on DistributedFilesystem, create a FileSystem instance
48   * each and will close down their instance on the way out.
49   */
50  public class MiniHBaseCluster {
51    static final Log LOG = LogFactory.getLog(MiniHBaseCluster.class.getName());
52    private Configuration conf;
53    public LocalHBaseCluster hbaseCluster;
54  
55    /**
56     * Start a MiniHBaseCluster.
57     * @param conf Configuration to be used for cluster
58     * @param numRegionServers initial number of region servers to start.
59     * @throws IOException
60     */
61    public MiniHBaseCluster(Configuration conf, int numRegionServers)
62    throws IOException {
63      this.conf = conf;
64      conf.set(HConstants.MASTER_PORT, "0");
65      init(numRegionServers);
66    }
67  
68    /**
69     * Override Master so can add inject behaviors testing.
70     */
71    public static class MiniHBaseClusterMaster extends HMaster {
72      private final Map<HServerInfo, List<HMsg>> messages =
73        new ConcurrentHashMap<HServerInfo, List<HMsg>>();
74  
75      private final Map<HServerInfo, IOException> exceptions =
76        new ConcurrentHashMap<HServerInfo, IOException>();
77  
78      public MiniHBaseClusterMaster(final Configuration conf)
79      throws IOException {
80        super(conf);
81      }
82  
83      /**
84       * Add a message to send to a regionserver next time it checks in.
85       * @param hsi RegionServer's HServerInfo.
86       * @param msg Message to add.
87       */
88      void addMessage(final HServerInfo hsi, HMsg msg) {
89        synchronized(this.messages) {
90          List<HMsg> hmsgs = this.messages.get(hsi);
91          if (hmsgs == null) {
92            hmsgs = new ArrayList<HMsg>();
93            this.messages.put(hsi, hmsgs);
94          }
95          hmsgs.add(msg);
96        }
97      }
98  
99      void addException(final HServerInfo hsi, final IOException ex) {
100       this.exceptions.put(hsi, ex);
101     }
102 
103     /**
104      * This implementation is special, exceptions will be treated first and
105      * message won't be sent back to the region servers even if some are
106      * specified.
107      * @param hsi the rs
108      * @param msgs Messages to add to
109      * @return
110      * @throws IOException will be throw if any added for this region server
111      */
112     @Override
113     protected HMsg[] adornRegionServerAnswer(final HServerInfo hsi,
114         final HMsg[] msgs) throws IOException {
115       IOException ex = this.exceptions.remove(hsi);
116       if (ex != null) {
117         throw ex;
118       }
119       HMsg [] answerMsgs = msgs;
120       synchronized (this.messages) {
121         List<HMsg> hmsgs = this.messages.get(hsi);
122         if (hmsgs != null && !hmsgs.isEmpty()) {
123           int size = answerMsgs.length;
124           HMsg [] newAnswerMsgs = new HMsg[size + hmsgs.size()];
125           System.arraycopy(answerMsgs, 0, newAnswerMsgs, 0, answerMsgs.length);
126           for (int i = 0; i < hmsgs.size(); i++) {
127             newAnswerMsgs[answerMsgs.length + i] = hmsgs.get(i);
128           }
129           answerMsgs = newAnswerMsgs;
130           hmsgs.clear();
131         }
132       }
133       return super.adornRegionServerAnswer(hsi, answerMsgs);
134     }
135   }
136 
137   /**
138    * Subclass so can get at protected methods (none at moment).  Also, creates
139    * a FileSystem instance per instantiation.  Adds a shutdown own FileSystem
140    * on the way out. Shuts down own Filesystem only, not All filesystems as
141    * the FileSystem system exit hook does.
142    */
143   public static class MiniHBaseClusterRegionServer extends HRegionServer {
144     private Thread shutdownThread = null;
145     private UserGroupInformation user = null;
146 
147     public MiniHBaseClusterRegionServer(Configuration conf)
148         throws IOException {
149       super(conf);
150       this.user = UserGroupInformation.getCurrentUser();
151     }
152 
153     public void setHServerInfo(final HServerInfo hsi) {
154       this.serverInfo = hsi;
155     }
156 
157     /*
158      * @param c
159      * @param currentfs We return this if we did not make a new one.
160      * @param uniqueName Same name used to help identify the created fs.
161      * @return A new fs instance if we are up on DistributeFileSystem.
162      * @throws IOException
163      */
164 
165     @Override
166     protected void init(MapWritable c) throws IOException {
167       super.init(c);
168       // Run this thread to shutdown our filesystem on way out.
169       this.shutdownThread = new SingleFileSystemShutdownThread(getFileSystem());
170     }
171 
172     @Override
173     public void run() {
174       try {
175         this.user.doAs(new PrivilegedAction<Object>(){
176           public Object run() {
177             runRegionServer();
178             return null;
179           }
180         });
181       } finally {
182         // Run this on the way out.
183         if (this.shutdownThread != null) {
184           this.shutdownThread.start();
185           Threads.shutdown(this.shutdownThread, 30000);
186         }
187       }
188     }
189 
190     private void runRegionServer() {
191       super.run();
192     }
193 
194     public void kill() {
195       super.kill();
196     }
197   }
198 
199   /**
200    * Alternate shutdown hook.
201    * Just shuts down the passed fs, not all as default filesystem hook does.
202    */
203   static class SingleFileSystemShutdownThread extends Thread {
204     private final FileSystem fs;
205     SingleFileSystemShutdownThread(final FileSystem fs) {
206       super("Shutdown of " + fs);
207       this.fs = fs;
208     }
209     @Override
210     public void run() {
211       try {
212         LOG.info("Hook closing fs=" + this.fs);
213         this.fs.close();
214       } catch (IOException e) {
215         LOG.warn("Running hook", e);
216       }
217     }
218   }
219 
220   private void init(final int nRegionNodes) throws IOException {
221     try {
222       // start up a LocalHBaseCluster
223       hbaseCluster = new LocalHBaseCluster(conf, nRegionNodes,
224           MiniHBaseCluster.MiniHBaseClusterMaster.class,
225           MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
226       hbaseCluster.startup();
227     } catch(IOException e) {
228       shutdown();
229       throw e;
230     }
231   }
232 
233   /**
234    * Starts a region server thread running
235    *
236    * @throws IOException
237    * @return New RegionServerThread
238    */
239   public JVMClusterUtil.RegionServerThread startRegionServer() throws IOException {
240     JVMClusterUtil.RegionServerThread t = this.hbaseCluster.addRegionServer();
241     t.start();
242     t.waitForServerOnline();
243     return t;
244   }
245 
246   /**
247    * @return Returns the rpc address actually used by the master server, because
248    * the supplied port is not necessarily the actual port used.
249    */
250   public HServerAddress getHMasterAddress() {
251     return this.hbaseCluster.getMaster().getMasterAddress();
252   }
253 
254   /**
255    * @return the HMaster
256    */
257   public HMaster getMaster() {
258     return this.hbaseCluster.getMaster();
259   }
260 
261   /**
262    * Cause a region server to exit doing basic clean up only on its way out.
263    * @param serverNumber  Used as index into a list.
264    */
265   public String abortRegionServer(int serverNumber) {
266     HRegionServer server = getRegionServer(serverNumber);
267     LOG.info("Aborting " + server.toString());
268     server.abort("Aborting for tests", new Exception("Trace info"));
269     return server.toString();
270   }
271 
272   /**
273    * Shut down the specified region server cleanly
274    *
275    * @param serverNumber  Used as index into a list.
276    * @return the region server that was stopped
277    */
278   public JVMClusterUtil.RegionServerThread stopRegionServer(int serverNumber) {
279     return stopRegionServer(serverNumber, true);
280   }
281 
282   /**
283    * Shut down the specified region server cleanly
284    *
285    * @param serverNumber  Used as index into a list.
286    * @param shutdownFS True is we are to shutdown the filesystem as part of this
287    * regionserver's shutdown.  Usually we do but you do not want to do this if
288    * you are running multiple regionservers in a test and you shut down one
289    * before end of the test.
290    * @return the region server that was stopped
291    */
292   public JVMClusterUtil.RegionServerThread stopRegionServer(int serverNumber,
293       final boolean shutdownFS) {
294     JVMClusterUtil.RegionServerThread server =
295       hbaseCluster.getRegionServers().get(serverNumber);
296     LOG.info("Stopping " + server.toString());
297     server.getRegionServer().stop();
298     return server;
299   }
300 
301   /**
302    * Wait for the specified region server to stop. Removes this thread from list
303    * of running threads.
304    * @param serverNumber
305    * @return Name of region server that just went down.
306    */
307   public String waitOnRegionServer(final int serverNumber) {
308     return this.hbaseCluster.waitOnRegionServer(serverNumber);
309   }
310 
311   /**
312    * Wait for Mini HBase Cluster to shut down.
313    */
314   public void join() {
315     this.hbaseCluster.join();
316   }
317 
318   /**
319    * Shut down the mini HBase cluster
320    * @throws IOException
321    */
322   public void shutdown() throws IOException {
323     if (this.hbaseCluster != null) {
324       this.hbaseCluster.shutdown();
325     }
326     HConnectionManager.deleteAllConnections(false);
327   }
328 
329   /**
330    * Call flushCache on all regions on all participating regionservers.
331    * @throws IOException
332    */
333   public void flushcache() throws IOException {
334     for (JVMClusterUtil.RegionServerThread t:
335         this.hbaseCluster.getRegionServers()) {
336       for(HRegion r: t.getRegionServer().getOnlineRegions()) {
337         r.flushcache();
338       }
339     }
340   }
341 
342   /**
343    * Call flushCache on all regions of the specified table.
344    * @throws IOException
345    */
346   public void flushcache(byte [] tableName) throws IOException {
347     for (JVMClusterUtil.RegionServerThread t:
348         this.hbaseCluster.getRegionServers()) {
349       for(HRegion r: t.getRegionServer().getOnlineRegions()) {
350         if(Bytes.equals(r.getTableDesc().getName(), tableName)) {
351           r.flushcache();
352         }
353       }
354     }
355   }
356 
357   /**
358    * @return List of region server threads.
359    */
360   public List<JVMClusterUtil.RegionServerThread> getRegionServerThreads() {
361     return this.hbaseCluster.getRegionServers();
362   }
363 
364   /**
365    * @return List of live region server threads (skips the aborted and the killed)
366    */
367   public List<JVMClusterUtil.RegionServerThread> getLiveRegionServerThreads() {
368     return this.hbaseCluster.getLiveRegionServers();
369   }
370 
371   /**
372    * Grab a numbered region server of your choice.
373    * @param serverNumber
374    * @return region server
375    */
376   public HRegionServer getRegionServer(int serverNumber) {
377     return hbaseCluster.getRegionServer(serverNumber);
378   }
379 
380   public List<HRegion> getRegions(byte[] tableName) {
381     List<HRegion> ret = new ArrayList<HRegion>();
382     for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
383       HRegionServer hrs = rst.getRegionServer();
384       for (HRegion region : hrs.getOnlineRegions()) {
385         if (Bytes.equals(region.getTableDesc().getName(), tableName)) {
386           ret.add(region);
387         }
388       }
389     }
390     return ret;
391   }
392 
393   /**
394    * @return Index into List of {@link MiniHBaseCluster#getRegionServerThreads()}
395    * of HRS carrying regionName. Returns -1 if none found.
396    */
397   public int getServerWithMeta() {
398     return getServerWith(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
399   }
400 
401   /**
402    * Get the location of the specified region
403    * @param regionName Name of the region in bytes
404    * @return Index into List of {@link MiniHBaseCluster#getRegionServerThreads()}
405    * of HRS carrying .META.. Returns -1 if none found.
406    */
407   public int getServerWith(byte[] regionName) {
408     int index = -1;
409     int count = 0;
410     for (JVMClusterUtil.RegionServerThread rst: getRegionServerThreads()) {
411       HRegionServer hrs = rst.getRegionServer();
412       HRegion metaRegion =
413         hrs.getOnlineRegion(regionName);
414       if (metaRegion != null) {
415         index = count;
416         break;
417       }
418       count++;
419     }
420     return index;
421   }
422 
423   /**
424    * Add an exception to send when a region server checks back in
425    * @param serverNumber Which server to send it to
426    * @param ex The exception that will be sent
427    * @throws IOException
428    */
429   public void addExceptionToSendRegionServer(final int serverNumber,
430       IOException ex) throws IOException {
431     MiniHBaseClusterRegionServer hrs =
432       (MiniHBaseClusterRegionServer)getRegionServer(serverNumber);
433     addExceptionToSendRegionServer(hrs, ex);
434   }
435 
436   /**
437    * Add an exception to send when a region server checks back in
438    * @param hrs Which server to send it to
439    * @param ex The exception that will be sent
440    * @throws IOException
441    */
442   public void addExceptionToSendRegionServer(
443       final MiniHBaseClusterRegionServer hrs, IOException ex)
444       throws IOException {
445     ((MiniHBaseClusterMaster)getMaster()).addException(hrs.getHServerInfo(),ex);
446   }
447 
448   /**
449    * Add a message to include in the responses send a regionserver when it
450    * checks back in.
451    * @param serverNumber Which server to send it to.
452    * @param msg The MESSAGE
453    * @throws IOException
454    */
455   public void addMessageToSendRegionServer(final int serverNumber,
456     final HMsg msg)
457   throws IOException {
458     MiniHBaseClusterRegionServer hrs =
459       (MiniHBaseClusterRegionServer)getRegionServer(serverNumber);
460     addMessageToSendRegionServer(hrs, msg);
461   }
462 
463   /**
464    * Add a message to include in the responses send a regionserver when it
465    * checks back in.
466    * @param hrs Which region server.
467    * @param msg The MESSAGE
468    * @throws IOException
469    */
470   public void addMessageToSendRegionServer(final MiniHBaseClusterRegionServer hrs,
471     final HMsg msg)
472   throws IOException {
473     ((MiniHBaseClusterMaster)getMaster()).addMessage(hrs.getHServerInfo(), msg);
474   }
475 }