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  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.FileSystem;
31  import org.apache.hadoop.hbase.ipc.HRegionInterface;
32  import org.apache.hadoop.hbase.ipc.HMasterInterface;
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.security.User;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.hadoop.hbase.util.JVMClusterUtil;
40  import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
41  import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
42  import org.apache.hadoop.hbase.util.Threads;
43  import org.apache.hadoop.io.MapWritable;
44  
45  /**
46   * This class creates a single process HBase cluster.
47   * each server.  The master uses the 'default' FileSystem.  The RegionServers,
48   * if we are running on DistributedFilesystem, create a FileSystem instance
49   * each and will close down their instance on the way out.
50   */
51  public class MiniHBaseCluster extends HBaseCluster {
52    static final Log LOG = LogFactory.getLog(MiniHBaseCluster.class.getName());
53    public LocalHBaseCluster hbaseCluster;
54    private static int index;
55  
56    /**
57     * Start a MiniHBaseCluster.
58     * @param conf Configuration to be used for cluster
59     * @param numRegionServers initial number of region servers to start.
60     * @throws IOException
61     */
62    public MiniHBaseCluster(Configuration conf, int numRegionServers)
63    throws IOException, InterruptedException {
64      this(conf, 1, numRegionServers);
65    }
66  
67    /**
68     * Start a MiniHBaseCluster.
69     * @param conf Configuration to be used for cluster
70     * @param numMasters initial number of masters to start.
71     * @param numRegionServers initial number of region servers to start.
72     * @throws IOException
73     */
74    public MiniHBaseCluster(Configuration conf, int numMasters,
75                               int numRegionServers)
76        throws IOException, InterruptedException {
77      this(conf, numMasters, numRegionServers, null, null);
78    }
79  
80    public MiniHBaseCluster(Configuration conf, int numMasters, int numRegionServers,
81           Class<? extends HMaster> masterClass,
82           Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
83        throws IOException, InterruptedException {
84      super(conf);
85      conf.set(HConstants.MASTER_PORT, "0");
86      init(numMasters, numRegionServers, masterClass, regionserverClass);
87      this.initialClusterStatus = getClusterStatus();
88    }
89  
90    public Configuration getConfiguration() {
91      return this.conf;
92    }
93  
94    /**
95     * Subclass so can get at protected methods (none at moment).  Also, creates
96     * a FileSystem instance per instantiation.  Adds a shutdown own FileSystem
97     * on the way out. Shuts down own Filesystem only, not All filesystems as
98     * the FileSystem system exit hook does.
99     */
100   public static class MiniHBaseClusterRegionServer extends HRegionServer {
101     private Thread shutdownThread = null;
102     private User user = null;
103     public static boolean TEST_SKIP_CLOSE = false;
104 
105     public MiniHBaseClusterRegionServer(Configuration conf)
106         throws IOException, InterruptedException {
107       super(conf);
108       this.user = User.getCurrent();
109     }
110 
111     /*
112      * @param c
113      * @param currentfs We return this if we did not make a new one.
114      * @param uniqueName Same name used to help identify the created fs.
115      * @return A new fs instance if we are up on DistributeFileSystem.
116      * @throws IOException
117      */
118 
119     @Override
120     protected void handleReportForDutyResponse(MapWritable c) throws IOException {
121       super.handleReportForDutyResponse(c);
122       // Run this thread to shutdown our filesystem on way out.
123       this.shutdownThread = new SingleFileSystemShutdownThread(getFileSystem());
124     }
125 
126     @Override
127     public void run() {
128       try {
129         this.user.runAs(new PrivilegedAction<Object>(){
130           public Object run() {
131             runRegionServer();
132             return null;
133           }
134         });
135       } catch (Throwable t) {
136         LOG.error("Exception in run", t);
137       } finally {
138         // Run this on the way out.
139         if (this.shutdownThread != null) {
140           this.shutdownThread.start();
141           Threads.shutdown(this.shutdownThread, 30000);
142         }
143       }
144     }
145 
146     private void runRegionServer() {
147       super.run();
148     }
149 
150     @Override
151     public void kill() {
152       super.kill();
153     }
154 
155     public void abort(final String reason, final Throwable cause) {
156       this.user.runAs(new PrivilegedAction<Object>() {
157         public Object run() {
158           abortRegionServer(reason, cause);
159           return null;
160         }
161       });
162     }
163 
164     private void abortRegionServer(String reason, Throwable cause) {
165       super.abort(reason, cause);
166     }
167   }
168 
169   /**
170    * Alternate shutdown hook.
171    * Just shuts down the passed fs, not all as default filesystem hook does.
172    */
173   static class SingleFileSystemShutdownThread extends Thread {
174     private final FileSystem fs;
175     SingleFileSystemShutdownThread(final FileSystem fs) {
176       super("Shutdown of " + fs);
177       this.fs = fs;
178     }
179     @Override
180     public void run() {
181       try {
182         LOG.info("Hook closing fs=" + this.fs);
183         this.fs.close();
184       } catch (NullPointerException npe) {
185         LOG.debug("Need to fix these: " + npe.toString());
186       } catch (IOException e) {
187         LOG.warn("Running hook", e);
188       }
189     }
190   }
191 
192   private void init(final int nMasterNodes, final int nRegionNodes,
193           Class<? extends HMaster> masterClass,
194           Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
195   throws IOException, InterruptedException {
196     try {
197       if (masterClass == null){
198        masterClass =  HMaster.class;
199       }
200       if (regionserverClass == null){
201        regionserverClass = MiniHBaseCluster.MiniHBaseClusterRegionServer.class;
202       }
203 
204       // start up a LocalHBaseCluster
205       hbaseCluster = new LocalHBaseCluster(conf, nMasterNodes, 0,
206          masterClass, regionserverClass);
207 
208       // manually add the regionservers as other users
209       for (int i=0; i<nRegionNodes; i++) {
210         Configuration rsConf = HBaseConfiguration.create(conf);
211         User user = HBaseTestingUtility.getDifferentUser(rsConf,
212             ".hfs."+index++);
213         hbaseCluster.addRegionServer(rsConf, i, user);
214       }
215 
216       hbaseCluster.startup();
217     } catch (IOException e) {
218       shutdown();
219       throw e;
220     } catch (Throwable t) {
221       LOG.error("Error starting cluster", t);
222       shutdown();
223       throw new IOException("Shutting down", t);
224     }
225   }
226 
227   @Override
228   public void startRegionServer(String hostname) throws IOException {
229     this.startRegionServer();
230   }
231 
232   @Override
233   public void killRegionServer(ServerName serverName) throws IOException {
234     HRegionServer server = getRegionServer(getRegionServerIndex(serverName));
235     if (server instanceof MiniHBaseClusterRegionServer) {
236       LOG.info("Killing " + server.toString());
237       ((MiniHBaseClusterRegionServer) server).kill();
238     } else {
239       abortRegionServer(getRegionServerIndex(serverName));
240     }
241   }
242 
243   @Override
244   public void stopRegionServer(ServerName serverName) throws IOException {
245     stopRegionServer(getRegionServerIndex(serverName));
246   }
247 
248   @Override
249   public void waitForRegionServerToStop(ServerName serverName, long timeout) throws IOException {
250     //ignore timeout for now
251     waitOnRegionServer(getRegionServerIndex(serverName));
252   }
253 
254   @Override
255   public void startMaster(String hostname) throws IOException {
256     this.startMaster();
257   }
258 
259   @Override
260   public void killMaster(ServerName serverName) throws IOException {
261     abortMaster(getMasterIndex(serverName));
262   }
263 
264   @Override
265   public void stopMaster(ServerName serverName) throws IOException {
266     stopMaster(getMasterIndex(serverName));
267   }
268 
269   @Override
270   public void waitForMasterToStop(ServerName serverName, long timeout) throws IOException {
271     //ignore timeout for now
272     waitOnMaster(getMasterIndex(serverName));
273   }
274 
275   /**
276    * Starts a region server thread running
277    *
278    * @throws IOException
279    * @return New RegionServerThread
280    */
281   public JVMClusterUtil.RegionServerThread startRegionServer()
282       throws IOException {
283     final Configuration newConf = HBaseConfiguration.create(conf);
284     User rsUser =
285         HBaseTestingUtility.getDifferentUser(newConf, ".hfs."+index++);
286     JVMClusterUtil.RegionServerThread t =  null;
287     try {
288       t = hbaseCluster.addRegionServer(
289           newConf, hbaseCluster.getRegionServers().size(), rsUser);
290       t.start();
291       t.waitForServerOnline();
292     } catch (InterruptedException ie) {
293       throw new IOException("Interrupted adding regionserver to cluster", ie);
294     }
295     return t;
296   }
297 
298   /**
299    * Cause a region server to exit doing basic clean up only on its way out.
300    * @param serverNumber  Used as index into a list.
301    */
302   public String abortRegionServer(int serverNumber) {
303     HRegionServer server = getRegionServer(serverNumber);
304     LOG.info("Aborting " + server.toString());
305     server.abort("Aborting for tests", new Exception("Trace info"));
306     return server.toString();
307   }
308 
309   /**
310    * Shut down the specified region server cleanly
311    *
312    * @param serverNumber  Used as index into a list.
313    * @return the region server that was stopped
314    */
315   public JVMClusterUtil.RegionServerThread stopRegionServer(int serverNumber) {
316     return stopRegionServer(serverNumber, true);
317   }
318 
319   /**
320    * Shut down the specified region server cleanly
321    *
322    * @param serverNumber  Used as index into a list.
323    * @param shutdownFS True is we are to shutdown the filesystem as part of this
324    * regionserver's shutdown.  Usually we do but you do not want to do this if
325    * you are running multiple regionservers in a test and you shut down one
326    * before end of the test.
327    * @return the region server that was stopped
328    */
329   public JVMClusterUtil.RegionServerThread stopRegionServer(int serverNumber,
330       final boolean shutdownFS) {
331     JVMClusterUtil.RegionServerThread server =
332       hbaseCluster.getRegionServers().get(serverNumber);
333     LOG.info("Stopping " + server.toString());
334     server.getRegionServer().stop("Stopping rs " + serverNumber);
335     return server;
336   }
337 
338   /**
339    * Wait for the specified region server to stop. Removes this thread from list
340    * of running threads.
341    * @param serverNumber
342    * @return Name of region server that just went down.
343    */
344   public String waitOnRegionServer(final int serverNumber) {
345     return this.hbaseCluster.waitOnRegionServer(serverNumber);
346   }
347 
348 
349   /**
350    * Starts a master thread running
351    *
352    * @throws IOException
353    * @return New RegionServerThread
354    */
355   public JVMClusterUtil.MasterThread startMaster() throws IOException {
356     Configuration c = HBaseConfiguration.create(conf);
357     User user =
358         HBaseTestingUtility.getDifferentUser(c, ".hfs."+index++);
359 
360     JVMClusterUtil.MasterThread t = null;
361     try {
362       t = hbaseCluster.addMaster(c, hbaseCluster.getMasters().size(), user);
363       t.start();
364     } catch (InterruptedException ie) {
365       throw new IOException("Interrupted adding master to cluster", ie);
366     }
367     return t;
368   }
369 
370   @Override
371   public HMasterInterface getMasterAdmin() {
372     return this.hbaseCluster.getActiveMaster();
373   }
374 
375   /**
376    * Returns the current active master, if available.
377    * @return the active HMaster, null if none is active.
378    */
379   public HMaster getMaster() {
380     return this.hbaseCluster.getActiveMaster();
381   }
382 
383   /**
384    * Returns the master at the specified index, if available.
385    * @return the active HMaster, null if none is active.
386    */
387   public HMaster getMaster(final int serverNumber) {
388     return this.hbaseCluster.getMaster(serverNumber);
389   }
390 
391   /**
392    * Cause a master to exit without shutting down entire cluster.
393    * @param serverNumber  Used as index into a list.
394    */
395   public String abortMaster(int serverNumber) {
396     HMaster server = getMaster(serverNumber);
397     LOG.info("Aborting " + server.toString());
398     server.abort("Aborting for tests", new Exception("Trace info"));
399     return server.toString();
400   }
401 
402   /**
403    * Shut down the specified master cleanly
404    *
405    * @param serverNumber  Used as index into a list.
406    * @return the region server that was stopped
407    */
408   public JVMClusterUtil.MasterThread stopMaster(int serverNumber) {
409     return stopMaster(serverNumber, true);
410   }
411 
412   /**
413    * Shut down the specified master cleanly
414    *
415    * @param serverNumber  Used as index into a list.
416    * @param shutdownFS True is we are to shutdown the filesystem as part of this
417    * master's shutdown.  Usually we do but you do not want to do this if
418    * you are running multiple master in a test and you shut down one
419    * before end of the test.
420    * @return the master that was stopped
421    */
422   public JVMClusterUtil.MasterThread stopMaster(int serverNumber,
423       final boolean shutdownFS) {
424     JVMClusterUtil.MasterThread server =
425       hbaseCluster.getMasters().get(serverNumber);
426     LOG.info("Stopping " + server.toString());
427     server.getMaster().stop("Stopping master " + serverNumber);
428     return server;
429   }
430 
431   /**
432    * Wait for the specified master to stop. Removes this thread from list
433    * of running threads.
434    * @param serverNumber
435    * @return Name of master that just went down.
436    */
437   public String waitOnMaster(final int serverNumber) {
438     return this.hbaseCluster.waitOnMaster(serverNumber);
439   }
440 
441   /**
442    * Blocks until there is an active master and that master has completed
443    * initialization.
444    *
445    * @return true if an active master becomes available.  false if there are no
446    *         masters left.
447    * @throws InterruptedException
448    */
449   public boolean waitForActiveAndReadyMaster(long timeout) throws IOException {
450     List<JVMClusterUtil.MasterThread> mts;
451     long start = System.currentTimeMillis();
452     while (!(mts = getMasterThreads()).isEmpty()
453         && (System.currentTimeMillis() - start) < timeout) {
454       for (JVMClusterUtil.MasterThread mt : mts) {
455         if (mt.getMaster().isActiveMaster() && mt.getMaster().isInitialized()) {
456           return true;
457         }
458       }
459 
460       Threads.sleep(100);
461     }
462     return false;
463   }
464 
465   /**
466    * @return List of master threads.
467    */
468   public List<JVMClusterUtil.MasterThread> getMasterThreads() {
469     return this.hbaseCluster.getMasters();
470   }
471 
472   /**
473    * @return List of live master threads (skips the aborted and the killed)
474    */
475   public List<JVMClusterUtil.MasterThread> getLiveMasterThreads() {
476     return this.hbaseCluster.getLiveMasters();
477   }
478 
479   /**
480    * Wait for Mini HBase Cluster to shut down.
481    */
482   public void join() {
483     this.hbaseCluster.join();
484   }
485 
486   /**
487    * Shut down the mini HBase cluster
488    * @throws IOException
489    */
490   public void shutdown() throws IOException {
491     if (this.hbaseCluster != null) {
492       this.hbaseCluster.shutdown();
493     }
494     HConnectionManager.deleteAllConnections();
495   }
496 
497   @Override
498   public void close() throws IOException {
499   }
500 
501   @Override
502   public ClusterStatus getClusterStatus() throws IOException {
503     HMaster master = getMaster();
504     return master == null ? null : master.getClusterStatus();
505   }
506 
507   /**
508    * Call flushCache on all regions on all participating regionservers.
509    * @throws IOException
510    */
511   public void flushcache() throws IOException {
512     for (JVMClusterUtil.RegionServerThread t:
513         this.hbaseCluster.getRegionServers()) {
514       for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
515         r.flushcache();
516       }
517     }
518   }
519 
520   /**
521    * Call flushCache on all regions of the specified table.
522    * @throws IOException
523    */
524   public void flushcache(byte [] tableName) throws IOException {
525     for (JVMClusterUtil.RegionServerThread t:
526         this.hbaseCluster.getRegionServers()) {
527       for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
528         if(Bytes.equals(r.getTableDesc().getName(), tableName)) {
529           r.flushcache();
530         }
531       }
532     }
533   }
534 
535   /**
536    * Call flushCache on all regions on all participating regionservers.
537    * @throws IOException
538    */
539   public void compact(boolean major) throws IOException {
540     for (JVMClusterUtil.RegionServerThread t:
541         this.hbaseCluster.getRegionServers()) {
542       for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
543         r.compactStores(major);
544       }
545     }
546   }
547 
548   /**
549    * Call flushCache on all regions of the specified table.
550    * @throws IOException
551    */
552   public void compact(byte [] tableName, boolean major) throws IOException {
553     for (JVMClusterUtil.RegionServerThread t:
554         this.hbaseCluster.getRegionServers()) {
555       for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
556         if(Bytes.equals(r.getTableDesc().getName(), tableName)) {
557           r.compactStores(major);
558         }
559       }
560     }
561   }
562 
563   /**
564    * @return List of region server threads.
565    */
566   public List<JVMClusterUtil.RegionServerThread> getRegionServerThreads() {
567     return this.hbaseCluster.getRegionServers();
568   }
569 
570   /**
571    * @return List of live region server threads (skips the aborted and the killed)
572    */
573   public List<JVMClusterUtil.RegionServerThread> getLiveRegionServerThreads() {
574     return this.hbaseCluster.getLiveRegionServers();
575   }
576 
577   /**
578    * Grab a numbered region server of your choice.
579    * @param serverNumber
580    * @return region server
581    */
582   public HRegionServer getRegionServer(int serverNumber) {
583     return hbaseCluster.getRegionServer(serverNumber);
584   }
585 
586   public List<HRegion> getRegions(byte[] tableName) {
587     List<HRegion> ret = new ArrayList<HRegion>();
588     for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
589       HRegionServer hrs = rst.getRegionServer();
590       for (HRegion region : hrs.getOnlineRegionsLocalContext()) {
591         if (Bytes.equals(region.getTableDesc().getName(), tableName)) {
592           ret.add(region);
593         }
594       }
595     }
596     return ret;
597   }
598 
599   /**
600    * @return Index into List of {@link MiniHBaseCluster#getRegionServerThreads()}
601    * of HRS carrying regionName. Returns -1 if none found.
602    */
603   public int getServerWithMeta() {
604     return getServerWith(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
605   }
606 
607   /**
608    * Get the location of the specified region
609    * @param regionName Name of the region in bytes
610    * @return Index into List of {@link MiniHBaseCluster#getRegionServerThreads()}
611    * of HRS carrying .META.. Returns -1 if none found.
612    */
613   public int getServerWith(byte[] regionName) {
614     int index = -1;
615     int count = 0;
616     for (JVMClusterUtil.RegionServerThread rst: getRegionServerThreads()) {
617       HRegionServer hrs = rst.getRegionServer();
618       HRegion metaRegion =
619         hrs.getOnlineRegion(regionName);
620       if (metaRegion != null) {
621         index = count;
622         break;
623       }
624       count++;
625     }
626     return index;
627   }
628 
629   @Override
630   public ServerName getServerHoldingRegion(byte[] regionName) throws IOException {
631     int index = getServerWith(regionName);
632     if (index < 0) {
633       return null;
634     }
635     return getRegionServer(index).getServerName();
636   }
637 
638   /**
639    * Counts the total numbers of regions being served by the currently online
640    * region servers by asking each how many regions they have.  Does not look
641    * at META at all.  Count includes catalog tables.
642    * @return number of regions being served by all region servers
643    */
644   public long countServedRegions() {
645     long count = 0;
646     for (JVMClusterUtil.RegionServerThread rst : getLiveRegionServerThreads()) {
647       count += rst.getRegionServer().getNumberOfOnlineRegions();
648     }
649     return count;
650   }
651 
652   @Override
653   public void waitUntilShutDown() {
654     this.hbaseCluster.join();
655   }
656 
657   protected int getRegionServerIndex(ServerName serverName) {
658     //we have a small number of region servers, this should be fine for now.
659     List<RegionServerThread> servers = getRegionServerThreads();
660     for (int i=0; i < servers.size(); i++) {
661       if (servers.get(i).getRegionServer().getServerName().equals(serverName)) {
662         return i;
663       }
664     }
665     return -1;
666   }
667 
668   protected int getMasterIndex(ServerName serverName) {
669     List<MasterThread> masters = getMasterThreads();
670     for (int i = 0; i < masters.size(); i++) {
671       if (masters.get(i).getMaster().getServerName().equals(serverName)) {
672         return i;
673       }
674     }
675     return -1;
676   }
677 }