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