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 == null || 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     return loadTable(t, f, null);
1049   }
1050 
1051   /**
1052    * Load table of multiple column families with rows from 'aaa' to 'zzz'.
1053    * @param t Table
1054    * @param f Array of Families to load
1055    * @param value the values of the KVs. If null is passed, the row key is used as value
1056    * @return Count of rows loaded.
1057    * @throws IOException
1058    */
1059   public int loadTable(final HTable t, final byte[][] f, byte[] value) throws IOException {
1060     t.setAutoFlush(false);
1061     byte[] k = new byte[3];
1062     int rowCount = 0;
1063     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1064       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1065         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1066           k[0] = b1;
1067           k[1] = b2;
1068           k[2] = b3;
1069           Put put = new Put(k);
1070           for (int i = 0; i < f.length; i++) {
1071             put.add(f[i], null, value != null ? value : k);
1072           }
1073           t.put(put);
1074           rowCount++;
1075         }
1076       }
1077     }
1078     t.flushCommits();
1079     return rowCount;
1080   }
1081 
1082   /** A tracker for tracking and validating table rows
1083    * generated with {@link HBaseTestingUtility#loadTable(HTable, byte[])}
1084    */
1085   public static class SeenRowTracker {
1086     int dim = 'z' - 'a' + 1;
1087     int[][][] seenRows = new int[dim][dim][dim]; //count of how many times the row is seen
1088     byte[] startRow;
1089     byte[] stopRow;
1090 
1091     public SeenRowTracker(byte[] startRow, byte[] stopRow) {
1092       this.startRow = startRow;
1093       this.stopRow = stopRow;
1094     }
1095 
1096     int i(byte b) {
1097       return b - 'a';
1098     }
1099 
1100     public void addRow(byte[] row) {
1101       seenRows[i(row[0])][i(row[1])][i(row[2])]++;
1102     }
1103 
1104     /** Validate that all the rows between startRow and stopRow are seen exactly once, and
1105      * all other rows none
1106      */
1107     public void validate() {
1108       for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1109         for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1110           for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1111             int count = seenRows[i(b1)][i(b2)][i(b3)];
1112             int expectedCount = 0;
1113             if (Bytes.compareTo(new byte[] {b1,b2,b3}, startRow) >= 0
1114                 && Bytes.compareTo(new byte[] {b1,b2,b3}, stopRow) < 0) {
1115               expectedCount = 1;
1116             }
1117             if (count != expectedCount) {
1118               String row = new String(new byte[] {b1,b2,b3});
1119               throw new RuntimeException("Row:" + row + " has a seen count of " + count + " instead of " + expectedCount);
1120             }
1121           }
1122         }
1123       }
1124     }
1125   }
1126 
1127   /**
1128    * Load region with rows from 'aaa' to 'zzz'.
1129    * @param r Region
1130    * @param f Family
1131    * @return Count of rows loaded.
1132    * @throws IOException
1133    */
1134   public int loadRegion(final HRegion r, final byte[] f)
1135   throws IOException {
1136     return loadRegion(r, f, false);
1137   }
1138 
1139   /**
1140    * Load region with rows from 'aaa' to 'zzz'.
1141    * @param r Region
1142    * @param f Family
1143    * @param flush flush the cache if true
1144    * @return Count of rows loaded.
1145    * @throws IOException
1146    */
1147   public int loadRegion(final HRegion r, final byte[] f, final boolean flush)
1148       throws IOException {
1149     byte[] k = new byte[3];
1150     int rowCount = 0;
1151     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1152       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1153         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1154           k[0] = b1;
1155           k[1] = b2;
1156           k[2] = b3;
1157           Put put = new Put(k);
1158           put.add(f, null, k);
1159           if (r.getLog() == null) put.setWriteToWAL(false);
1160           r.put(put);
1161           rowCount++;
1162         }
1163       }
1164       if (flush) {
1165         r.flushcache();
1166       }
1167     }
1168     return rowCount;
1169   }
1170 
1171   /**
1172    * Return the number of rows in the given table.
1173    */
1174   public int countRows(final HTable table) throws IOException {
1175     Scan scan = new Scan();
1176     ResultScanner results = table.getScanner(scan);
1177     int count = 0;
1178     for (@SuppressWarnings("unused") Result res : results) {
1179       count++;
1180     }
1181     results.close();
1182     return count;
1183   }
1184 
1185   public int countRows(final HTable table, final byte[]... families) throws IOException {
1186     Scan scan = new Scan();
1187     for (byte[] family: families) {
1188       scan.addFamily(family);
1189     }
1190     ResultScanner results = table.getScanner(scan);
1191     int count = 0;
1192     for (@SuppressWarnings("unused") Result res : results) {
1193       count++;
1194     }
1195     results.close();
1196     return count;
1197   }
1198 
1199   /**
1200    * Return an md5 digest of the entire contents of a table.
1201    */
1202   public String checksumRows(final HTable table) throws Exception {
1203     Scan scan = new Scan();
1204     ResultScanner results = table.getScanner(scan);
1205     MessageDigest digest = MessageDigest.getInstance("MD5");
1206     for (Result res : results) {
1207       digest.update(res.getRow());
1208     }
1209     results.close();
1210     return digest.toString();
1211   }
1212 
1213   /**
1214    * Creates many regions names "aaa" to "zzz".
1215    *
1216    * @param table  The table to use for the data.
1217    * @param columnFamily  The family to insert the data into.
1218    * @return count of regions created.
1219    * @throws IOException When creating the regions fails.
1220    */
1221   public int createMultiRegions(HTable table, byte[] columnFamily)
1222   throws IOException {
1223     return createMultiRegions(table, columnFamily, true);
1224   }
1225 
1226   public static final byte[][] KEYS = {
1227     HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"),
1228     Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
1229     Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
1230     Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
1231     Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
1232     Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
1233     Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
1234     Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
1235     Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
1236   };
1237 
1238   public static final byte[][] KEYS_FOR_HBA_CREATE_TABLE = {
1239       Bytes.toBytes("bbb"),
1240       Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
1241       Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
1242       Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
1243       Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
1244       Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
1245       Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
1246       Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
1247       Bytes.toBytes("xxx"), Bytes.toBytes("yyy"), Bytes.toBytes("zzz")
1248   };
1249 
1250 
1251   /**
1252    * Creates many regions names "aaa" to "zzz".
1253    *
1254    * @param table  The table to use for the data.
1255    * @param columnFamily  The family to insert the data into.
1256    * @param cleanupFS  True if a previous region should be remove from the FS
1257    * @return count of regions created.
1258    * @throws IOException When creating the regions fails.
1259    */
1260   public int createMultiRegions(HTable table, byte[] columnFamily, boolean cleanupFS)
1261   throws IOException {
1262     return createMultiRegions(getConfiguration(), table, columnFamily, KEYS, cleanupFS);
1263   }
1264 
1265   /**
1266    * Creates the specified number of regions in the specified table.
1267    * @param c
1268    * @param table
1269    * @param family
1270    * @param numRegions
1271    * @return
1272    * @throws IOException
1273    */
1274   public int createMultiRegions(final Configuration c, final HTable table,
1275       final byte [] family, int numRegions)
1276   throws IOException {
1277     if (numRegions < 3) throw new IOException("Must create at least 3 regions");
1278     byte [] startKey = Bytes.toBytes("aaaaa");
1279     byte [] endKey = Bytes.toBytes("zzzzz");
1280     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
1281     byte [][] regionStartKeys = new byte[splitKeys.length+1][];
1282     for (int i=0;i<splitKeys.length;i++) {
1283       regionStartKeys[i+1] = splitKeys[i];
1284     }
1285     regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
1286     return createMultiRegions(c, table, family, regionStartKeys);
1287   }
1288 
1289   public int createMultiRegions(final Configuration c, final HTable table,
1290       final byte[] columnFamily, byte [][] startKeys) throws IOException {
1291     return createMultiRegions(c, table, columnFamily, startKeys, true);
1292   }
1293 
1294   public int createMultiRegions(final Configuration c, final HTable table,
1295           final byte[] columnFamily, byte [][] startKeys, boolean cleanupFS)
1296   throws IOException {
1297     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
1298     HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
1299     HTableDescriptor htd = table.getTableDescriptor();
1300     if(!htd.hasFamily(columnFamily)) {
1301       HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
1302       htd.addFamily(hcd);
1303     }
1304     // remove empty region - this is tricky as the mini cluster during the test
1305     // setup already has the "<tablename>,,123456789" row with an empty start
1306     // and end key. Adding the custom regions below adds those blindly,
1307     // including the new start region from empty to "bbb". lg
1308     List<byte[]> rows = getMetaTableRows(htd.getName());
1309     String regionToDeleteInFS = table
1310         .getRegionsInRange(Bytes.toBytes(""), Bytes.toBytes("")).get(0)
1311         .getRegionInfo().getEncodedName();
1312     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
1313     // add custom ones
1314     int count = 0;
1315     for (int i = 0; i < startKeys.length; i++) {
1316       int j = (i + 1) % startKeys.length;
1317       HRegionInfo hri = new HRegionInfo(table.getTableName(),
1318         startKeys[i], startKeys[j]);
1319       Put put = new Put(hri.getRegionName());
1320       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
1321         Writables.getBytes(hri));
1322       meta.put(put);
1323       LOG.info("createMultiRegions: inserted " + hri.toString());
1324       newRegions.add(hri);
1325       count++;
1326     }
1327     // see comment above, remove "old" (or previous) single region
1328     for (byte[] row : rows) {
1329       LOG.info("createMultiRegions: deleting meta row -> " +
1330         Bytes.toStringBinary(row));
1331       meta.delete(new Delete(row));
1332     }
1333     if (cleanupFS) {
1334       // see HBASE-7417 - this confused TestReplication
1335       // remove the "old" region from FS
1336       Path tableDir = new Path(getDefaultRootDirPath().toString()
1337           + System.getProperty("file.separator") + htd.getNameAsString()
1338           + System.getProperty("file.separator") + regionToDeleteInFS);
1339       FileSystem.get(c).delete(tableDir);
1340     }
1341     // flush cache of regions
1342     HConnection conn = table.getConnection();
1343     conn.clearRegionCache();
1344     // assign all the new regions IF table is enabled.
1345     HBaseAdmin admin = getHBaseAdmin();
1346     if (admin.isTableEnabled(table.getTableName())) {
1347       for(HRegionInfo hri : newRegions) {
1348         admin.assign(hri.getRegionName());
1349       }
1350     }
1351 
1352     meta.close();
1353 
1354     return count;
1355   }
1356 
1357   /**
1358    * Create rows in META for regions of the specified table with the specified
1359    * start keys.  The first startKey should be a 0 length byte array if you
1360    * want to form a proper range of regions.
1361    * @param conf
1362    * @param htd
1363    * @param startKeys
1364    * @return list of region info for regions added to meta
1365    * @throws IOException
1366    */
1367   public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
1368       final HTableDescriptor htd, byte [][] startKeys)
1369   throws IOException {
1370     HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
1371     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
1372     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
1373     // add custom ones
1374     for (int i = 0; i < startKeys.length; i++) {
1375       int j = (i + 1) % startKeys.length;
1376       HRegionInfo hri = new HRegionInfo(htd.getName(), startKeys[i],
1377           startKeys[j]);
1378       Put put = new Put(hri.getRegionName());
1379       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
1380         Writables.getBytes(hri));
1381       meta.put(put);
1382       LOG.info("createMultiRegionsInMeta: inserted " + hri.toString());
1383       newRegions.add(hri);
1384     }
1385 
1386     meta.close();
1387     return newRegions;
1388   }
1389 
1390   /**
1391    * Returns all rows from the .META. table.
1392    *
1393    * @throws IOException When reading the rows fails.
1394    */
1395   public List<byte[]> getMetaTableRows() throws IOException {
1396     // TODO: Redo using MetaReader class
1397     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
1398     List<byte[]> rows = new ArrayList<byte[]>();
1399     ResultScanner s = t.getScanner(new Scan());
1400     for (Result result : s) {
1401       LOG.info("getMetaTableRows: row -> " +
1402         Bytes.toStringBinary(result.getRow()));
1403       rows.add(result.getRow());
1404     }
1405     s.close();
1406     t.close();
1407     return rows;
1408   }
1409 
1410   /**
1411    * Returns all rows from the .META. table for a given user table
1412    *
1413    * @throws IOException When reading the rows fails.
1414    */
1415   public List<byte[]> getMetaTableRows(byte[] tableName) throws IOException {
1416     // TODO: Redo using MetaReader.
1417     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
1418     List<byte[]> rows = new ArrayList<byte[]>();
1419     ResultScanner s = t.getScanner(new Scan());
1420     for (Result result : s) {
1421       byte[] val = result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1422       if (val == null) {
1423         LOG.error("No region info for row " + Bytes.toString(result.getRow()));
1424         // TODO figure out what to do for this new hosed case.
1425         continue;
1426       }
1427       HRegionInfo info = Writables.getHRegionInfo(val);
1428       if (Bytes.compareTo(info.getTableName(), tableName) == 0) {
1429         LOG.info("getMetaTableRows: row -> " +
1430             Bytes.toStringBinary(result.getRow()) + info);
1431         rows.add(result.getRow());
1432       }
1433     }
1434     s.close();
1435     t.close();
1436     return rows;
1437   }
1438 
1439   /**
1440    * Tool to get the reference to the region server object that holds the
1441    * region of the specified user table.
1442    * It first searches for the meta rows that contain the region of the
1443    * specified table, then gets the index of that RS, and finally retrieves
1444    * the RS's reference.
1445    * @param tableName user table to lookup in .META.
1446    * @return region server that holds it, null if the row doesn't exist
1447    * @throws IOException
1448    */
1449   public HRegionServer getRSForFirstRegionInTable(byte[] tableName)
1450       throws IOException {
1451     List<byte[]> metaRows = getMetaTableRows(tableName);
1452     if (metaRows == null || metaRows.isEmpty()) {
1453       return null;
1454     }
1455     LOG.debug("Found " + metaRows.size() + " rows for table " +
1456       Bytes.toString(tableName));
1457     byte [] firstrow = metaRows.get(0);
1458     LOG.debug("FirstRow=" + Bytes.toString(firstrow));
1459     int index = getMiniHBaseCluster().getServerWith(firstrow);
1460     return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer();
1461   }
1462 
1463   /**
1464    * Starts a <code>MiniMRCluster</code> with a default number of
1465    * <code>TaskTracker</code>'s.
1466    *
1467    * @throws IOException When starting the cluster fails.
1468    */
1469   public void startMiniMapReduceCluster() throws IOException {
1470     startMiniMapReduceCluster(2);
1471   }
1472 
1473   /**
1474    * Starts a <code>MiniMRCluster</code>.
1475    *
1476    * @param servers  The number of <code>TaskTracker</code>'s to start.
1477    * @throws IOException When starting the cluster fails.
1478    */
1479   public void startMiniMapReduceCluster(final int servers) throws IOException {
1480     LOG.info("Starting mini mapreduce cluster...");
1481     if (dataTestDir == null) {
1482       setupDataTestDir();
1483     }
1484     // These are needed for the new and improved Map/Reduce framework
1485     Configuration c = getConfiguration();
1486     String logDir = c.get("hadoop.log.dir");
1487     String tmpDir = c.get("hadoop.tmp.dir");
1488     if (logDir == null) {
1489       logDir = tmpDir;
1490     }
1491     System.setProperty("hadoop.log.dir", logDir);
1492     c.set("mapred.output.dir", tmpDir);
1493 
1494     // Tests were failing because this process used 6GB of virtual memory and was getting killed.
1495     // we up the VM usable so that processes don't get killed.
1496     conf.setFloat("yarn.nodemanager.vmem-pmem-ratio", 8.0f);
1497 
1498     mrCluster = new MiniMRCluster(0, 0, servers,
1499       FileSystem.get(conf).getUri().toString(), 1, null, null, null, new JobConf(conf));
1500 
1501     JobConf jobConf = MapreduceTestingShim.getJobConf(mrCluster);
1502     if (jobConf == null) {
1503       jobConf = mrCluster.createJobConf();
1504     }
1505     jobConf.set("mapred.local.dir",
1506       conf.get("mapred.local.dir")); //Hadoop MiniMR overwrites this while it should not
1507 
1508     LOG.info("Mini mapreduce cluster started");
1509     JobConf mrClusterJobConf = mrCluster.createJobConf();
1510     c.set("mapred.job.tracker", mrClusterJobConf.get("mapred.job.tracker"));
1511     /* this for mrv2 support */
1512     conf.set("mapreduce.framework.name", "yarn");
1513     conf.setBoolean("yarn.is.minicluster", true);
1514     String rmAdress = mrClusterJobConf.get("yarn.resourcemanager.address");
1515     if (rmAdress != null) {
1516       conf.set("yarn.resourcemanager.address", rmAdress);
1517     }
1518     String historyAddress = jobConf.get("mapreduce.jobhistory.address");
1519     if (historyAddress != null) {
1520       conf.set("mapreduce.jobhistory.address", historyAddress);
1521     }
1522     String schedulerAdress =
1523       mrClusterJobConf.get("yarn.resourcemanager.scheduler.address");
1524     if (schedulerAdress != null) {
1525       conf.set("yarn.resourcemanager.scheduler.address", schedulerAdress);
1526     }
1527   }
1528 
1529   /**
1530    * Stops the previously started <code>MiniMRCluster</code>.
1531    */
1532   public void shutdownMiniMapReduceCluster() {
1533     LOG.info("Stopping mini mapreduce cluster...");
1534     if (mrCluster != null) {
1535       mrCluster.shutdown();
1536       mrCluster = null;
1537     }
1538     // Restore configuration to point to local jobtracker
1539     conf.set("mapred.job.tracker", "local");
1540     LOG.info("Mini mapreduce cluster stopped");
1541   }
1542 
1543   /**
1544    * Switches the logger for the given class to DEBUG level.
1545    *
1546    * @param clazz  The class for which to switch to debug logging.
1547    */
1548   public void enableDebug(Class<?> clazz) {
1549     Log l = LogFactory.getLog(clazz);
1550     if (l instanceof Log4JLogger) {
1551       ((Log4JLogger) l).getLogger().setLevel(org.apache.log4j.Level.DEBUG);
1552     } else if (l instanceof Jdk14Logger) {
1553       ((Jdk14Logger) l).getLogger().setLevel(java.util.logging.Level.ALL);
1554     }
1555   }
1556 
1557   /**
1558    * Expire the Master's session
1559    * @throws Exception
1560    */
1561   public void expireMasterSession() throws Exception {
1562     HMaster master = getMiniHBaseCluster().getMaster();
1563     expireSession(master.getZooKeeper(), false);
1564   }
1565 
1566   /**
1567    * Expire a region server's session
1568    * @param index which RS
1569    * @throws Exception
1570    */
1571   public void expireRegionServerSession(int index) throws Exception {
1572     HRegionServer rs = getMiniHBaseCluster().getRegionServer(index);
1573     expireSession(rs.getZooKeeper(), false);
1574     decrementMinRegionServerCount();
1575   }
1576 
1577   private void decrementMinRegionServerCount() {
1578     // decrement the count for this.conf, for newly spwaned master
1579     // this.hbaseCluster shares this configuration too
1580     decrementMinRegionServerCount(getConfiguration());
1581 
1582     // each master thread keeps a copy of configuration
1583     for (MasterThread master : getHBaseCluster().getMasterThreads()) {
1584       decrementMinRegionServerCount(master.getMaster().getConfiguration());
1585     }
1586   }
1587 
1588   private void decrementMinRegionServerCount(Configuration conf) {
1589     int currentCount = conf.getInt(
1590         ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
1591     if (currentCount != -1) {
1592       conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
1593           Math.max(currentCount - 1, 1));
1594     }
1595   }
1596 
1597    /**
1598     * Expire a ZooKeeper session as recommended in ZooKeeper documentation
1599     * http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A4
1600     * There are issues when doing this:
1601     * [1] http://www.mail-archive.com/dev@zookeeper.apache.org/msg01942.html
1602     * [2] https://issues.apache.org/jira/browse/ZOOKEEPER-1105
1603     *
1604     * @param nodeZK - the ZK to make expiry
1605     * @param checkStatus - true to check if the we can create a HTable with the
1606     *                    current configuration.
1607     */
1608   public void expireSession(ZooKeeperWatcher nodeZK, boolean checkStatus)
1609     throws Exception {
1610     Configuration c = new Configuration(this.conf);
1611     String quorumServers = ZKConfig.getZKQuorumServersString(c);
1612     int sessionTimeout = 500;
1613     ZooKeeper zk = nodeZK.getRecoverableZooKeeper().getZooKeeper();
1614     byte[] password = zk.getSessionPasswd();
1615     long sessionID = zk.getSessionId();
1616 
1617     // Expiry seems to be asynchronous (see comment from P. Hunt in [1]),
1618     //  so we create a first watcher to be sure that the
1619     //  event was sent. We expect that if our watcher receives the event
1620     //  other watchers on the same machine will get is as well.
1621     // When we ask to close the connection, ZK does not close it before
1622     //  we receive all the events, so don't have to capture the event, just
1623     //  closing the connection should be enough.
1624     ZooKeeper monitor = new ZooKeeper(quorumServers,
1625       1000, new org.apache.zookeeper.Watcher(){
1626       @Override
1627       public void process(WatchedEvent watchedEvent) {
1628         LOG.info("Monitor ZKW received event="+watchedEvent);
1629       }
1630     } , sessionID, password);
1631 
1632     // Making it expire
1633     ZooKeeper newZK = new ZooKeeper(quorumServers,
1634         sessionTimeout, EmptyWatcher.instance, sessionID, password);
1635     newZK.close();
1636     LOG.info("ZK Closed Session 0x" + Long.toHexString(sessionID));
1637 
1638      // Now closing & waiting to be sure that the clients get it.
1639      monitor.close();
1640 
1641     if (checkStatus) {
1642       new HTable(new Configuration(conf), HConstants.META_TABLE_NAME).close();
1643     }
1644   }
1645 
1646   /**
1647    * Get the Mini HBase cluster.
1648    *
1649    * @return hbase cluster
1650    * @see #getHBaseClusterInterface()
1651    */
1652   public MiniHBaseCluster getHBaseCluster() {
1653     return getMiniHBaseCluster();
1654   }
1655 
1656   /**
1657    * Returns the HBaseCluster instance.
1658    * <p>Returned object can be any of the subclasses of HBaseCluster, and the
1659    * tests referring this should not assume that the cluster is a mini cluster or a
1660    * distributed one. If the test only works on a mini cluster, then specific
1661    * method {@link #getMiniHBaseCluster()} can be used instead w/o the
1662    * need to type-cast.
1663    */
1664   public HBaseCluster getHBaseClusterInterface() {
1665     //implementation note: we should rename this method as #getHBaseCluster(),
1666     //but this would require refactoring 90+ calls.
1667     return hbaseCluster;
1668   }
1669 
1670   /**
1671    * Returns a HBaseAdmin instance.
1672    * This instance is shared between HBaseTestingUtility intance users.
1673    * Don't close it, it will be closed automatically when the
1674    * cluster shutdowns
1675    *
1676    * @return The HBaseAdmin instance.
1677    * @throws IOException
1678    */
1679   public synchronized HBaseAdmin getHBaseAdmin()
1680   throws IOException {
1681     if (hbaseAdmin == null){
1682       hbaseAdmin = new HBaseAdmin(new Configuration(getConfiguration()));
1683     }
1684     return hbaseAdmin;
1685   }
1686   private HBaseAdmin hbaseAdmin = null;
1687 
1688   /**
1689    * Closes the named region.
1690    *
1691    * @param regionName  The region to close.
1692    * @throws IOException
1693    */
1694   public void closeRegion(String regionName) throws IOException {
1695     closeRegion(Bytes.toBytes(regionName));
1696   }
1697 
1698   /**
1699    * Closes the named region.
1700    *
1701    * @param regionName  The region to close.
1702    * @throws IOException
1703    */
1704   public void closeRegion(byte[] regionName) throws IOException {
1705     getHBaseAdmin().closeRegion(regionName, null);
1706   }
1707 
1708   /**
1709    * Closes the region containing the given row.
1710    *
1711    * @param row  The row to find the containing region.
1712    * @param table  The table to find the region.
1713    * @throws IOException
1714    */
1715   public void closeRegionByRow(String row, HTable table) throws IOException {
1716     closeRegionByRow(Bytes.toBytes(row), table);
1717   }
1718 
1719   /**
1720    * Closes the region containing the given row.
1721    *
1722    * @param row  The row to find the containing region.
1723    * @param table  The table to find the region.
1724    * @throws IOException
1725    */
1726   public void closeRegionByRow(byte[] row, HTable table) throws IOException {
1727     HRegionLocation hrl = table.getRegionLocation(row);
1728     closeRegion(hrl.getRegionInfo().getRegionName());
1729   }
1730 
1731   public MiniZooKeeperCluster getZkCluster() {
1732     return zkCluster;
1733   }
1734 
1735   public void setZkCluster(MiniZooKeeperCluster zkCluster) {
1736     this.passedZkCluster = true;
1737     this.zkCluster = zkCluster;
1738     conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkCluster.getClientPort());
1739   }
1740 
1741   public MiniDFSCluster getDFSCluster() {
1742     return dfsCluster;
1743   }
1744 
1745   public void setDFSCluster(MiniDFSCluster cluster) throws IOException {
1746     if (dfsCluster != null && dfsCluster.isClusterUp()) {
1747       throw new IOException("DFSCluster is already running! Shut it down first.");
1748     }
1749     this.dfsCluster = cluster;
1750   }
1751 
1752   public FileSystem getTestFileSystem() throws IOException {
1753     return HFileSystem.get(conf);
1754   }
1755 
1756   /**
1757    * @return True if we removed the test dir
1758    * @throws IOException
1759    */
1760   public boolean cleanupTestDir() throws IOException {
1761     if (dataTestDir == null ){
1762       return false;
1763     } else {
1764       boolean ret = deleteDir(getDataTestDir());
1765       dataTestDir = null;
1766       return ret;
1767     }
1768   }
1769 
1770   /**
1771    * @param subdir Test subdir name.
1772    * @return True if we removed the test dir
1773    * @throws IOException
1774    */
1775   public boolean cleanupTestDir(final String subdir) throws IOException {
1776     if (dataTestDir == null){
1777       return false;
1778     }
1779     return deleteDir(getDataTestDir(subdir));
1780   }
1781 
1782   /**
1783    * @param dir Directory to delete
1784    * @return True if we deleted it.
1785    * @throws IOException
1786    */
1787   public boolean deleteDir(final Path dir) throws IOException {
1788     FileSystem fs = getTestFileSystem();
1789     if (fs.exists(dir)) {
1790       return fs.delete(getDataTestDir(), true);
1791     }
1792     return false;
1793   }
1794 
1795   public void waitTableAvailable(byte[] table, long timeoutMillis)
1796   throws InterruptedException, IOException {
1797     long startWait = System.currentTimeMillis();
1798     while (!getHBaseAdmin().isTableAvailable(table)) {
1799       assertTrue("Timed out waiting for table to become available " +
1800         Bytes.toStringBinary(table),
1801         System.currentTimeMillis() - startWait < timeoutMillis);
1802       Thread.sleep(200);
1803     }
1804   }
1805 
1806   public void waitTableEnabled(byte[] table, long timeoutMillis)
1807   throws InterruptedException, IOException {
1808     long startWait = System.currentTimeMillis();
1809     while (!getHBaseAdmin().isTableAvailable(table) &&
1810            !getHBaseAdmin().isTableEnabled(table)) {
1811       assertTrue("Timed out waiting for table to become available and enabled " +
1812          Bytes.toStringBinary(table),
1813          System.currentTimeMillis() - startWait < timeoutMillis);
1814       Thread.sleep(200);
1815     }
1816   }
1817 
1818   /**
1819    * Make sure that at least the specified number of region servers
1820    * are running
1821    * @param num minimum number of region servers that should be running
1822    * @return true if we started some servers
1823    * @throws IOException
1824    */
1825   public boolean ensureSomeRegionServersAvailable(final int num)
1826       throws IOException {
1827     boolean startedServer = false;
1828     MiniHBaseCluster hbaseCluster = getMiniHBaseCluster();
1829     for (int i=hbaseCluster.getLiveRegionServerThreads().size(); i<num; ++i) {
1830       LOG.info("Started new server=" + hbaseCluster.startRegionServer());
1831       startedServer = true;
1832     }
1833 
1834     return startedServer;
1835   }
1836 
1837   /**
1838    * Make sure that at least the specified number of region servers
1839    * are running. We don't count the ones that are currently stopping or are
1840    * stopped.
1841    * @param num minimum number of region servers that should be running
1842    * @return true if we started some servers
1843    * @throws IOException
1844    */
1845   public boolean ensureSomeNonStoppedRegionServersAvailable(final int num)
1846     throws IOException {
1847     boolean startedServer = ensureSomeRegionServersAvailable(num);
1848 
1849     int nonStoppedServers = 0;
1850     for (JVMClusterUtil.RegionServerThread rst :
1851       getMiniHBaseCluster().getRegionServerThreads()) {
1852 
1853       HRegionServer hrs = rst.getRegionServer();
1854       if (hrs.isStopping() || hrs.isStopped()) {
1855         LOG.info("A region server is stopped or stopping:"+hrs);
1856       } else {
1857         nonStoppedServers++;
1858       }
1859     }
1860     for (int i=nonStoppedServers; i<num; ++i) {
1861       LOG.info("Started new server=" + getMiniHBaseCluster().startRegionServer());
1862       startedServer = true;
1863     }
1864     return startedServer;
1865   }
1866 
1867 
1868   /**
1869    * This method clones the passed <code>c</code> configuration setting a new
1870    * user into the clone.  Use it getting new instances of FileSystem.  Only
1871    * works for DistributedFileSystem.
1872    * @param c Initial configuration
1873    * @param differentiatingSuffix Suffix to differentiate this user from others.
1874    * @return A new configuration instance with a different user set into it.
1875    * @throws IOException
1876    */
1877   public static User getDifferentUser(final Configuration c,
1878     final String differentiatingSuffix)
1879   throws IOException {
1880     FileSystem currentfs = FileSystem.get(c);
1881     if (!(currentfs instanceof DistributedFileSystem)) {
1882       return User.getCurrent();
1883     }
1884     // Else distributed filesystem.  Make a new instance per daemon.  Below
1885     // code is taken from the AppendTestUtil over in hdfs.
1886     String username = User.getCurrent().getName() +
1887       differentiatingSuffix;
1888     User user = User.createUserForTesting(c, username,
1889         new String[]{"supergroup"});
1890     return user;
1891   }
1892 
1893   /**
1894    * Set maxRecoveryErrorCount in DFSClient.  In 0.20 pre-append its hard-coded to 5 and
1895    * makes tests linger.  Here is the exception you'll see:
1896    * <pre>
1897    * 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...
1898    * </pre>
1899    * @param stream A DFSClient.DFSOutputStream.
1900    * @param max
1901    * @throws NoSuchFieldException
1902    * @throws SecurityException
1903    * @throws IllegalAccessException
1904    * @throws IllegalArgumentException
1905    */
1906   public static void setMaxRecoveryErrorCount(final OutputStream stream,
1907       final int max) {
1908     try {
1909       Class<?> [] clazzes = DFSClient.class.getDeclaredClasses();
1910       for (Class<?> clazz: clazzes) {
1911         String className = clazz.getSimpleName();
1912         if (className.equals("DFSOutputStream")) {
1913           if (clazz.isInstance(stream)) {
1914             Field maxRecoveryErrorCountField =
1915               stream.getClass().getDeclaredField("maxRecoveryErrorCount");
1916             maxRecoveryErrorCountField.setAccessible(true);
1917             maxRecoveryErrorCountField.setInt(stream, max);
1918             break;
1919           }
1920         }
1921       }
1922     } catch (Exception e) {
1923       LOG.info("Could not set max recovery field", e);
1924     }
1925   }
1926 
1927   /**
1928    * Wait until all regions for a table in .META. have a non-empty
1929    * info:server, up to 60 seconds.  This means all regions have been deployed,
1930    * master has been informed and updated .META. with the regions deployed
1931    * server.
1932    * @param tableName the table name
1933    * @throws IOException
1934    */
1935   public void waitUntilAllRegionsAssigned(final byte[] tableName) throws IOException {
1936     waitUntilAllRegionsAssigned(tableName, 60000);
1937   }
1938 
1939   /**
1940    * Wait until all regions for a table in .META. have a non-empty
1941    * info:server, or until timeout.  This means all regions have been
1942    * deployed, master has been informed and updated .META. with the regions
1943    * deployed server.
1944    * @param tableName the table name
1945    * @param timeout timeout, in milliseconds
1946    * @throws IOException
1947    */
1948   public void waitUntilAllRegionsAssigned(final byte[] tableName, final long timeout)
1949       throws IOException {
1950     long deadline = System.currentTimeMillis() + timeout;
1951     HTable meta = new HTable(getConfiguration(), HConstants.META_TABLE_NAME);
1952     try {
1953       while (true) {
1954         boolean allRegionsAssigned = true;
1955         Scan scan = new Scan();
1956         scan.addFamily(HConstants.CATALOG_FAMILY);
1957         ResultScanner s = meta.getScanner(scan);
1958         try {
1959           Result r;
1960           while ((r = s.next()) != null) {
1961             byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1962             HRegionInfo info = Writables.getHRegionInfoOrNull(b);
1963             if (info != null && Bytes.equals(info.getTableName(), tableName)) {
1964               b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
1965               allRegionsAssigned &= (b != null);
1966             }
1967           }
1968         } finally {
1969           s.close();
1970         }
1971         if (allRegionsAssigned) {
1972           return;
1973         }
1974         long now = System.currentTimeMillis();
1975         if (now > deadline) {
1976           throw new IOException("Timeout waiting for all regions of " +
1977             Bytes.toStringBinary(tableName) + " to be assigned");
1978         }
1979         try {
1980           Thread.sleep(deadline - now < 200 ? deadline - now : 200);
1981         } catch (InterruptedException e) {
1982           throw new IOException(e);
1983         }
1984       }
1985     } finally {
1986       meta.close();
1987     }
1988   }
1989 
1990   /**
1991    * Do a small get/scan against one store. This is required because store
1992    * has no actual methods of querying itself, and relies on StoreScanner.
1993    */
1994   public static List<KeyValue> getFromStoreFile(Store store,
1995                                                 Get get) throws IOException {
1996     MultiVersionConsistencyControl.resetThreadReadPoint();
1997     Scan scan = new Scan(get);
1998     InternalScanner scanner = (InternalScanner) store.getScanner(scan,
1999         scan.getFamilyMap().get(store.getFamily().getName()));
2000 
2001     List<KeyValue> result = new ArrayList<KeyValue>();
2002     scanner.next(result);
2003     if (!result.isEmpty()) {
2004       // verify that we are on the row we want:
2005       KeyValue kv = result.get(0);
2006       if (!Bytes.equals(kv.getRow(), get.getRow())) {
2007         result.clear();
2008       }
2009     }
2010     scanner.close();
2011     return result;
2012   }
2013 
2014   /**
2015    * Do a small get/scan against one store. This is required because store
2016    * has no actual methods of querying itself, and relies on StoreScanner.
2017    */
2018   public static List<KeyValue> getFromStoreFile(Store store,
2019                                                 byte [] row,
2020                                                 NavigableSet<byte[]> columns
2021                                                 ) throws IOException {
2022     Get get = new Get(row);
2023     Map<byte[], NavigableSet<byte[]>> s = get.getFamilyMap();
2024     s.put(store.getFamily().getName(), columns);
2025 
2026     return getFromStoreFile(store,get);
2027   }
2028 
2029   /**
2030    * Gets a ZooKeeperWatcher.
2031    * @param TEST_UTIL
2032    */
2033   public static ZooKeeperWatcher getZooKeeperWatcher(
2034       HBaseTestingUtility TEST_UTIL) throws ZooKeeperConnectionException,
2035       IOException {
2036     ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
2037         "unittest", new Abortable() {
2038           boolean aborted = false;
2039 
2040           @Override
2041           public void abort(String why, Throwable e) {
2042             aborted = true;
2043             throw new RuntimeException("Fatal ZK error, why=" + why, e);
2044           }
2045 
2046           @Override
2047           public boolean isAborted() {
2048             return aborted;
2049           }
2050         });
2051     return zkw;
2052   }
2053 
2054   /**
2055    * Creates a znode with OPENED state.
2056    * @param TEST_UTIL
2057    * @param region
2058    * @param serverName
2059    * @return
2060    * @throws IOException
2061    * @throws ZooKeeperConnectionException
2062    * @throws KeeperException
2063    * @throws NodeExistsException
2064    */
2065   public static ZooKeeperWatcher createAndForceNodeToOpenedState(
2066       HBaseTestingUtility TEST_UTIL, HRegion region,
2067       ServerName serverName) throws ZooKeeperConnectionException,
2068       IOException, KeeperException, NodeExistsException {
2069     ZooKeeperWatcher zkw = getZooKeeperWatcher(TEST_UTIL);
2070     ZKAssign.createNodeOffline(zkw, region.getRegionInfo(), serverName);
2071     int version = ZKAssign.transitionNodeOpening(zkw, region
2072         .getRegionInfo(), serverName);
2073     ZKAssign.transitionNodeOpened(zkw, region.getRegionInfo(), serverName,
2074         version);
2075     return zkw;
2076   }
2077 
2078   public static void assertKVListsEqual(String additionalMsg,
2079       final List<KeyValue> expected,
2080       final List<KeyValue> actual) {
2081     final int eLen = expected.size();
2082     final int aLen = actual.size();
2083     final int minLen = Math.min(eLen, aLen);
2084 
2085     int i;
2086     for (i = 0; i < minLen
2087         && KeyValue.COMPARATOR.compare(expected.get(i), actual.get(i)) == 0;
2088         ++i) {}
2089 
2090     if (additionalMsg == null) {
2091       additionalMsg = "";
2092     }
2093     if (!additionalMsg.isEmpty()) {
2094       additionalMsg = ". " + additionalMsg;
2095     }
2096 
2097     if (eLen != aLen || i != minLen) {
2098       throw new AssertionError(
2099           "Expected and actual KV arrays differ at position " + i + ": " +
2100           safeGetAsStr(expected, i) + " (length " + eLen +") vs. " +
2101           safeGetAsStr(actual, i) + " (length " + aLen + ")" + additionalMsg);
2102     }
2103   }
2104 
2105   private static <T> String safeGetAsStr(List<T> lst, int i) {
2106     if (0 <= i && i < lst.size()) {
2107       return lst.get(i).toString();
2108     } else {
2109       return "<out_of_range>";
2110     }
2111   }
2112 
2113   public String getClusterKey() {
2114     return conf.get(HConstants.ZOOKEEPER_QUORUM) + ":"
2115         + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":"
2116         + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
2117             HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
2118   }
2119 
2120   /** Creates a random table with the given parameters */
2121   public HTable createRandomTable(String tableName,
2122       final Collection<String> families,
2123       final int maxVersions,
2124       final int numColsPerRow,
2125       final int numFlushes,
2126       final int numRegions,
2127       final int numRowsPerFlush)
2128       throws IOException, InterruptedException {
2129 
2130     LOG.info("\n\nCreating random table " + tableName + " with " + numRegions +
2131         " regions, " + numFlushes + " storefiles per region, " +
2132         numRowsPerFlush + " rows per flush, maxVersions=" +  maxVersions +
2133         "\n");
2134 
2135     final Random rand = new Random(tableName.hashCode() * 17L + 12938197137L);
2136     final int numCF = families.size();
2137     final byte[][] cfBytes = new byte[numCF][];
2138     final byte[] tableNameBytes = Bytes.toBytes(tableName);
2139 
2140     {
2141       int cfIndex = 0;
2142       for (String cf : families) {
2143         cfBytes[cfIndex++] = Bytes.toBytes(cf);
2144       }
2145     }
2146 
2147     final int actualStartKey = 0;
2148     final int actualEndKey = Integer.MAX_VALUE;
2149     final int keysPerRegion = (actualEndKey - actualStartKey) / numRegions;
2150     final int splitStartKey = actualStartKey + keysPerRegion;
2151     final int splitEndKey = actualEndKey - keysPerRegion;
2152     final String keyFormat = "%08x";
2153     final HTable table = createTable(tableNameBytes, cfBytes,
2154         maxVersions,
2155         Bytes.toBytes(String.format(keyFormat, splitStartKey)),
2156         Bytes.toBytes(String.format(keyFormat, splitEndKey)),
2157         numRegions);
2158     if (hbaseCluster != null) {
2159       getMiniHBaseCluster().flushcache(HConstants.META_TABLE_NAME);
2160     }
2161 
2162     for (int iFlush = 0; iFlush < numFlushes; ++iFlush) {
2163       for (int iRow = 0; iRow < numRowsPerFlush; ++iRow) {
2164         final byte[] row = Bytes.toBytes(String.format(keyFormat,
2165             actualStartKey + rand.nextInt(actualEndKey - actualStartKey)));
2166 
2167         Put put = new Put(row);
2168         Delete del = new Delete(row);
2169         for (int iCol = 0; iCol < numColsPerRow; ++iCol) {
2170           final byte[] cf = cfBytes[rand.nextInt(numCF)];
2171           final long ts = rand.nextInt();
2172           final byte[] qual = Bytes.toBytes("col" + iCol);
2173           if (rand.nextBoolean()) {
2174             final byte[] value = Bytes.toBytes("value_for_row_" + iRow +
2175                 "_cf_" + Bytes.toStringBinary(cf) + "_col_" + iCol + "_ts_" +
2176                 ts + "_random_" + rand.nextLong());
2177             put.add(cf, qual, ts, value);
2178           } else if (rand.nextDouble() < 0.8) {
2179             del.deleteColumn(cf, qual, ts);
2180           } else {
2181             del.deleteColumns(cf, qual, ts);
2182           }
2183         }
2184 
2185         if (!put.isEmpty()) {
2186           table.put(put);
2187         }
2188 
2189         if (!del.isEmpty()) {
2190           table.delete(del);
2191         }
2192       }
2193       LOG.info("Initiating flush #" + iFlush + " for table " + tableName);
2194       table.flushCommits();
2195       if (hbaseCluster != null) {
2196         getMiniHBaseCluster().flushcache(tableNameBytes);
2197       }
2198     }
2199 
2200     return table;
2201   }
2202 
2203   private static final int MIN_RANDOM_PORT = 0xc000;
2204   private static final int MAX_RANDOM_PORT = 0xfffe;
2205 
2206   /**
2207    * Returns a random port. These ports cannot be registered with IANA and are
2208    * intended for dynamic allocation (see http://bit.ly/dynports).
2209    */
2210   public static int randomPort() {
2211     return MIN_RANDOM_PORT
2212         + new Random().nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT);
2213   }
2214 
2215   public static int randomFreePort() {
2216     int port = 0;
2217     do {
2218       port = randomPort();
2219       try {
2220         ServerSocket sock = new ServerSocket(port);
2221         sock.close();
2222       } catch (IOException ex) {
2223         port = 0;
2224       }
2225     } while (port == 0);
2226     return port;
2227   }
2228 
2229   public static void waitForHostPort(String host, int port)
2230       throws IOException {
2231     final int maxTimeMs = 10000;
2232     final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
2233     IOException savedException = null;
2234     LOG.info("Waiting for server at " + host + ":" + port);
2235     for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
2236       try {
2237         Socket sock = new Socket(InetAddress.getByName(host), port);
2238         sock.close();
2239         savedException = null;
2240         LOG.info("Server at " + host + ":" + port + " is available");
2241         break;
2242       } catch (UnknownHostException e) {
2243         throw new IOException("Failed to look up " + host, e);
2244       } catch (IOException e) {
2245         savedException = e;
2246       }
2247       Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
2248     }
2249 
2250     if (savedException != null) {
2251       throw savedException;
2252     }
2253   }
2254 
2255   /**
2256    * Creates a pre-split table for load testing. If the table already exists,
2257    * logs a warning and continues.
2258    * @return the number of regions the table was split into
2259    */
2260   public static int createPreSplitLoadTestTable(Configuration conf,
2261       byte[] tableName, byte[] columnFamily, Algorithm compression,
2262       DataBlockEncoding dataBlockEncoding) throws IOException {
2263     HTableDescriptor desc = new HTableDescriptor(tableName);
2264     HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
2265     hcd.setDataBlockEncoding(dataBlockEncoding);
2266     hcd.setCompressionType(compression);
2267     return createPreSplitLoadTestTable(conf, desc, hcd);
2268   }
2269 
2270   /**
2271    * Creates a pre-split table for load testing. If the table already exists,
2272    * logs a warning and continues.
2273    * @return the number of regions the table was split into
2274    */
2275   public static int createPreSplitLoadTestTable(Configuration conf,
2276       HTableDescriptor desc, HColumnDescriptor hcd) throws IOException {
2277     if (!desc.hasFamily(hcd.getName())) {
2278       desc.addFamily(hcd);
2279     }
2280 
2281     int totalNumberOfRegions = 0;
2282     HBaseAdmin admin = new HBaseAdmin(conf);
2283     try {
2284       // create a table a pre-splits regions.
2285       // The number of splits is set as:
2286       //    region servers * regions per region server).
2287       int numberOfServers = admin.getClusterStatus().getServers().size();
2288       if (numberOfServers == 0) {
2289         throw new IllegalStateException("No live regionservers");
2290       }
2291 
2292       totalNumberOfRegions = numberOfServers * DEFAULT_REGIONS_PER_SERVER;
2293       LOG.info("Number of live regionservers: " + numberOfServers + ", " +
2294           "pre-splitting table into " + totalNumberOfRegions + " regions " +
2295           "(default regions per server: " + DEFAULT_REGIONS_PER_SERVER + ")");
2296 
2297       byte[][] splits = new RegionSplitter.HexStringSplit().split(
2298           totalNumberOfRegions);
2299 
2300       admin.createTable(desc, splits);
2301       admin.close();
2302     } catch (MasterNotRunningException e) {
2303       LOG.error("Master not running", e);
2304       throw new IOException(e);
2305     } catch (TableExistsException e) {
2306       LOG.warn("Table " + Bytes.toStringBinary(desc.getName()) +
2307           " already exists, continuing");
2308     } finally {
2309       admin.close();
2310     }
2311     return totalNumberOfRegions;
2312   }
2313 
2314   public static int getMetaRSPort(Configuration conf) throws IOException {
2315     HTable table = new HTable(conf, HConstants.META_TABLE_NAME);
2316     HRegionLocation hloc = table.getRegionLocation(Bytes.toBytes(""));
2317     table.close();
2318     return hloc.getPort();
2319   }
2320 
2321   public HRegion createTestRegion(String tableName, HColumnDescriptor hcd)
2322       throws IOException {
2323     HTableDescriptor htd = new HTableDescriptor(tableName);
2324     htd.addFamily(hcd);
2325     HRegionInfo info =
2326         new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
2327     HRegion region =
2328         HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), htd);
2329     return region;
2330   }
2331 
2332   /**
2333    * Create region split keys between startkey and endKey
2334    *
2335    * @param startKey
2336    * @param endKey
2337    * @param numRegions the number of regions to be created. it has to be greater than 3.
2338    * @return
2339    */
2340   public byte[][] getRegionSplitStartKeys(byte[] startKey, byte[] endKey, int numRegions){
2341     assertTrue(numRegions>3);
2342     byte [][] tmpSplitKeys = Bytes.split(startKey, endKey, numRegions - 3);
2343     byte [][] result = new byte[tmpSplitKeys.length+1][];
2344     for (int i=0;i<tmpSplitKeys.length;i++) {
2345       result[i+1] = tmpSplitKeys[i];
2346     }
2347     result[0] = HConstants.EMPTY_BYTE_ARRAY;
2348     return result;
2349   }
2350 
2351   /**
2352    * Create a set of column descriptors with the combination of compression,
2353    * encoding, bloom codecs available.
2354    * @return the list of column descriptors
2355    */
2356   public static List<HColumnDescriptor> generateColumnDescriptors() {
2357     return generateColumnDescriptors("");
2358   }
2359 
2360   /**
2361    * Create a set of column descriptors with the combination of compression,
2362    * encoding, bloom codecs available.
2363    * @param prefix family names prefix
2364    * @return the list of column descriptors
2365    */
2366   public static List<HColumnDescriptor> generateColumnDescriptors(final String prefix) {
2367     List<HColumnDescriptor> htds = new ArrayList<HColumnDescriptor>();
2368     long familyId = 0;
2369     for (Compression.Algorithm compressionType: getSupportedCompressionAlgorithms()) {
2370       for (DataBlockEncoding encodingType: DataBlockEncoding.values()) {
2371         for (StoreFile.BloomType bloomType: StoreFile.BloomType.values()) {
2372           String name = String.format("%s-cf-!@#&-%d!@#", prefix, familyId);
2373           HColumnDescriptor htd = new HColumnDescriptor(name);
2374           htd.setCompressionType(compressionType);
2375           htd.setDataBlockEncoding(encodingType);
2376           htd.setBloomFilterType(bloomType);
2377           htds.add(htd);
2378           familyId++;
2379         }
2380       }
2381     }
2382     return htds;
2383   }
2384 
2385   /**
2386    * Get supported compression algorithms.
2387    * @return supported compression algorithms.
2388    */
2389   public static Compression.Algorithm[] getSupportedCompressionAlgorithms() {
2390     String[] allAlgos = HFile.getSupportedCompressionAlgorithms();
2391     List<Compression.Algorithm> supportedAlgos = new ArrayList<Compression.Algorithm>();
2392     for (String algoName : allAlgos) {
2393       try {
2394         Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName);
2395         algo.getCompressor();
2396         supportedAlgos.add(algo);
2397       } catch (Throwable t) {
2398         // this algo is not available
2399       }
2400     }
2401     return supportedAlgos.toArray(new Compression.Algorithm[0]);
2402   }
2403 }