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