1   /**
2    * Copyright 2009 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 static org.junit.Assert.assertTrue;
23  
24  import java.io.File;
25  import java.io.IOException;
26  import java.io.OutputStream;
27  import java.lang.reflect.Field;
28  import java.net.InetAddress;
29  import java.net.ServerSocket;
30  import java.net.Socket;
31  import java.net.UnknownHostException;
32  import java.security.MessageDigest;
33  import java.util.ArrayList;
34  import java.util.Arrays;
35  import java.util.Collection;
36  import java.util.Collections;
37  import java.util.List;
38  import java.util.Map;
39  import java.util.NavigableSet;
40  import java.util.Random;
41  import java.util.UUID;
42  
43  import org.apache.commons.logging.Log;
44  import org.apache.commons.logging.LogFactory;
45  import org.apache.commons.logging.impl.Jdk14Logger;
46  import org.apache.commons.logging.impl.Log4JLogger;
47  import org.apache.hadoop.conf.Configuration;
48  import org.apache.hadoop.fs.FileSystem;
49  import org.apache.hadoop.fs.Path;
50  import org.apache.hadoop.hbase.client.Delete;
51  import org.apache.hadoop.hbase.client.Get;
52  import org.apache.hadoop.hbase.client.HBaseAdmin;
53  import org.apache.hadoop.hbase.client.HConnection;
54  import org.apache.hadoop.hbase.client.HTable;
55  import org.apache.hadoop.hbase.client.Put;
56  import org.apache.hadoop.hbase.client.Result;
57  import org.apache.hadoop.hbase.client.ResultScanner;
58  import org.apache.hadoop.hbase.client.Scan;
59  import org.apache.hadoop.hbase.fs.HFileSystem;
60  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
61  import org.apache.hadoop.hbase.io.hfile.ChecksumUtil;
62  import org.apache.hadoop.hbase.io.hfile.Compression;
63  import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
64  import org.apache.hadoop.hbase.io.hfile.HFile;
65  import org.apache.hadoop.hbase.mapreduce.MapreduceTestingShim;
66  import org.apache.hadoop.hbase.master.HMaster;
67  import org.apache.hadoop.hbase.master.ServerManager;
68  import org.apache.hadoop.hbase.regionserver.HRegion;
69  import org.apache.hadoop.hbase.regionserver.HRegionServer;
70  import org.apache.hadoop.hbase.regionserver.InternalScanner;
71  import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
72  import org.apache.hadoop.hbase.regionserver.Store;
73  import org.apache.hadoop.hbase.regionserver.StoreFile;
74  import org.apache.hadoop.hbase.security.User;
75  import org.apache.hadoop.hbase.util.Bytes;
76  import org.apache.hadoop.hbase.util.FSUtils;
77  import org.apache.hadoop.hbase.util.JVMClusterUtil;
78  import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
79  import org.apache.hadoop.hbase.util.RegionSplitter;
80  import org.apache.hadoop.hbase.util.Threads;
81  import org.apache.hadoop.hbase.util.Writables;
82  import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
83  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
84  import org.apache.hadoop.hbase.zookeeper.ZKConfig;
85  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
86  import org.apache.hadoop.hdfs.DFSClient;
87  import org.apache.hadoop.hdfs.DistributedFileSystem;
88  import org.apache.hadoop.hdfs.MiniDFSCluster;
89  import org.apache.hadoop.mapred.JobConf;
90  import org.apache.hadoop.mapred.MiniMRCluster;
91  import org.apache.zookeeper.KeeperException;
92  import org.apache.zookeeper.KeeperException.NodeExistsException;
93  import org.apache.zookeeper.WatchedEvent;
94  import org.apache.zookeeper.ZooKeeper;
95  
96  /**
97   * Facility for testing HBase. Replacement for
98   * old HBaseTestCase and HBaseClusterTestCase functionality.
99   * Create an instance and keep it around testing HBase.  This class is
100  * meant to be your one-stop shop for anything you might need testing.  Manages
101  * one cluster at a time only. Managed cluster can be an in-process
102  * {@link MiniHBaseCluster}, or a deployed cluster of type {@link DistributedHBaseCluster}.
103  * Not all methods work with the real cluster.
104  * Depends on log4j being on classpath and
105  * hbase-site.xml for logging and test-run configuration.  It does not set
106  * logging levels nor make changes to configuration parameters.
107  */
108 public class HBaseTestingUtility {
109   private static final Log LOG = LogFactory.getLog(HBaseTestingUtility.class);
110   private Configuration conf;
111   private MiniZooKeeperCluster zkCluster = null;
112 
113   /**
114    * The default number of regions per regionserver when creating a pre-split
115    * table.
116    */
117   private static int DEFAULT_REGIONS_PER_SERVER = 5;
118 
119   /**
120    * Set if we were passed a zkCluster.  If so, we won't shutdown zk as
121    * part of general shutdown.
122    */
123   private boolean passedZkCluster = false;
124   private MiniDFSCluster dfsCluster = null;
125 
126   private HBaseCluster hbaseCluster = null;
127   private MiniMRCluster mrCluster = null;
128 
129   // Directory where we put the data for this instance of HBaseTestingUtility
130   private File dataTestDir = null;
131 
132   // Directory (usually a subdirectory of dataTestDir) used by the dfs cluster
133   //  if any
134   private File clusterTestDir = null;
135 
136   /**
137    * System property key to get test directory value.
138    * Name is as it is because mini dfs has hard-codings to put test data here.
139    * It should NOT be used directly in HBase, as it's a property used in
140    *  mini dfs.
141    *  @deprecated can be used only with mini dfs
142    */
143   private static final String TEST_DIRECTORY_KEY = "test.build.data";
144 
145   /**
146    * System property key to get base test directory value
147    */
148   public static final String BASE_TEST_DIRECTORY_KEY =
149     "test.build.data.basedirectory";
150 
151   /**
152    * Default base directory for test output.
153    */
154   public static final String DEFAULT_BASE_TEST_DIRECTORY = "target/test-data";
155 
156   /** Compression algorithms to use in parameterized JUnit 4 tests */
157   public static final List<Object[]> COMPRESSION_ALGORITHMS_PARAMETERIZED =
158     Arrays.asList(new Object[][] {
159       { Compression.Algorithm.NONE },
160       { Compression.Algorithm.GZ }
161     });
162 
163   /** This is for unit tests parameterized with a single boolean. */
164   public static final List<Object[]> BOOLEAN_PARAMETERIZED =
165       Arrays.asList(new Object[][] {
166           { new Boolean(false) },
167           { new Boolean(true) }
168       });
169 
170   /** Compression algorithms to use in testing */
171   public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS ={
172       Compression.Algorithm.NONE, Compression.Algorithm.GZ
173     };
174 
175   /**
176    * Create all combinations of Bloom filters and compression algorithms for
177    * testing.
178    */
179   private static List<Object[]> bloomAndCompressionCombinations() {
180     List<Object[]> configurations = new ArrayList<Object[]>();
181     for (Compression.Algorithm comprAlgo :
182          HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
183       for (StoreFile.BloomType bloomType : StoreFile.BloomType.values()) {
184         configurations.add(new Object[] { comprAlgo, bloomType });
185       }
186     }
187     return Collections.unmodifiableList(configurations);
188   }
189 
190   public static final Collection<Object[]> BLOOM_AND_COMPRESSION_COMBINATIONS =
191       bloomAndCompressionCombinations();
192 
193   public HBaseTestingUtility() {
194     this(HBaseConfiguration.create());
195   }
196 
197   public HBaseTestingUtility(Configuration conf) {
198     this.conf = conf;
199 
200     // a hbase checksum verification failure will cause unit tests to fail
201     ChecksumUtil.generateExceptionForChecksumFailureForTest(true);
202     setHDFSClientRetryProperty();
203   }
204 
205   private void setHDFSClientRetryProperty() {
206     this.conf.setInt("hdfs.client.retries.number", 1);
207     HBaseFileSystem.setRetryCounts(conf);
208   }
209 
210   /**
211    * Returns this classes's instance of {@link Configuration}.  Be careful how
212    * you use the returned Configuration since {@link HConnection} instances
213    * can be shared.  The Map of HConnections is keyed by the Configuration.  If
214    * say, a Connection was being used against a cluster that had been shutdown,
215    * see {@link #shutdownMiniCluster()}, then the Connection will no longer
216    * be wholesome.  Rather than use the return direct, its usually best to
217    * make a copy and use that.  Do
218    * <code>Configuration c = new Configuration(INSTANCE.getConfiguration());</code>
219    * @return Instance of Configuration.
220    */
221   public Configuration getConfiguration() {
222     return this.conf;
223   }
224 
225   public void setHBaseCluster(HBaseCluster hbaseCluster) {
226     this.hbaseCluster = hbaseCluster;
227   }
228 
229   /**
230    * @return Where to write test data on local filesystem; usually
231    * {@link #DEFAULT_BASE_TEST_DIRECTORY}
232    * Should not be used by the unit tests, hence its's private.
233    * Unit test will use a subdirectory of this directory.
234    * @see #setupDataTestDir()
235    * @see #getTestFileSystem()
236    */
237   private Path getBaseTestDir() {
238     String PathName = System.getProperty(
239       BASE_TEST_DIRECTORY_KEY, DEFAULT_BASE_TEST_DIRECTORY);
240 
241     return new Path(PathName);
242   }
243 
244   /**
245    * @return Where to write test data on local filesystem, specific to
246    *  the test.  Useful for tests that do not use a cluster.
247    * Creates it if it does not exist already.
248    * @see #getTestFileSystem()
249    */
250   public Path getDataTestDir() {
251     if (dataTestDir == null){
252       setupDataTestDir();
253     }
254     return new Path(dataTestDir.getAbsolutePath());
255   }
256 
257   /**
258    * @return Where the DFS cluster will write data on the local subsystem.
259    * Creates it if it does not exist already.
260    * @see #getTestFileSystem()
261    */
262   public Path getClusterTestDir() {
263     if (clusterTestDir == null){
264       setupClusterTestDir();
265     }
266     return new Path(clusterTestDir.getAbsolutePath());
267   }
268 
269   /**
270    * @param subdirName
271    * @return Path to a subdirectory named <code>subdirName</code> under
272    * {@link #getDataTestDir()}.
273    * Does *NOT* create it if it does not exist.
274    */
275   public Path getDataTestDir(final String subdirName) {
276     return new Path(getDataTestDir(), subdirName);
277   }
278 
279   /**
280    * Home our data in a dir under {@link #DEFAULT_BASE_TEST_DIRECTORY}.
281    * Give it a random name so can have many concurrent tests running if
282    * we need to.  It needs to amend the {@link #TEST_DIRECTORY_KEY}
283    * System property, as it's what minidfscluster bases
284    * it data dir on.  Moding a System property is not the way to do concurrent
285    * instances -- another instance could grab the temporary
286    * value unintentionally -- but not anything can do about it at moment;
287    * single instance only is how the minidfscluster works.
288    *
289    * We also create the underlying directory for
290    *  hadoop.log.dir, mapred.local.dir and hadoop.tmp.dir, and set the values
291    *  in the conf, and as a system property for hadoop.tmp.dir
292    *
293    * @return The calculated data test build directory.
294    */
295   private void setupDataTestDir() {
296     if (dataTestDir != null) {
297       LOG.warn("Data test dir already setup in " +
298         dataTestDir.getAbsolutePath());
299       return;
300     }
301 
302     String randomStr = UUID.randomUUID().toString();
303     Path testPath= new Path(getBaseTestDir(), randomStr);
304 
305     dataTestDir = new File(testPath.toString()).getAbsoluteFile();
306     dataTestDir.deleteOnExit();
307 
308     createSubDirAndSystemProperty(
309       "hadoop.log.dir",
310       testPath, "hadoop-log-dir");
311 
312     // This is defaulted in core-default.xml to /tmp/hadoop-${user.name}, but
313     //  we want our own value to ensure uniqueness on the same machine
314     createSubDirAndSystemProperty(
315       "hadoop.tmp.dir",
316       testPath, "hadoop-tmp-dir");
317 
318     // Read and modified in org.apache.hadoop.mapred.MiniMRCluster
319     createSubDir(
320       "mapred.local.dir",
321       testPath, "mapred-local-dir");
322 
323     createSubDirAndSystemProperty(
324       "mapred.working.dir",
325       testPath, "mapred-working-dir");
326 
327     createSubDir(
328       "hbase.local.dir",
329       testPath, "hbase-local-dir");
330   }
331 
332   private void createSubDir(String propertyName, Path parent, String subDirName){
333     Path newPath= new Path(parent, subDirName);
334     File newDir = new File(newPath.toString()).getAbsoluteFile();
335     newDir.deleteOnExit();
336     conf.set(propertyName, newDir.getAbsolutePath());
337   }
338 
339   private void createSubDirAndSystemProperty(
340     String propertyName, Path parent, String subDirName){
341 
342     String sysValue = System.getProperty(propertyName);
343 
344     if (sysValue != null) {
345       // There is already a value set. So we do nothing but hope
346       //  that there will be no conflicts
347       LOG.info("System.getProperty(\""+propertyName+"\") already set to: "+
348         sysValue + " so I do NOT create it in "+dataTestDir.getAbsolutePath());
349       String confValue = conf.get(propertyName);
350       if (confValue != null && !confValue.endsWith(sysValue)){
351        LOG.warn(
352          propertyName + " property value differs in configuration and system: "+
353          "Configuration="+confValue+" while System="+sysValue+
354          " Erasing configuration value by system value."
355        );
356       }
357       conf.set(propertyName, sysValue);
358     } else {
359       // Ok, it's not set, so we create it as a subdirectory
360       createSubDir(propertyName, parent, subDirName);
361       System.setProperty(propertyName, conf.get(propertyName));
362     }
363   }
364 
365   /**
366    * Creates a directory for the DFS cluster, under the test data
367    */
368   private void setupClusterTestDir() {
369     if (clusterTestDir != null) {
370       LOG.warn("Cluster test dir already setup in " +
371         clusterTestDir.getAbsolutePath());
372       return;
373     }
374 
375     // Using randomUUID ensures that multiple clusters can be launched by
376     //  a same test, if it stops & starts them
377     Path testDir = getDataTestDir("dfscluster_" + UUID.randomUUID().toString());
378     clusterTestDir = new File(testDir.toString()).getAbsoluteFile();
379     // Have it cleaned up on exit
380     clusterTestDir.deleteOnExit();
381   }
382 
383   /**
384    * @throws IOException If a cluster -- zk, dfs, or hbase -- already running.
385    */
386   public void isRunningCluster() throws IOException {
387     if (dfsCluster == null) return;
388     throw new IOException("Cluster already running at " +
389       this.clusterTestDir);
390   }
391 
392   /**
393    * Start a minidfscluster.
394    * @param servers How many DNs to start.
395    * @throws Exception
396    * @see {@link #shutdownMiniDFSCluster()}
397    * @return The mini dfs cluster created.
398    */
399   public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception {
400     return startMiniDFSCluster(servers, null);
401   }
402 
403   /**
404    * Start a minidfscluster.
405    * This is useful if you want to run datanode on distinct hosts for things
406    * like HDFS block location verification.
407    * If you start MiniDFSCluster without host names, all instances of the
408    * datanodes will have the same host name.
409    * @param hosts hostnames DNs to run on.
410    * @throws Exception
411    * @see {@link #shutdownMiniDFSCluster()}
412    * @return The mini dfs cluster created.
413    */
414   public MiniDFSCluster startMiniDFSCluster(final String hosts[])
415     throws Exception {
416     if ( hosts != null && hosts.length != 0) {
417       return startMiniDFSCluster(hosts.length, hosts);
418     } else {
419       return startMiniDFSCluster(1, null);
420     }
421   }
422 
423   /**
424    * Start a minidfscluster.
425    * Can only create one.
426    * @param servers How many DNs to start.
427    * @param hosts hostnames DNs to run on.
428    * @throws Exception
429    * @see {@link #shutdownMiniDFSCluster()}
430    * @return The mini dfs cluster created.
431    */
432   public MiniDFSCluster startMiniDFSCluster(int servers, final String hosts[])
433   throws Exception {
434 
435     // Check that there is not already a cluster running
436     isRunningCluster();
437 
438     // Initialize the local directory used by the MiniDFS
439     if (clusterTestDir == null) {
440       setupClusterTestDir();
441     }
442 
443     // We have to set this property as it is used by MiniCluster
444     System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestDir.toString());
445 
446     // Some tests also do this:
447     //  System.getProperty("test.cache.data", "build/test/cache");
448     // It's also deprecated
449     System.setProperty("test.cache.data", this.clusterTestDir.toString());
450 
451     // Ok, now we can start
452     this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
453       true, null, null, hosts, null);
454 
455     // Set this just-started cluster as our filesystem.
456     FileSystem fs = this.dfsCluster.getFileSystem();
457     this.conf.set("fs.defaultFS", fs.getUri().toString());
458     // Do old style too just to be safe.
459     this.conf.set("fs.default.name", fs.getUri().toString());
460 
461     // Wait for the cluster to be totally up
462     this.dfsCluster.waitClusterUp();
463 
464     return this.dfsCluster;
465   }
466 
467   /**
468    * Shuts down instance created by call to {@link #startMiniDFSCluster(int)}
469    * or does nothing.
470    * @throws Exception
471    */
472   public void shutdownMiniDFSCluster() throws Exception {
473     if (this.dfsCluster != null) {
474       // The below throws an exception per dn, AsynchronousCloseException.
475       this.dfsCluster.shutdown();
476       dfsCluster = null;
477     }
478 
479   }
480 
481   /**
482    * Call this if you only want a zk cluster.
483    * @see #startMiniZKCluster() if you want zk + dfs + hbase mini cluster.
484    * @throws Exception
485    * @see #shutdownMiniZKCluster()
486    * @return zk cluster started.
487    */
488   public MiniZooKeeperCluster startMiniZKCluster() throws Exception {
489     return startMiniZKCluster(1);
490   }
491 
492   /**
493    * Call this if you only want a zk cluster.
494    * @param zooKeeperServerNum
495    * @see #startMiniZKCluster() if you want zk + dfs + hbase mini cluster.
496    * @throws Exception
497    * @see #shutdownMiniZKCluster()
498    * @return zk cluster started.
499    */
500   public MiniZooKeeperCluster startMiniZKCluster(int zooKeeperServerNum)
501       throws Exception {
502     File zkClusterFile = new File(getClusterTestDir().toString());
503     return startMiniZKCluster(zkClusterFile, zooKeeperServerNum);
504   }
505 
506   private MiniZooKeeperCluster startMiniZKCluster(final File dir)
507     throws Exception {
508     return startMiniZKCluster(dir,1);
509   }
510 
511   private MiniZooKeeperCluster startMiniZKCluster(final File dir,
512       int zooKeeperServerNum)
513   throws Exception {
514     if (this.zkCluster != null) {
515       throw new IOException("Cluster already running at " + dir);
516     }
517     this.passedZkCluster = false;
518     this.zkCluster = new MiniZooKeeperCluster(this.getConfiguration());
519     int clientPort =   this.zkCluster.startup(dir,zooKeeperServerNum);
520     this.conf.set(HConstants.ZOOKEEPER_CLIENT_PORT,
521       Integer.toString(clientPort));
522     return this.zkCluster;
523   }
524 
525   /**
526    * Shuts down zk cluster created by call to {@link #startMiniZKCluster(File)}
527    * or does nothing.
528    * @throws IOException
529    * @see #startMiniZKCluster()
530    */
531   public void shutdownMiniZKCluster() throws IOException {
532     if (this.zkCluster != null) {
533       this.zkCluster.shutdown();
534       this.zkCluster = null;
535     }
536   }
537 
538   /**
539    * Start up a minicluster of hbase, dfs, and zookeeper.
540    * @throws Exception
541    * @return Mini hbase cluster instance created.
542    * @see {@link #shutdownMiniDFSCluster()}
543    */
544   public MiniHBaseCluster startMiniCluster() throws Exception {
545     return startMiniCluster(1, 1);
546   }
547 
548   /**
549    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
550    * Modifies Configuration.  Homes the cluster data directory under a random
551    * subdirectory in a directory under System property test.build.data.
552    * Directory is cleaned up on exit.
553    * @param numSlaves Number of slaves to start up.  We'll start this many
554    * datanodes and regionservers.  If numSlaves is > 1, then make sure
555    * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
556    * bind errors.
557    * @throws Exception
558    * @see {@link #shutdownMiniCluster()}
559    * @return Mini hbase cluster instance created.
560    */
561   public MiniHBaseCluster startMiniCluster(final int numSlaves)
562   throws Exception {
563     return startMiniCluster(1, numSlaves);
564   }
565 
566 
567   /**
568    * start minicluster
569    * @throws Exception
570    * @see {@link #shutdownMiniCluster()}
571    * @return Mini hbase cluster instance created.
572    */
573   public MiniHBaseCluster startMiniCluster(final int numMasters,
574     final int numSlaves)
575   throws Exception {
576     return startMiniCluster(numMasters, numSlaves, null);
577   }
578 
579 
580   /**
581    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
582    * Modifies Configuration.  Homes the cluster data directory under a random
583    * subdirectory in a directory under System property test.build.data.
584    * Directory is cleaned up on exit.
585    * @param numMasters Number of masters to start up.  We'll start this many
586    * hbase masters.  If numMasters > 1, you can find the active/primary master
587    * with {@link MiniHBaseCluster#getMaster()}.
588    * @param numSlaves Number of slaves to start up.  We'll start this many
589    * regionservers. If dataNodeHosts == null, this also indicates the number of
590    * datanodes to start. If dataNodeHosts != null, the number of datanodes is
591    * based on dataNodeHosts.length.
592    * If numSlaves is > 1, then make sure
593    * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
594    * bind errors.
595    * @param dataNodeHosts hostnames DNs to run on.
596    * This is useful if you want to run datanode on distinct hosts for things
597    * like HDFS block location verification.
598    * If you start MiniDFSCluster without host names,
599    * all instances of the datanodes will have the same host name.
600    * @throws Exception
601    * @see {@link #shutdownMiniCluster()}
602    * @return Mini hbase cluster instance created.
603    */
604   public MiniHBaseCluster startMiniCluster(final int numMasters,
605     final int numSlaves, final String[] dataNodeHosts)
606   throws Exception {
607     int numDataNodes = numSlaves;
608     if ( dataNodeHosts != null && dataNodeHosts.length != 0) {
609       numDataNodes = dataNodeHosts.length;
610     }
611 
612     LOG.info("Starting up minicluster with " + numMasters + " master(s) and " +
613         numSlaves + " regionserver(s) and " + numDataNodes + " datanode(s)");
614 
615     // If we already put up a cluster, fail.
616     isRunningCluster();
617 
618     // Bring up mini dfs cluster. This spews a bunch of warnings about missing
619     // scheme. Complaints are 'Scheme is undefined for build/test/data/dfs/name1'.
620     startMiniDFSCluster(numDataNodes, dataNodeHosts);
621 
622     // Start up a zk cluster.
623     if (this.zkCluster == null) {
624       startMiniZKCluster(clusterTestDir);
625     }
626 
627     // Start the MiniHBaseCluster
628     return startMiniHBaseCluster(numMasters, numSlaves);
629   }
630 
631   /**
632    * Starts up mini hbase cluster.  Usually used after call to
633    * {@link #startMiniCluster(int, int)} when doing stepped startup of clusters.
634    * Usually you won't want this.  You'll usually want {@link #startMiniCluster()}.
635    * @param numMasters
636    * @param numSlaves
637    * @return Reference to the hbase mini hbase cluster.
638    * @throws IOException
639    * @throws InterruptedException
640    * @see {@link #startMiniCluster()}
641    */
642   public MiniHBaseCluster startMiniHBaseCluster(final int numMasters,
643       final int numSlaves)
644   throws IOException, InterruptedException {
645     // Now do the mini hbase cluster.  Set the hbase.rootdir in config.
646     createRootDir();
647 
648     // These settings will make the server waits until this exact number of
649     // regions servers are connected.
650     if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1) == -1) {
651       conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, numSlaves);
652     }
653     if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1) == -1) {
654       conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, numSlaves);
655     }
656 
657     Configuration c = new Configuration(this.conf);
658     this.hbaseCluster = new MiniHBaseCluster(c, numMasters, numSlaves);
659     // Don't leave here till we've done a successful scan of the .META.
660     HTable t = new HTable(c, HConstants.META_TABLE_NAME);
661     ResultScanner s = t.getScanner(new Scan());
662     while (s.next() != null) {
663       continue;
664     }
665     s.close();
666     t.close();
667 
668     getHBaseAdmin(); // create immediately the hbaseAdmin
669     LOG.info("Minicluster is up");
670     return (MiniHBaseCluster)this.hbaseCluster;
671   }
672 
673   /**
674    * Starts the hbase cluster up again after shutting it down previously in a
675    * test.  Use this if you want to keep dfs/zk up and just stop/start hbase.
676    * @param servers number of region servers
677    * @throws IOException
678    */
679   public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
680     this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
681     // Don't leave here till we've done a successful scan of the .META.
682     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
683     ResultScanner s = t.getScanner(new Scan());
684     while (s.next() != null) {
685       // do nothing
686     }
687     LOG.info("HBase has been restarted");
688     s.close();
689     t.close();
690   }
691 
692   /**
693    * @return Current mini hbase cluster. Only has something in it after a call
694    * to {@link #startMiniCluster()}.
695    * @see #startMiniCluster()
696    */
697   public MiniHBaseCluster getMiniHBaseCluster() {
698     if (this.hbaseCluster instanceof MiniHBaseCluster) {
699       return (MiniHBaseCluster)this.hbaseCluster;
700     }
701     throw new RuntimeException(hbaseCluster + " not an instance of " +
702                                MiniHBaseCluster.class.getName());
703   }
704 
705   /**
706    * Stops mini hbase, zk, and hdfs clusters.
707    * @throws IOException
708    * @see {@link #startMiniCluster(int)}
709    */
710   public void shutdownMiniCluster() throws Exception {
711     LOG.info("Shutting down minicluster");
712     shutdownMiniHBaseCluster();
713     if (!this.passedZkCluster){
714       shutdownMiniZKCluster();
715     }
716     shutdownMiniDFSCluster();
717 
718     // Clean up our directory.
719     if (this.clusterTestDir != null && this.clusterTestDir.exists()) {
720       // Need to use deleteDirectory because File.delete required dir is empty.
721       if (!FSUtils.deleteDirectory(FileSystem.getLocal(this.conf),
722           new Path(this.clusterTestDir.toString()))) {
723         LOG.warn("Failed delete of " + this.clusterTestDir.toString());
724       }
725       this.clusterTestDir = null;
726     }
727     LOG.info("Minicluster is down");
728   }
729 
730   /**
731    * Shutdown HBase mini cluster.  Does not shutdown zk or dfs if running.
732    * @throws IOException
733    */
734   public void shutdownMiniHBaseCluster() throws IOException {
735     if (hbaseAdmin != null) {
736       hbaseAdmin.close();
737       hbaseAdmin = null;
738     }
739     // unset the configuration for MIN and MAX RS to start
740     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
741     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1);
742     if (this.hbaseCluster != null) {
743       this.hbaseCluster.shutdown();
744       // Wait till hbase is down before going on to shutdown zk.
745       this.hbaseCluster.waitUntilShutDown();
746       this.hbaseCluster = null;
747     }
748   }
749 
750   /**
751    * Returns the path to the default root dir the minicluster uses.
752    * Note: this does not cause the root dir to be created.
753    * @return Fully qualified path for the default hbase root dir
754    * @throws IOException
755    */
756   public Path getDefaultRootDirPath() throws IOException {
757 	FileSystem fs = FileSystem.get(this.conf);
758 	return new Path(fs.makeQualified(fs.getHomeDirectory()),"hbase");
759   }
760 
761   /**
762    * Creates an hbase rootdir in user home directory.  Also creates hbase
763    * version file.  Normally you won't make use of this method.  Root hbasedir
764    * is created for you as part of mini cluster startup.  You'd only use this
765    * method if you were doing manual operation.
766    * @return Fully qualified path to hbase root dir
767    * @throws IOException
768    */
769   public Path createRootDir() throws IOException {
770     FileSystem fs = FileSystem.get(this.conf);
771     Path hbaseRootdir = getDefaultRootDirPath();
772     this.conf.set(HConstants.HBASE_DIR, hbaseRootdir.toString());
773     fs.mkdirs(hbaseRootdir);
774     FSUtils.setVersion(fs, hbaseRootdir);
775     return hbaseRootdir;
776   }
777 
778   /**
779    * Flushes all caches in the mini hbase cluster
780    * @throws IOException
781    */
782   public void flush() throws IOException {
783     getMiniHBaseCluster().flushcache();
784   }
785 
786   /**
787    * Flushes all caches in the mini hbase cluster
788    * @throws IOException
789    */
790   public void flush(byte [] tableName) throws IOException {
791     getMiniHBaseCluster().flushcache(tableName);
792   }
793 
794   /**
795    * Compact all regions in the mini hbase cluster
796    * @throws IOException
797    */
798   public void compact(boolean major) throws IOException {
799     getMiniHBaseCluster().compact(major);
800   }
801 
802   /**
803    * Compact all of a table's reagion in the mini hbase cluster
804    * @throws IOException
805    */
806   public void compact(byte [] tableName, boolean major) throws IOException {
807     getMiniHBaseCluster().compact(tableName, major);
808   }
809 
810 
811   /**
812    * Create a table.
813    * @param tableName
814    * @param family
815    * @return An HTable instance for the created table.
816    * @throws IOException
817    */
818   public HTable createTable(byte[] tableName, byte[] family)
819   throws IOException{
820     return createTable(tableName, new byte[][]{family});
821   }
822 
823   /**
824    * Create a table.
825    * @param tableName
826    * @param families
827    * @return An HTable instance for the created table.
828    * @throws IOException
829    */
830   public HTable createTable(byte[] tableName, byte[][] families)
831   throws IOException {
832     return createTable(tableName, families,
833         new Configuration(getConfiguration()));
834   }
835 
836   public HTable createTable(byte[] tableName, byte[][] families,
837       int numVersions, byte[] startKey, byte[] endKey, int numRegions)
838   throws IOException{
839     HTableDescriptor desc = new HTableDescriptor(tableName);
840     for (byte[] family : families) {
841       HColumnDescriptor hcd = new HColumnDescriptor(family)
842           .setMaxVersions(numVersions);
843       desc.addFamily(hcd);
844     }
845     getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
846     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
847     waitUntilAllRegionsAssigned(tableName);
848     return new HTable(getConfiguration(), tableName);
849   }
850 
851   /**
852    * Create a table.
853    * @param tableName
854    * @param families
855    * @param c Configuration to use
856    * @return An HTable instance for the created table.
857    * @throws IOException
858    */
859   public HTable createTable(byte[] tableName, byte[][] families,
860       final Configuration c)
861   throws IOException {
862     HTableDescriptor desc = new HTableDescriptor(tableName);
863     for(byte[] family : families) {
864       desc.addFamily(new HColumnDescriptor(family));
865     }
866     getHBaseAdmin().createTable(desc);
867     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
868     waitUntilAllRegionsAssigned(tableName);
869     return new HTable(c, tableName);
870   }
871 
872   /**
873    * Create a table.
874    * @param tableName
875    * @param families
876    * @param c Configuration to use
877    * @param numVersions
878    * @return An HTable instance for the created table.
879    * @throws IOException
880    */
881   public HTable createTable(byte[] tableName, byte[][] families,
882       final Configuration c, int numVersions)
883   throws IOException {
884     HTableDescriptor desc = new HTableDescriptor(tableName);
885     for(byte[] family : families) {
886       HColumnDescriptor hcd = new HColumnDescriptor(family)
887           .setMaxVersions(numVersions);
888       desc.addFamily(hcd);
889     }
890     getHBaseAdmin().createTable(desc);
891     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
892     waitUntilAllRegionsAssigned(tableName);
893     return new HTable(c, tableName);
894   }
895 
896   /**
897    * Create a table.
898    * @param tableName
899    * @param family
900    * @param numVersions
901    * @return An HTable instance for the created table.
902    * @throws IOException
903    */
904   public HTable createTable(byte[] tableName, byte[] family, int numVersions)
905   throws IOException {
906     return createTable(tableName, new byte[][]{family}, numVersions);
907   }
908 
909   /**
910    * Create a table.
911    * @param tableName
912    * @param families
913    * @param numVersions
914    * @return An HTable instance for the created table.
915    * @throws IOException
916    */
917   public HTable createTable(byte[] tableName, byte[][] families,
918       int numVersions)
919   throws IOException {
920     HTableDescriptor desc = new HTableDescriptor(tableName);
921     for (byte[] family : families) {
922       HColumnDescriptor hcd = new HColumnDescriptor(family)
923           .setMaxVersions(numVersions);
924       desc.addFamily(hcd);
925     }
926     getHBaseAdmin().createTable(desc);
927     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
928     waitUntilAllRegionsAssigned(tableName);
929     return new HTable(new Configuration(getConfiguration()), tableName);
930   }
931 
932   /**
933    * Create a table.
934    * @param tableName
935    * @param families
936    * @param numVersions
937    * @return An HTable instance for the created table.
938    * @throws IOException
939    */
940   public HTable createTable(byte[] tableName, byte[][] families,
941     int numVersions, int blockSize) throws IOException {
942     HTableDescriptor desc = new HTableDescriptor(tableName);
943     for (byte[] family : families) {
944       HColumnDescriptor hcd = new HColumnDescriptor(family)
945           .setMaxVersions(numVersions)
946           .setBlocksize(blockSize);
947       desc.addFamily(hcd);
948     }
949     getHBaseAdmin().createTable(desc);
950     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
951     waitUntilAllRegionsAssigned(tableName);
952     return new HTable(new Configuration(getConfiguration()), tableName);
953   }
954 
955   /**
956    * Create a table.
957    * @param tableName
958    * @param families
959    * @param numVersions
960    * @return An HTable instance for the created table.
961    * @throws IOException
962    */
963   public HTable createTable(byte[] tableName, byte[][] families,
964       int[] numVersions)
965   throws IOException {
966     HTableDescriptor desc = new HTableDescriptor(tableName);
967     int i = 0;
968     for (byte[] family : families) {
969       HColumnDescriptor hcd = new HColumnDescriptor(family)
970           .setMaxVersions(numVersions[i]);
971       desc.addFamily(hcd);
972       i++;
973     }
974     getHBaseAdmin().createTable(desc);
975     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
976     waitUntilAllRegionsAssigned(tableName);
977     return new HTable(new Configuration(getConfiguration()), tableName);
978   }
979 
980   /**
981    * Drop an existing table
982    * @param tableName existing table
983    */
984   public void deleteTable(byte[] tableName) throws IOException {
985     try {
986       getHBaseAdmin().disableTable(tableName);
987     } catch (TableNotEnabledException e) {
988       LOG.debug("Table: " + Bytes.toString(tableName) + " already disabled, so just deleting it.");
989     }
990     getHBaseAdmin().deleteTable(tableName);
991   }
992 
993   /**
994    * Provide an existing table name to truncate
995    * @param tableName existing table
996    * @return HTable to that new table
997    * @throws IOException
998    */
999   public HTable truncateTable(byte [] tableName) throws IOException {
1000     HTable table = new HTable(getConfiguration(), tableName);
1001     Scan scan = new Scan();
1002     ResultScanner resScan = table.getScanner(scan);
1003     for(Result res : resScan) {
1004       Delete del = new Delete(res.getRow());
1005       table.delete(del);
1006     }
1007     resScan = table.getScanner(scan);
1008     resScan.close();
1009     return table;
1010   }
1011 
1012   /**
1013    * Load table with rows from 'aaa' to 'zzz'.
1014    * @param t Table
1015    * @param f Family
1016    * @return Count of rows loaded.
1017    * @throws IOException
1018    */
1019   public int loadTable(final HTable t, final byte[] f) throws IOException {
1020     t.setAutoFlush(false);
1021     byte[] k = new byte[3];
1022     int rowCount = 0;
1023     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1024       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1025         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1026           k[0] = b1;
1027           k[1] = b2;
1028           k[2] = b3;
1029           Put put = new Put(k);
1030           put.add(f, null, k);
1031           t.put(put);
1032           rowCount++;
1033         }
1034       }
1035     }
1036     t.flushCommits();
1037     return rowCount;
1038   }
1039 
1040   /**
1041    * Load table of multiple column families with rows from 'aaa' to 'zzz'.
1042    * @param t Table
1043    * @param f Array of Families to load
1044    * @return Count of rows loaded.
1045    * @throws IOException
1046    */
1047   public int loadTable(final HTable t, final byte[][] f) throws IOException {
1048     t.setAutoFlush(false);
1049     byte[] k = new byte[3];
1050     int rowCount = 0;
1051     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1052       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1053         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1054           k[0] = b1;
1055           k[1] = b2;
1056           k[2] = b3;
1057           Put put = new Put(k);
1058           for (int i = 0; i < f.length; i++) {
1059             put.add(f[i], null, k);
1060           }
1061           t.put(put);
1062           rowCount++;
1063         }
1064       }
1065     }
1066     t.flushCommits();
1067     return rowCount;
1068   }
1069 
1070   /**
1071    * Load region with rows from 'aaa' to 'zzz'.
1072    * @param r Region
1073    * @param f Family
1074    * @return Count of rows loaded.
1075    * @throws IOException
1076    */
1077   public int loadRegion(final HRegion r, final byte[] f)
1078   throws IOException {
1079     return loadRegion(r, f, false);
1080   }
1081 
1082   /**
1083    * Load region with rows from 'aaa' to 'zzz'.
1084    * @param r Region
1085    * @param f Family
1086    * @param flush flush the cache if true
1087    * @return Count of rows loaded.
1088    * @throws IOException
1089    */
1090   public int loadRegion(final HRegion r, final byte[] f, final boolean flush)
1091       throws IOException {
1092     byte[] k = new byte[3];
1093     int rowCount = 0;
1094     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1095       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1096         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1097           k[0] = b1;
1098           k[1] = b2;
1099           k[2] = b3;
1100           Put put = new Put(k);
1101           put.add(f, null, k);
1102           if (r.getLog() == null) put.setWriteToWAL(false);
1103           r.put(put);
1104           rowCount++;
1105         }
1106       }
1107       if (flush) {
1108         r.flushcache();
1109       }
1110     }
1111     return rowCount;
1112   }
1113 
1114   /**
1115    * Return the number of rows in the given table.
1116    */
1117   public int countRows(final HTable table) throws IOException {
1118     Scan scan = new Scan();
1119     ResultScanner results = table.getScanner(scan);
1120     int count = 0;
1121     for (@SuppressWarnings("unused") Result res : results) {
1122       count++;
1123     }
1124     results.close();
1125     return count;
1126   }
1127 
1128   public int countRows(final HTable table, final byte[]... families) throws IOException {
1129     Scan scan = new Scan();
1130     for (byte[] family: families) {
1131       scan.addFamily(family);
1132     }
1133     ResultScanner results = table.getScanner(scan);
1134     int count = 0;
1135     for (@SuppressWarnings("unused") Result res : results) {
1136       count++;
1137     }
1138     results.close();
1139     return count;
1140   }
1141 
1142   /**
1143    * Return an md5 digest of the entire contents of a table.
1144    */
1145   public String checksumRows(final HTable table) throws Exception {
1146     Scan scan = new Scan();
1147     ResultScanner results = table.getScanner(scan);
1148     MessageDigest digest = MessageDigest.getInstance("MD5");
1149     for (Result res : results) {
1150       digest.update(res.getRow());
1151     }
1152     results.close();
1153     return digest.toString();
1154   }
1155 
1156   /**
1157    * Creates many regions names "aaa" to "zzz".
1158    *
1159    * @param table  The table to use for the data.
1160    * @param columnFamily  The family to insert the data into.
1161    * @return count of regions created.
1162    * @throws IOException When creating the regions fails.
1163    */
1164   public int createMultiRegions(HTable table, byte[] columnFamily)
1165   throws IOException {
1166     return createMultiRegions(table, columnFamily, true);
1167   }
1168 
1169   public static final byte[][] KEYS = {
1170     HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"),
1171     Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
1172     Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
1173     Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
1174     Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
1175     Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
1176     Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
1177     Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
1178     Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
1179   };
1180 
1181   public static final byte[][] KEYS_FOR_HBA_CREATE_TABLE = {
1182       Bytes.toBytes("bbb"),
1183       Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
1184       Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
1185       Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
1186       Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
1187       Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
1188       Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
1189       Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
1190       Bytes.toBytes("xxx"), Bytes.toBytes("yyy"), Bytes.toBytes("zzz")
1191   };
1192 
1193 
1194   /**
1195    * Creates many regions names "aaa" to "zzz".
1196    *
1197    * @param table  The table to use for the data.
1198    * @param columnFamily  The family to insert the data into.
1199    * @param cleanupFS  True if a previous region should be remove from the FS
1200    * @return count of regions created.
1201    * @throws IOException When creating the regions fails.
1202    */
1203   public int createMultiRegions(HTable table, byte[] columnFamily, boolean cleanupFS)
1204   throws IOException {
1205     return createMultiRegions(getConfiguration(), table, columnFamily, KEYS, cleanupFS);
1206   }
1207 
1208   /**
1209    * Creates the specified number of regions in the specified table.
1210    * @param c
1211    * @param table
1212    * @param family
1213    * @param numRegions
1214    * @return
1215    * @throws IOException
1216    */
1217   public int createMultiRegions(final Configuration c, final HTable table,
1218       final byte [] family, int numRegions)
1219   throws IOException {
1220     if (numRegions < 3) throw new IOException("Must create at least 3 regions");
1221     byte [] startKey = Bytes.toBytes("aaaaa");
1222     byte [] endKey = Bytes.toBytes("zzzzz");
1223     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
1224     byte [][] regionStartKeys = new byte[splitKeys.length+1][];
1225     for (int i=0;i<splitKeys.length;i++) {
1226       regionStartKeys[i+1] = splitKeys[i];
1227     }
1228     regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
1229     return createMultiRegions(c, table, family, regionStartKeys);
1230   }
1231 
1232   public int createMultiRegions(final Configuration c, final HTable table,
1233       final byte[] columnFamily, byte [][] startKeys) throws IOException {
1234     return createMultiRegions(c, table, columnFamily, startKeys, true);
1235   }
1236 
1237   public int createMultiRegions(final Configuration c, final HTable table,
1238           final byte[] columnFamily, byte [][] startKeys, boolean cleanupFS)
1239   throws IOException {
1240     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
1241     HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
1242     HTableDescriptor htd = table.getTableDescriptor();
1243     if(!htd.hasFamily(columnFamily)) {
1244       HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
1245       htd.addFamily(hcd);
1246     }
1247     // remove empty region - this is tricky as the mini cluster during the test
1248     // setup already has the "<tablename>,,123456789" row with an empty start
1249     // and end key. Adding the custom regions below adds those blindly,
1250     // including the new start region from empty to "bbb". lg
1251     List<byte[]> rows = getMetaTableRows(htd.getName());
1252     String regionToDeleteInFS = table
1253         .getRegionsInRange(Bytes.toBytes(""), Bytes.toBytes("")).get(0)
1254         .getRegionInfo().getEncodedName();
1255     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
1256     // add custom ones
1257     int count = 0;
1258     for (int i = 0; i < startKeys.length; i++) {
1259       int j = (i + 1) % startKeys.length;
1260       HRegionInfo hri = new HRegionInfo(table.getTableName(),
1261         startKeys[i], startKeys[j]);
1262       Put put = new Put(hri.getRegionName());
1263       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
1264         Writables.getBytes(hri));
1265       meta.put(put);
1266       LOG.info("createMultiRegions: inserted " + hri.toString());
1267       newRegions.add(hri);
1268       count++;
1269     }
1270     // see comment above, remove "old" (or previous) single region
1271     for (byte[] row : rows) {
1272       LOG.info("createMultiRegions: deleting meta row -> " +
1273         Bytes.toStringBinary(row));
1274       meta.delete(new Delete(row));
1275     }
1276     if (cleanupFS) {
1277       // see HBASE-7417 - this confused TestReplication
1278       // remove the "old" region from FS
1279       Path tableDir = new Path(getDefaultRootDirPath().toString()
1280           + System.getProperty("file.separator") + htd.getNameAsString()
1281           + System.getProperty("file.separator") + regionToDeleteInFS);
1282       FileSystem.get(c).delete(tableDir);
1283     }
1284     // flush cache of regions
1285     HConnection conn = table.getConnection();
1286     conn.clearRegionCache();
1287     // assign all the new regions IF table is enabled.
1288     HBaseAdmin admin = getHBaseAdmin();
1289     if (admin.isTableEnabled(table.getTableName())) {
1290       for(HRegionInfo hri : newRegions) {
1291         admin.assign(hri.getRegionName());
1292       }
1293     }
1294 
1295     meta.close();
1296 
1297     return count;
1298   }
1299 
1300   /**
1301    * Create rows in META for regions of the specified table with the specified
1302    * start keys.  The first startKey should be a 0 length byte array if you
1303    * want to form a proper range of regions.
1304    * @param conf
1305    * @param htd
1306    * @param startKeys
1307    * @return list of region info for regions added to meta
1308    * @throws IOException
1309    */
1310   public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
1311       final HTableDescriptor htd, byte [][] startKeys)
1312   throws IOException {
1313     HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
1314     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
1315     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
1316     // add custom ones
1317     for (int i = 0; i < startKeys.length; i++) {
1318       int j = (i + 1) % startKeys.length;
1319       HRegionInfo hri = new HRegionInfo(htd.getName(), startKeys[i],
1320           startKeys[j]);
1321       Put put = new Put(hri.getRegionName());
1322       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
1323         Writables.getBytes(hri));
1324       meta.put(put);
1325       LOG.info("createMultiRegionsInMeta: inserted " + hri.toString());
1326       newRegions.add(hri);
1327     }
1328 
1329     meta.close();
1330     return newRegions;
1331   }
1332 
1333   /**
1334    * Returns all rows from the .META. table.
1335    *
1336    * @throws IOException When reading the rows fails.
1337    */
1338   public List<byte[]> getMetaTableRows() throws IOException {
1339     // TODO: Redo using MetaReader class
1340     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
1341     List<byte[]> rows = new ArrayList<byte[]>();
1342     ResultScanner s = t.getScanner(new Scan());
1343     for (Result result : s) {
1344       LOG.info("getMetaTableRows: row -> " +
1345         Bytes.toStringBinary(result.getRow()));
1346       rows.add(result.getRow());
1347     }
1348     s.close();
1349     t.close();
1350     return rows;
1351   }
1352 
1353   /**
1354    * Returns all rows from the .META. table for a given user table
1355    *
1356    * @throws IOException When reading the rows fails.
1357    */
1358   public List<byte[]> getMetaTableRows(byte[] tableName) throws IOException {
1359     // TODO: Redo using MetaReader.
1360     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
1361     List<byte[]> rows = new ArrayList<byte[]>();
1362     ResultScanner s = t.getScanner(new Scan());
1363     for (Result result : s) {
1364       byte[] val = result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1365       if (val == null) {
1366         LOG.error("No region info for row " + Bytes.toString(result.getRow()));
1367         // TODO figure out what to do for this new hosed case.
1368         continue;
1369       }
1370       HRegionInfo info = Writables.getHRegionInfo(val);
1371       if (Bytes.compareTo(info.getTableName(), tableName) == 0) {
1372         LOG.info("getMetaTableRows: row -> " +
1373             Bytes.toStringBinary(result.getRow()) + info);
1374         rows.add(result.getRow());
1375       }
1376     }
1377     s.close();
1378     t.close();
1379     return rows;
1380   }
1381 
1382   /**
1383    * Tool to get the reference to the region server object that holds the
1384    * region of the specified user table.
1385    * It first searches for the meta rows that contain the region of the
1386    * specified table, then gets the index of that RS, and finally retrieves
1387    * the RS's reference.
1388    * @param tableName user table to lookup in .META.
1389    * @return region server that holds it, null if the row doesn't exist
1390    * @throws IOException
1391    */
1392   public HRegionServer getRSForFirstRegionInTable(byte[] tableName)
1393       throws IOException {
1394     List<byte[]> metaRows = getMetaTableRows(tableName);
1395     if (metaRows == null || metaRows.isEmpty()) {
1396       return null;
1397     }
1398     LOG.debug("Found " + metaRows.size() + " rows for table " +
1399       Bytes.toString(tableName));
1400     byte [] firstrow = metaRows.get(0);
1401     LOG.debug("FirstRow=" + Bytes.toString(firstrow));
1402     int index = getMiniHBaseCluster().getServerWith(firstrow);
1403     return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer();
1404   }
1405 
1406   /**
1407    * Starts a <code>MiniMRCluster</code> with a default number of
1408    * <code>TaskTracker</code>'s.
1409    *
1410    * @throws IOException When starting the cluster fails.
1411    */
1412   public void startMiniMapReduceCluster() throws IOException {
1413     startMiniMapReduceCluster(2);
1414   }
1415 
1416   /**
1417    * Starts a <code>MiniMRCluster</code>.
1418    *
1419    * @param servers  The number of <code>TaskTracker</code>'s to start.
1420    * @throws IOException When starting the cluster fails.
1421    */
1422   public void startMiniMapReduceCluster(final int servers) throws IOException {
1423     LOG.info("Starting mini mapreduce cluster...");
1424     if (dataTestDir == null) {
1425       setupDataTestDir();
1426     }
1427     // These are needed for the new and improved Map/Reduce framework
1428     Configuration c = getConfiguration();
1429     String logDir = c.get("hadoop.log.dir");
1430     String tmpDir = c.get("hadoop.tmp.dir");
1431     if (logDir == null) {
1432       logDir = tmpDir;
1433     }
1434     System.setProperty("hadoop.log.dir", logDir);
1435     c.set("mapred.output.dir", tmpDir);
1436 
1437     // Tests were failing because this process used 6GB of virtual memory and was getting killed.
1438     // we up the VM usable so that processes don't get killed.
1439     conf.setFloat("yarn.nodemanager.vmem-pmem-ratio", 8.0f);
1440 
1441     mrCluster = new MiniMRCluster(0, 0, servers,
1442       FileSystem.get(conf).getUri().toString(), 1, null, null, null, new JobConf(conf));
1443 
1444     JobConf jobConf = MapreduceTestingShim.getJobConf(mrCluster);
1445     if (jobConf == null) {
1446       jobConf = mrCluster.createJobConf();
1447     }
1448     jobConf.set("mapred.local.dir",
1449       conf.get("mapred.local.dir")); //Hadoop MiniMR overwrites this while it should not
1450 
1451     LOG.info("Mini mapreduce cluster started");
1452     JobConf mrClusterJobConf = mrCluster.createJobConf();
1453     c.set("mapred.job.tracker", mrClusterJobConf.get("mapred.job.tracker"));
1454     /* this for mrv2 support */
1455     conf.set("mapreduce.framework.name", "yarn");
1456     conf.setBoolean("yarn.is.minicluster", true);
1457     String rmAdress = mrClusterJobConf.get("yarn.resourcemanager.address");
1458     if (rmAdress != null) {
1459       conf.set("yarn.resourcemanager.address", rmAdress);
1460     }
1461     String schedulerAdress =
1462       mrClusterJobConf.get("yarn.resourcemanager.scheduler.address");
1463     if (schedulerAdress != null) {
1464       conf.set("yarn.resourcemanager.scheduler.address", schedulerAdress);
1465     }
1466   }
1467 
1468   /**
1469    * Stops the previously started <code>MiniMRCluster</code>.
1470    */
1471   public void shutdownMiniMapReduceCluster() {
1472     LOG.info("Stopping mini mapreduce cluster...");
1473     if (mrCluster != null) {
1474       mrCluster.shutdown();
1475       mrCluster = null;
1476     }
1477     // Restore configuration to point to local jobtracker
1478     conf.set("mapred.job.tracker", "local");
1479     LOG.info("Mini mapreduce cluster stopped");
1480   }
1481 
1482   /**
1483    * Switches the logger for the given class to DEBUG level.
1484    *
1485    * @param clazz  The class for which to switch to debug logging.
1486    */
1487   public void enableDebug(Class<?> clazz) {
1488     Log l = LogFactory.getLog(clazz);
1489     if (l instanceof Log4JLogger) {
1490       ((Log4JLogger) l).getLogger().setLevel(org.apache.log4j.Level.DEBUG);
1491     } else if (l instanceof Jdk14Logger) {
1492       ((Jdk14Logger) l).getLogger().setLevel(java.util.logging.Level.ALL);
1493     }
1494   }
1495 
1496   /**
1497    * Expire the Master's session
1498    * @throws Exception
1499    */
1500   public void expireMasterSession() throws Exception {
1501     HMaster master = getMiniHBaseCluster().getMaster();
1502     expireSession(master.getZooKeeper(), false);
1503   }
1504 
1505   /**
1506    * Expire a region server's session
1507    * @param index which RS
1508    * @throws Exception
1509    */
1510   public void expireRegionServerSession(int index) throws Exception {
1511     HRegionServer rs = getMiniHBaseCluster().getRegionServer(index);
1512     expireSession(rs.getZooKeeper(), false);
1513     decrementMinRegionServerCount();
1514   }
1515 
1516   private void decrementMinRegionServerCount() {
1517     // decrement the count for this.conf, for newly spwaned master
1518     // this.hbaseCluster shares this configuration too
1519     decrementMinRegionServerCount(getConfiguration());
1520 
1521     // each master thread keeps a copy of configuration
1522     for (MasterThread master : getHBaseCluster().getMasterThreads()) {
1523       decrementMinRegionServerCount(master.getMaster().getConfiguration());
1524     }
1525   }
1526 
1527   private void decrementMinRegionServerCount(Configuration conf) {
1528     int currentCount = conf.getInt(
1529         ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
1530     if (currentCount != -1) {
1531       conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
1532           Math.max(currentCount - 1, 1));
1533     }
1534   }
1535 
1536    /**
1537     * Expire a ZooKeeper session as recommended in ZooKeeper documentation
1538     * http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A4
1539     * There are issues when doing this:
1540     * [1] http://www.mail-archive.com/dev@zookeeper.apache.org/msg01942.html
1541     * [2] https://issues.apache.org/jira/browse/ZOOKEEPER-1105
1542     *
1543     * @param nodeZK - the ZK to make expiry
1544     * @param checkStatus - true to check if the we can create a HTable with the
1545     *                    current configuration.
1546     */
1547   public void expireSession(ZooKeeperWatcher nodeZK, boolean checkStatus)
1548     throws Exception {
1549     Configuration c = new Configuration(this.conf);
1550     String quorumServers = ZKConfig.getZKQuorumServersString(c);
1551     int sessionTimeout = 500;
1552     ZooKeeper zk = nodeZK.getRecoverableZooKeeper().getZooKeeper();
1553     byte[] password = zk.getSessionPasswd();
1554     long sessionID = zk.getSessionId();
1555 
1556     // Expiry seems to be asynchronous (see comment from P. Hunt in [1]),
1557     //  so we create a first watcher to be sure that the
1558     //  event was sent. We expect that if our watcher receives the event
1559     //  other watchers on the same machine will get is as well.
1560     // When we ask to close the connection, ZK does not close it before
1561     //  we receive all the events, so don't have to capture the event, just
1562     //  closing the connection should be enough.
1563     ZooKeeper monitor = new ZooKeeper(quorumServers,
1564       1000, new org.apache.zookeeper.Watcher(){
1565       @Override
1566       public void process(WatchedEvent watchedEvent) {
1567         LOG.info("Monitor ZKW received event="+watchedEvent);
1568       }
1569     } , sessionID, password);
1570 
1571     // Making it expire
1572     ZooKeeper newZK = new ZooKeeper(quorumServers,
1573         sessionTimeout, EmptyWatcher.instance, sessionID, password);
1574     newZK.close();
1575     LOG.info("ZK Closed Session 0x" + Long.toHexString(sessionID));
1576 
1577      // Now closing & waiting to be sure that the clients get it.
1578      monitor.close();
1579 
1580     if (checkStatus) {
1581       new HTable(new Configuration(conf), HConstants.META_TABLE_NAME).close();
1582     }
1583   }
1584 
1585   /**
1586    * Get the Mini HBase cluster.
1587    *
1588    * @return hbase cluster
1589    * @see #getHBaseClusterInterface()
1590    */
1591   public MiniHBaseCluster getHBaseCluster() {
1592     return getMiniHBaseCluster();
1593   }
1594 
1595   /**
1596    * Returns the HBaseCluster instance.
1597    * <p>Returned object can be any of the subclasses of HBaseCluster, and the
1598    * tests referring this should not assume that the cluster is a mini cluster or a
1599    * distributed one. If the test only works on a mini cluster, then specific
1600    * method {@link #getMiniHBaseCluster()} can be used instead w/o the
1601    * need to type-cast.
1602    */
1603   public HBaseCluster getHBaseClusterInterface() {
1604     //implementation note: we should rename this method as #getHBaseCluster(),
1605     //but this would require refactoring 90+ calls.
1606     return hbaseCluster;
1607   }
1608 
1609   /**
1610    * Returns a HBaseAdmin instance.
1611    * This instance is shared between HBaseTestingUtility intance users.
1612    * Don't close it, it will be closed automatically when the
1613    * cluster shutdowns
1614    *
1615    * @return The HBaseAdmin instance.
1616    * @throws IOException
1617    */
1618   public synchronized HBaseAdmin getHBaseAdmin()
1619   throws IOException {
1620     if (hbaseAdmin == null){
1621       hbaseAdmin = new HBaseAdmin(new Configuration(getConfiguration()));
1622     }
1623     return hbaseAdmin;
1624   }
1625   private HBaseAdmin hbaseAdmin = null;
1626 
1627   /**
1628    * Closes the named region.
1629    *
1630    * @param regionName  The region to close.
1631    * @throws IOException
1632    */
1633   public void closeRegion(String regionName) throws IOException {
1634     closeRegion(Bytes.toBytes(regionName));
1635   }
1636 
1637   /**
1638    * Closes the named region.
1639    *
1640    * @param regionName  The region to close.
1641    * @throws IOException
1642    */
1643   public void closeRegion(byte[] regionName) throws IOException {
1644     getHBaseAdmin().closeRegion(regionName, null);
1645   }
1646 
1647   /**
1648    * Closes the region containing the given row.
1649    *
1650    * @param row  The row to find the containing region.
1651    * @param table  The table to find the region.
1652    * @throws IOException
1653    */
1654   public void closeRegionByRow(String row, HTable table) throws IOException {
1655     closeRegionByRow(Bytes.toBytes(row), table);
1656   }
1657 
1658   /**
1659    * Closes the region containing the given row.
1660    *
1661    * @param row  The row to find the containing region.
1662    * @param table  The table to find the region.
1663    * @throws IOException
1664    */
1665   public void closeRegionByRow(byte[] row, HTable table) throws IOException {
1666     HRegionLocation hrl = table.getRegionLocation(row);
1667     closeRegion(hrl.getRegionInfo().getRegionName());
1668   }
1669 
1670   public MiniZooKeeperCluster getZkCluster() {
1671     return zkCluster;
1672   }
1673 
1674   public void setZkCluster(MiniZooKeeperCluster zkCluster) {
1675     this.passedZkCluster = true;
1676     this.zkCluster = zkCluster;
1677     conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkCluster.getClientPort());
1678   }
1679 
1680   public MiniDFSCluster getDFSCluster() {
1681     return dfsCluster;
1682   }
1683 
1684   public void setDFSCluster(MiniDFSCluster cluster) throws IOException {
1685     if (dfsCluster != null && dfsCluster.isClusterUp()) {
1686       throw new IOException("DFSCluster is already running! Shut it down first.");
1687     }
1688     this.dfsCluster = cluster;
1689   }
1690 
1691   public FileSystem getTestFileSystem() throws IOException {
1692     return HFileSystem.get(conf);
1693   }
1694 
1695   /**
1696    * @return True if we removed the test dir
1697    * @throws IOException
1698    */
1699   public boolean cleanupTestDir() throws IOException {
1700     if (dataTestDir == null ){
1701       return false;
1702     } else {
1703       boolean ret = deleteDir(getDataTestDir());
1704       dataTestDir = null;
1705       return ret;
1706     }
1707   }
1708 
1709   /**
1710    * @param subdir Test subdir name.
1711    * @return True if we removed the test dir
1712    * @throws IOException
1713    */
1714   public boolean cleanupTestDir(final String subdir) throws IOException {
1715     if (dataTestDir == null){
1716       return false;
1717     }
1718     return deleteDir(getDataTestDir(subdir));
1719   }
1720 
1721   /**
1722    * @param dir Directory to delete
1723    * @return True if we deleted it.
1724    * @throws IOException
1725    */
1726   public boolean deleteDir(final Path dir) throws IOException {
1727     FileSystem fs = getTestFileSystem();
1728     if (fs.exists(dir)) {
1729       return fs.delete(getDataTestDir(), true);
1730     }
1731     return false;
1732   }
1733 
1734   public void waitTableAvailable(byte[] table, long timeoutMillis)
1735   throws InterruptedException, IOException {
1736     long startWait = System.currentTimeMillis();
1737     while (!getHBaseAdmin().isTableAvailable(table)) {
1738       assertTrue("Timed out waiting for table to become available " +
1739         Bytes.toStringBinary(table),
1740         System.currentTimeMillis() - startWait < timeoutMillis);
1741       Thread.sleep(200);
1742     }
1743   }
1744 
1745   public void waitTableEnabled(byte[] table, long timeoutMillis)
1746   throws InterruptedException, IOException {
1747     long startWait = System.currentTimeMillis();
1748     while (!getHBaseAdmin().isTableAvailable(table) &&
1749            !getHBaseAdmin().isTableEnabled(table)) {
1750       assertTrue("Timed out waiting for table to become available and enabled " +
1751          Bytes.toStringBinary(table),
1752          System.currentTimeMillis() - startWait < timeoutMillis);
1753       Thread.sleep(200);
1754     }
1755   }
1756 
1757   /**
1758    * Make sure that at least the specified number of region servers
1759    * are running
1760    * @param num minimum number of region servers that should be running
1761    * @return true if we started some servers
1762    * @throws IOException
1763    */
1764   public boolean ensureSomeRegionServersAvailable(final int num)
1765       throws IOException {
1766     boolean startedServer = false;
1767     MiniHBaseCluster hbaseCluster = getMiniHBaseCluster();
1768     for (int i=hbaseCluster.getLiveRegionServerThreads().size(); i<num; ++i) {
1769       LOG.info("Started new server=" + hbaseCluster.startRegionServer());
1770       startedServer = true;
1771     }
1772 
1773     return startedServer;
1774   }
1775 
1776   /**
1777    * Make sure that at least the specified number of region servers
1778    * are running. We don't count the ones that are currently stopping or are
1779    * stopped.
1780    * @param num minimum number of region servers that should be running
1781    * @return true if we started some servers
1782    * @throws IOException
1783    */
1784   public boolean ensureSomeNonStoppedRegionServersAvailable(final int num)
1785     throws IOException {
1786     boolean startedServer = ensureSomeRegionServersAvailable(num);
1787 
1788     int nonStoppedServers = 0;
1789     for (JVMClusterUtil.RegionServerThread rst :
1790       getMiniHBaseCluster().getRegionServerThreads()) {
1791 
1792       HRegionServer hrs = rst.getRegionServer();
1793       if (hrs.isStopping() || hrs.isStopped()) {
1794         LOG.info("A region server is stopped or stopping:"+hrs);
1795       } else {
1796         nonStoppedServers++;
1797       }
1798     }
1799     for (int i=nonStoppedServers; i<num; ++i) {
1800       LOG.info("Started new server=" + getMiniHBaseCluster().startRegionServer());
1801       startedServer = true;
1802     }
1803     return startedServer;
1804   }
1805 
1806 
1807   /**
1808    * This method clones the passed <code>c</code> configuration setting a new
1809    * user into the clone.  Use it getting new instances of FileSystem.  Only
1810    * works for DistributedFileSystem.
1811    * @param c Initial configuration
1812    * @param differentiatingSuffix Suffix to differentiate this user from others.
1813    * @return A new configuration instance with a different user set into it.
1814    * @throws IOException
1815    */
1816   public static User getDifferentUser(final Configuration c,
1817     final String differentiatingSuffix)
1818   throws IOException {
1819     FileSystem currentfs = FileSystem.get(c);
1820     if (!(currentfs instanceof DistributedFileSystem)) {
1821       return User.getCurrent();
1822     }
1823     // Else distributed filesystem.  Make a new instance per daemon.  Below
1824     // code is taken from the AppendTestUtil over in hdfs.
1825     String username = User.getCurrent().getName() +
1826       differentiatingSuffix;
1827     User user = User.createUserForTesting(c, username,
1828         new String[]{"supergroup"});
1829     return user;
1830   }
1831 
1832   /**
1833    * Set maxRecoveryErrorCount in DFSClient.  In 0.20 pre-append its hard-coded to 5 and
1834    * makes tests linger.  Here is the exception you'll see:
1835    * <pre>
1836    * 2010-06-15 11:52:28,511 WARN  [DataStreamer for file /hbase/.logs/hlog.1276627923013 block blk_928005470262850423_1021] hdfs.DFSClient$DFSOutputStream(2657): Error Recovery for block blk_928005470262850423_1021 failed  because recovery from primary datanode 127.0.0.1:53683 failed 4 times.  Pipeline was 127.0.0.1:53687, 127.0.0.1:53683. Will retry...
1837    * </pre>
1838    * @param stream A DFSClient.DFSOutputStream.
1839    * @param max
1840    * @throws NoSuchFieldException
1841    * @throws SecurityException
1842    * @throws IllegalAccessException
1843    * @throws IllegalArgumentException
1844    */
1845   public static void setMaxRecoveryErrorCount(final OutputStream stream,
1846       final int max) {
1847     try {
1848       Class<?> [] clazzes = DFSClient.class.getDeclaredClasses();
1849       for (Class<?> clazz: clazzes) {
1850         String className = clazz.getSimpleName();
1851         if (className.equals("DFSOutputStream")) {
1852           if (clazz.isInstance(stream)) {
1853             Field maxRecoveryErrorCountField =
1854               stream.getClass().getDeclaredField("maxRecoveryErrorCount");
1855             maxRecoveryErrorCountField.setAccessible(true);
1856             maxRecoveryErrorCountField.setInt(stream, max);
1857             break;
1858           }
1859         }
1860       }
1861     } catch (Exception e) {
1862       LOG.info("Could not set max recovery field", e);
1863     }
1864   }
1865 
1866   /**
1867    * Wait until all regions for a table in .META. have a non-empty
1868    * info:server, up to 60 seconds.  This means all regions have been deployed,
1869    * master has been informed and updated .META. with the regions deployed
1870    * server.
1871    * @param tableName the table name
1872    * @throws IOException
1873    */
1874   public void waitUntilAllRegionsAssigned(final byte[] tableName) throws IOException {
1875     waitUntilAllRegionsAssigned(tableName, 60000);
1876   }
1877 
1878   /**
1879    * Wait until all regions for a table in .META. have a non-empty
1880    * info:server, or until timeout.  This means all regions have been
1881    * deployed, master has been informed and updated .META. with the regions
1882    * deployed server.
1883    * @param tableName the table name
1884    * @param timeout timeout, in milliseconds
1885    * @throws IOException
1886    */
1887   public void waitUntilAllRegionsAssigned(final byte[] tableName, final long timeout)
1888       throws IOException {
1889     long deadline = System.currentTimeMillis() + timeout;
1890     HTable meta = new HTable(getConfiguration(), HConstants.META_TABLE_NAME);
1891     try {
1892       while (true) {
1893         boolean allRegionsAssigned = true;
1894         Scan scan = new Scan();
1895         scan.addFamily(HConstants.CATALOG_FAMILY);
1896         ResultScanner s = meta.getScanner(scan);
1897         try {
1898           Result r;
1899           while ((r = s.next()) != null) {
1900             byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1901             HRegionInfo info = Writables.getHRegionInfoOrNull(b);
1902             if (info != null && Bytes.equals(info.getTableName(), tableName)) {
1903               b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
1904               allRegionsAssigned &= (b != null);
1905             }
1906           }
1907         } finally {
1908           s.close();
1909         }
1910         if (allRegionsAssigned) {
1911           return;
1912         }
1913         long now = System.currentTimeMillis();
1914         if (now > deadline) {
1915           throw new IOException("Timeout waiting for all regions of " +
1916             Bytes.toStringBinary(tableName) + " to be assigned");
1917         }
1918         try {
1919           Thread.sleep(deadline - now < 200 ? deadline - now : 200);
1920         } catch (InterruptedException e) {
1921           throw new IOException(e);
1922         }
1923       }
1924     } finally {
1925       meta.close();
1926     }
1927   }
1928 
1929   /**
1930    * Do a small get/scan against one store. This is required because store
1931    * has no actual methods of querying itself, and relies on StoreScanner.
1932    */
1933   public static List<KeyValue> getFromStoreFile(Store store,
1934                                                 Get get) throws IOException {
1935     MultiVersionConsistencyControl.resetThreadReadPoint();
1936     Scan scan = new Scan(get);
1937     InternalScanner scanner = (InternalScanner) store.getScanner(scan,
1938         scan.getFamilyMap().get(store.getFamily().getName()));
1939 
1940     List<KeyValue> result = new ArrayList<KeyValue>();
1941     scanner.next(result);
1942     if (!result.isEmpty()) {
1943       // verify that we are on the row we want:
1944       KeyValue kv = result.get(0);
1945       if (!Bytes.equals(kv.getRow(), get.getRow())) {
1946         result.clear();
1947       }
1948     }
1949     scanner.close();
1950     return result;
1951   }
1952 
1953   /**
1954    * Do a small get/scan against one store. This is required because store
1955    * has no actual methods of querying itself, and relies on StoreScanner.
1956    */
1957   public static List<KeyValue> getFromStoreFile(Store store,
1958                                                 byte [] row,
1959                                                 NavigableSet<byte[]> columns
1960                                                 ) throws IOException {
1961     Get get = new Get(row);
1962     Map<byte[], NavigableSet<byte[]>> s = get.getFamilyMap();
1963     s.put(store.getFamily().getName(), columns);
1964 
1965     return getFromStoreFile(store,get);
1966   }
1967 
1968   /**
1969    * Gets a ZooKeeperWatcher.
1970    * @param TEST_UTIL
1971    */
1972   public static ZooKeeperWatcher getZooKeeperWatcher(
1973       HBaseTestingUtility TEST_UTIL) throws ZooKeeperConnectionException,
1974       IOException {
1975     ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
1976         "unittest", new Abortable() {
1977           boolean aborted = false;
1978 
1979           @Override
1980           public void abort(String why, Throwable e) {
1981             aborted = true;
1982             throw new RuntimeException("Fatal ZK error, why=" + why, e);
1983           }
1984 
1985           @Override
1986           public boolean isAborted() {
1987             return aborted;
1988           }
1989         });
1990     return zkw;
1991   }
1992 
1993   /**
1994    * Creates a znode with OPENED state.
1995    * @param TEST_UTIL
1996    * @param region
1997    * @param serverName
1998    * @return
1999    * @throws IOException
2000    * @throws ZooKeeperConnectionException
2001    * @throws KeeperException
2002    * @throws NodeExistsException
2003    */
2004   public static ZooKeeperWatcher createAndForceNodeToOpenedState(
2005       HBaseTestingUtility TEST_UTIL, HRegion region,
2006       ServerName serverName) throws ZooKeeperConnectionException,
2007       IOException, KeeperException, NodeExistsException {
2008     ZooKeeperWatcher zkw = getZooKeeperWatcher(TEST_UTIL);
2009     ZKAssign.createNodeOffline(zkw, region.getRegionInfo(), serverName);
2010     int version = ZKAssign.transitionNodeOpening(zkw, region
2011         .getRegionInfo(), serverName);
2012     ZKAssign.transitionNodeOpened(zkw, region.getRegionInfo(), serverName,
2013         version);
2014     return zkw;
2015   }
2016 
2017   public static void assertKVListsEqual(String additionalMsg,
2018       final List<KeyValue> expected,
2019       final List<KeyValue> actual) {
2020     final int eLen = expected.size();
2021     final int aLen = actual.size();
2022     final int minLen = Math.min(eLen, aLen);
2023 
2024     int i;
2025     for (i = 0; i < minLen
2026         && KeyValue.COMPARATOR.compare(expected.get(i), actual.get(i)) == 0;
2027         ++i) {}
2028 
2029     if (additionalMsg == null) {
2030       additionalMsg = "";
2031     }
2032     if (!additionalMsg.isEmpty()) {
2033       additionalMsg = ". " + additionalMsg;
2034     }
2035 
2036     if (eLen != aLen || i != minLen) {
2037       throw new AssertionError(
2038           "Expected and actual KV arrays differ at position " + i + ": " +
2039           safeGetAsStr(expected, i) + " (length " + eLen +") vs. " +
2040           safeGetAsStr(actual, i) + " (length " + aLen + ")" + additionalMsg);
2041     }
2042   }
2043 
2044   private static <T> String safeGetAsStr(List<T> lst, int i) {
2045     if (0 <= i && i < lst.size()) {
2046       return lst.get(i).toString();
2047     } else {
2048       return "<out_of_range>";
2049     }
2050   }
2051 
2052   public String getClusterKey() {
2053     return conf.get(HConstants.ZOOKEEPER_QUORUM) + ":"
2054         + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":"
2055         + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
2056             HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
2057   }
2058 
2059   /** Creates a random table with the given parameters */
2060   public HTable createRandomTable(String tableName,
2061       final Collection<String> families,
2062       final int maxVersions,
2063       final int numColsPerRow,
2064       final int numFlushes,
2065       final int numRegions,
2066       final int numRowsPerFlush)
2067       throws IOException, InterruptedException {
2068 
2069     LOG.info("\n\nCreating random table " + tableName + " with " + numRegions +
2070         " regions, " + numFlushes + " storefiles per region, " +
2071         numRowsPerFlush + " rows per flush, maxVersions=" +  maxVersions +
2072         "\n");
2073 
2074     final Random rand = new Random(tableName.hashCode() * 17L + 12938197137L);
2075     final int numCF = families.size();
2076     final byte[][] cfBytes = new byte[numCF][];
2077     final byte[] tableNameBytes = Bytes.toBytes(tableName);
2078 
2079     {
2080       int cfIndex = 0;
2081       for (String cf : families) {
2082         cfBytes[cfIndex++] = Bytes.toBytes(cf);
2083       }
2084     }
2085 
2086     final int actualStartKey = 0;
2087     final int actualEndKey = Integer.MAX_VALUE;
2088     final int keysPerRegion = (actualEndKey - actualStartKey) / numRegions;
2089     final int splitStartKey = actualStartKey + keysPerRegion;
2090     final int splitEndKey = actualEndKey - keysPerRegion;
2091     final String keyFormat = "%08x";
2092     final HTable table = createTable(tableNameBytes, cfBytes,
2093         maxVersions,
2094         Bytes.toBytes(String.format(keyFormat, splitStartKey)),
2095         Bytes.toBytes(String.format(keyFormat, splitEndKey)),
2096         numRegions);
2097     if (hbaseCluster != null) {
2098       getMiniHBaseCluster().flushcache(HConstants.META_TABLE_NAME);
2099     }
2100 
2101     for (int iFlush = 0; iFlush < numFlushes; ++iFlush) {
2102       for (int iRow = 0; iRow < numRowsPerFlush; ++iRow) {
2103         final byte[] row = Bytes.toBytes(String.format(keyFormat,
2104             actualStartKey + rand.nextInt(actualEndKey - actualStartKey)));
2105 
2106         Put put = new Put(row);
2107         Delete del = new Delete(row);
2108         for (int iCol = 0; iCol < numColsPerRow; ++iCol) {
2109           final byte[] cf = cfBytes[rand.nextInt(numCF)];
2110           final long ts = rand.nextInt();
2111           final byte[] qual = Bytes.toBytes("col" + iCol);
2112           if (rand.nextBoolean()) {
2113             final byte[] value = Bytes.toBytes("value_for_row_" + iRow +
2114                 "_cf_" + Bytes.toStringBinary(cf) + "_col_" + iCol + "_ts_" +
2115                 ts + "_random_" + rand.nextLong());
2116             put.add(cf, qual, ts, value);
2117           } else if (rand.nextDouble() < 0.8) {
2118             del.deleteColumn(cf, qual, ts);
2119           } else {
2120             del.deleteColumns(cf, qual, ts);
2121           }
2122         }
2123 
2124         if (!put.isEmpty()) {
2125           table.put(put);
2126         }
2127 
2128         if (!del.isEmpty()) {
2129           table.delete(del);
2130         }
2131       }
2132       LOG.info("Initiating flush #" + iFlush + " for table " + tableName);
2133       table.flushCommits();
2134       if (hbaseCluster != null) {
2135         getMiniHBaseCluster().flushcache(tableNameBytes);
2136       }
2137     }
2138 
2139     return table;
2140   }
2141 
2142   private static final int MIN_RANDOM_PORT = 0xc000;
2143   private static final int MAX_RANDOM_PORT = 0xfffe;
2144 
2145   /**
2146    * Returns a random port. These ports cannot be registered with IANA and are
2147    * intended for dynamic allocation (see http://bit.ly/dynports).
2148    */
2149   public static int randomPort() {
2150     return MIN_RANDOM_PORT
2151         + new Random().nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT);
2152   }
2153 
2154   public static int randomFreePort() {
2155     int port = 0;
2156     do {
2157       port = randomPort();
2158       try {
2159         ServerSocket sock = new ServerSocket(port);
2160         sock.close();
2161       } catch (IOException ex) {
2162         port = 0;
2163       }
2164     } while (port == 0);
2165     return port;
2166   }
2167 
2168   public static void waitForHostPort(String host, int port)
2169       throws IOException {
2170     final int maxTimeMs = 10000;
2171     final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
2172     IOException savedException = null;
2173     LOG.info("Waiting for server at " + host + ":" + port);
2174     for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
2175       try {
2176         Socket sock = new Socket(InetAddress.getByName(host), port);
2177         sock.close();
2178         savedException = null;
2179         LOG.info("Server at " + host + ":" + port + " is available");
2180         break;
2181       } catch (UnknownHostException e) {
2182         throw new IOException("Failed to look up " + host, e);
2183       } catch (IOException e) {
2184         savedException = e;
2185       }
2186       Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
2187     }
2188 
2189     if (savedException != null) {
2190       throw savedException;
2191     }
2192   }
2193 
2194   /**
2195    * Creates a pre-split table for load testing. If the table already exists,
2196    * logs a warning and continues.
2197    * @return the number of regions the table was split into
2198    */
2199   public static int createPreSplitLoadTestTable(Configuration conf,
2200       byte[] tableName, byte[] columnFamily, Algorithm compression,
2201       DataBlockEncoding dataBlockEncoding) throws IOException {
2202     HTableDescriptor desc = new HTableDescriptor(tableName);
2203     HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
2204     hcd.setDataBlockEncoding(dataBlockEncoding);
2205     hcd.setCompressionType(compression);
2206     return createPreSplitLoadTestTable(conf, desc, hcd);
2207   }
2208 
2209   /**
2210    * Creates a pre-split table for load testing. If the table already exists,
2211    * logs a warning and continues.
2212    * @return the number of regions the table was split into
2213    */
2214   public static int createPreSplitLoadTestTable(Configuration conf,
2215       HTableDescriptor desc, HColumnDescriptor hcd) throws IOException {
2216     if (!desc.hasFamily(hcd.getName())) {
2217       desc.addFamily(hcd);
2218     }
2219 
2220     int totalNumberOfRegions = 0;
2221     HBaseAdmin admin = new HBaseAdmin(conf);
2222     try {
2223       // create a table a pre-splits regions.
2224       // The number of splits is set as:
2225       //    region servers * regions per region server).
2226       int numberOfServers = admin.getClusterStatus().getServers().size();
2227       if (numberOfServers == 0) {
2228         throw new IllegalStateException("No live regionservers");
2229       }
2230 
2231       totalNumberOfRegions = numberOfServers * DEFAULT_REGIONS_PER_SERVER;
2232       LOG.info("Number of live regionservers: " + numberOfServers + ", " +
2233           "pre-splitting table into " + totalNumberOfRegions + " regions " +
2234           "(default regions per server: " + DEFAULT_REGIONS_PER_SERVER + ")");
2235 
2236       byte[][] splits = new RegionSplitter.HexStringSplit().split(
2237           totalNumberOfRegions);
2238 
2239       admin.createTable(desc, splits);
2240       admin.close();
2241     } catch (MasterNotRunningException e) {
2242       LOG.error("Master not running", e);
2243       throw new IOException(e);
2244     } catch (TableExistsException e) {
2245       LOG.warn("Table " + Bytes.toStringBinary(desc.getName()) +
2246           " already exists, continuing");
2247     } finally {
2248       admin.close();
2249     }
2250     return totalNumberOfRegions;
2251   }
2252 
2253   public static int getMetaRSPort(Configuration conf) throws IOException {
2254     HTable table = new HTable(conf, HConstants.META_TABLE_NAME);
2255     HRegionLocation hloc = table.getRegionLocation(Bytes.toBytes(""));
2256     table.close();
2257     return hloc.getPort();
2258   }
2259 
2260   public HRegion createTestRegion(String tableName, HColumnDescriptor hcd)
2261       throws IOException {
2262     HTableDescriptor htd = new HTableDescriptor(tableName);
2263     htd.addFamily(hcd);
2264     HRegionInfo info =
2265         new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
2266     HRegion region =
2267         HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), htd);
2268     return region;
2269   }
2270 
2271   /**
2272    * Create region split keys between startkey and endKey
2273    *
2274    * @param startKey
2275    * @param endKey
2276    * @param numRegions the number of regions to be created. it has to be greater than 3.
2277    * @return
2278    */
2279   public byte[][] getRegionSplitStartKeys(byte[] startKey, byte[] endKey, int numRegions){
2280     assertTrue(numRegions>3);
2281     byte [][] tmpSplitKeys = Bytes.split(startKey, endKey, numRegions - 3);
2282     byte [][] result = new byte[tmpSplitKeys.length+1][];
2283     for (int i=0;i<tmpSplitKeys.length;i++) {
2284       result[i+1] = tmpSplitKeys[i];
2285     }
2286     result[0] = HConstants.EMPTY_BYTE_ARRAY;
2287     return result;
2288   }
2289 
2290   /**
2291    * Create a set of column descriptors with the combination of compression,
2292    * encoding, bloom codecs available.
2293    * @return the list of column descriptors
2294    */
2295   public static List<HColumnDescriptor> generateColumnDescriptors() {
2296     return generateColumnDescriptors("");
2297   }
2298 
2299   /**
2300    * Create a set of column descriptors with the combination of compression,
2301    * encoding, bloom codecs available.
2302    * @param prefix family names prefix
2303    * @return the list of column descriptors
2304    */
2305   public static List<HColumnDescriptor> generateColumnDescriptors(final String prefix) {
2306     List<HColumnDescriptor> htds = new ArrayList<HColumnDescriptor>();
2307     long familyId = 0;
2308     for (Compression.Algorithm compressionType: getSupportedCompressionAlgorithms()) {
2309       for (DataBlockEncoding encodingType: DataBlockEncoding.values()) {
2310         for (StoreFile.BloomType bloomType: StoreFile.BloomType.values()) {
2311           String name = String.format("%s-cf-!@#&-%d!@#", prefix, familyId);
2312           HColumnDescriptor htd = new HColumnDescriptor(name);
2313           htd.setCompressionType(compressionType);
2314           htd.setDataBlockEncoding(encodingType);
2315           htd.setBloomFilterType(bloomType);
2316           htds.add(htd);
2317           familyId++;
2318         }
2319       }
2320     }
2321     return htds;
2322   }
2323 
2324   /**
2325    * Get supported compression algorithms.
2326    * @return supported compression algorithms.
2327    */
2328   public static Compression.Algorithm[] getSupportedCompressionAlgorithms() {
2329     String[] allAlgos = HFile.getSupportedCompressionAlgorithms();
2330     List<Compression.Algorithm> supportedAlgos = new ArrayList<Compression.Algorithm>();
2331     for (String algoName : allAlgos) {
2332       try {
2333         Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName);
2334         algo.getCompressor();
2335         supportedAlgos.add(algo);
2336       } catch (Throwable t) {
2337         // this algo is not available
2338       }
2339     }
2340     return supportedAlgos.toArray(new Compression.Algorithm[0]);
2341   }
2342 }