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