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