View Javadoc

1   
2   
3   /**
4    *
5    * Licensed to the Apache Software Foundation (ASF) under one
6    * or more contributor license agreements.  See the NOTICE file
7    * distributed with this work for additional information
8    * regarding copyright ownership.  The ASF licenses this file
9    * to you under the Apache License, Version 2.0 (the
10   * "License"); you may not use this file except in compliance
11   * with the License.  You may obtain a copy of the License at
12   *
13   *     http://www.apache.org/licenses/LICENSE-2.0
14   *
15   * Unless required by applicable law or agreed to in writing, software
16   * distributed under the License is distributed on an "AS IS" BASIS,
17   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
18   * See the License for the specific language governing permissions and
19   * limitations under the License.
20   */
21  package org.apache.hadoop.hbase;
22  
23  import static org.junit.Assert.assertTrue;
24  import static org.junit.Assert.fail;
25  
26  import java.io.File;
27  import java.io.IOException;
28  import java.io.OutputStream;
29  import java.lang.reflect.Field;
30  import java.lang.reflect.Modifier;
31  import java.net.InetAddress;
32  import java.net.ServerSocket;
33  import java.net.Socket;
34  import java.net.UnknownHostException;
35  import java.security.MessageDigest;
36  import java.util.ArrayList;
37  import java.util.Arrays;
38  import java.util.Collection;
39  import java.util.Collections;
40  import java.util.HashSet;
41  import java.util.List;
42  import java.util.Map;
43  import java.util.NavigableSet;
44  import java.util.Random;
45  import java.util.Set;
46  import java.util.UUID;
47  import java.util.concurrent.TimeUnit;
48  
49  import org.apache.commons.logging.Log;
50  import org.apache.commons.logging.LogFactory;
51  import org.apache.commons.logging.impl.Jdk14Logger;
52  import org.apache.commons.logging.impl.Log4JLogger;
53  import org.apache.hadoop.classification.InterfaceAudience;
54  import org.apache.hadoop.classification.InterfaceStability;
55  import org.apache.hadoop.conf.Configuration;
56  import org.apache.hadoop.fs.FileSystem;
57  import org.apache.hadoop.fs.Path;
58  import org.apache.hadoop.hbase.Waiter.Predicate;
59  import org.apache.hadoop.hbase.catalog.MetaEditor;
60  import org.apache.hadoop.hbase.client.Delete;
61  import org.apache.hadoop.hbase.client.Durability;
62  import org.apache.hadoop.hbase.client.Get;
63  import org.apache.hadoop.hbase.client.HBaseAdmin;
64  import org.apache.hadoop.hbase.client.HConnection;
65  import org.apache.hadoop.hbase.client.HTable;
66  import org.apache.hadoop.hbase.client.Put;
67  import org.apache.hadoop.hbase.client.Result;
68  import org.apache.hadoop.hbase.client.ResultScanner;
69  import org.apache.hadoop.hbase.client.Scan;
70  import org.apache.hadoop.hbase.fs.HFileSystem;
71  import org.apache.hadoop.hbase.io.compress.Compression;
72  import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
73  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
74  import org.apache.hadoop.hbase.io.hfile.ChecksumUtil;
75  import org.apache.hadoop.hbase.mapreduce.MapreduceTestingShim;
76  import org.apache.hadoop.hbase.master.HMaster;
77  import org.apache.hadoop.hbase.master.RegionStates;
78  import org.apache.hadoop.hbase.master.ServerManager;
79  import org.apache.hadoop.hbase.regionserver.BloomType;
80  import org.apache.hadoop.hbase.regionserver.HRegion;
81  import org.apache.hadoop.hbase.regionserver.HRegionServer;
82  import org.apache.hadoop.hbase.regionserver.HStore;
83  import org.apache.hadoop.hbase.regionserver.InternalScanner;
84  import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
85  import org.apache.hadoop.hbase.security.User;
86  import org.apache.hadoop.hbase.util.Bytes;
87  import org.apache.hadoop.hbase.util.FSUtils;
88  import org.apache.hadoop.hbase.util.JVMClusterUtil;
89  import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
90  import org.apache.hadoop.hbase.util.RegionSplitter;
91  import org.apache.hadoop.hbase.util.RetryCounter;
92  import org.apache.hadoop.hbase.util.Threads;
93  import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
94  import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
95  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
96  import org.apache.hadoop.hbase.zookeeper.ZKConfig;
97  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
98  import org.apache.hadoop.hbase.tool.Canary;
99  import org.apache.hadoop.hdfs.DFSClient;
100 import org.apache.hadoop.hdfs.DistributedFileSystem;
101 import org.apache.hadoop.hdfs.MiniDFSCluster;
102 import org.apache.hadoop.mapred.JobConf;
103 import org.apache.hadoop.mapred.MiniMRCluster;
104 import org.apache.hadoop.mapred.TaskLog;
105 import org.apache.zookeeper.KeeperException;
106 import org.apache.zookeeper.KeeperException.NodeExistsException;
107 import org.apache.zookeeper.WatchedEvent;
108 import org.apache.zookeeper.ZooKeeper;
109 import org.apache.zookeeper.ZooKeeper.States;
110 
111 /**
112  * Facility for testing HBase. Replacement for
113  * old HBaseTestCase and HBaseClusterTestCase functionality.
114  * Create an instance and keep it around testing HBase.  This class is
115  * meant to be your one-stop shop for anything you might need testing.  Manages
116  * one cluster at a time only. Managed cluster can be an in-process
117  * {@link MiniHBaseCluster}, or a deployed cluster of type {@link DistributedHBaseCluster}.
118  * Not all methods work with the real cluster.
119  * Depends on log4j being on classpath and
120  * hbase-site.xml for logging and test-run configuration.  It does not set
121  * logging levels nor make changes to configuration parameters.
122  */
123 @InterfaceAudience.Public
124 @InterfaceStability.Evolving
125 public class HBaseTestingUtility extends HBaseCommonTestingUtility {
126    private Configuration conf;
127    private MiniZooKeeperCluster zkCluster = null;
128 
129   /**
130    * The default number of regions per regionserver when creating a pre-split
131    * table.
132    */
133   private static int DEFAULT_REGIONS_PER_SERVER = 5;
134 
135   /**
136    * Set if we were passed a zkCluster.  If so, we won't shutdown zk as
137    * part of general shutdown.
138    */
139   private boolean passedZkCluster = false;
140   private MiniDFSCluster dfsCluster = null;
141 
142   private HBaseCluster hbaseCluster = null;
143   private MiniMRCluster mrCluster = null;
144 
145   /** If there is a mini cluster running for this testing utility instance. */
146   private boolean miniClusterRunning;
147 
148   private String hadoopLogDir;
149 
150   /** Directory (a subdirectory of dataTestDir) used by the dfs cluster if any */
151   private File clusterTestDir = null;
152 
153   /** Directory on test filesystem where we put the data for this instance of
154     * HBaseTestingUtility*/
155   private Path dataTestDirOnTestFS = null;
156 
157   /**
158    * System property key to get test directory value.
159    * Name is as it is because mini dfs has hard-codings to put test data here.
160    * It should NOT be used directly in HBase, as it's a property used in
161    *  mini dfs.
162    *  @deprecated can be used only with mini dfs
163    */
164   private static final String TEST_DIRECTORY_KEY = "test.build.data";
165 
166   /** Filesystem URI used for map-reduce mini-cluster setup */
167   private static String FS_URI;
168 
169   /** A set of ports that have been claimed using {@link #randomFreePort()}. */
170   private static final Set<Integer> takenRandomPorts = new HashSet<Integer>();
171 
172   /** Compression algorithms to use in parameterized JUnit 4 tests */
173   public static final List<Object[]> COMPRESSION_ALGORITHMS_PARAMETERIZED =
174     Arrays.asList(new Object[][] {
175       { Compression.Algorithm.NONE },
176       { Compression.Algorithm.GZ }
177     });
178 
179   /** This is for unit tests parameterized with a single boolean. */
180   public static final List<Object[]> BOOLEAN_PARAMETERIZED =
181       Arrays.asList(new Object[][] {
182           { new Boolean(false) },
183           { new Boolean(true) }
184       });
185 
186   /** Compression algorithms to use in testing */
187   public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS ={
188       Compression.Algorithm.NONE, Compression.Algorithm.GZ
189     };
190 
191   /**
192    * Create all combinations of Bloom filters and compression algorithms for
193    * testing.
194    */
195   private static List<Object[]> bloomAndCompressionCombinations() {
196     List<Object[]> configurations = new ArrayList<Object[]>();
197     for (Compression.Algorithm comprAlgo :
198          HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
199       for (BloomType bloomType : BloomType.values()) {
200         configurations.add(new Object[] { comprAlgo, bloomType });
201       }
202     }
203     return Collections.unmodifiableList(configurations);
204   }
205 
206   public static final Collection<Object[]> BLOOM_AND_COMPRESSION_COMBINATIONS =
207       bloomAndCompressionCombinations();
208 
209   public HBaseTestingUtility() {
210     this(HBaseConfiguration.create());
211   }
212 
213   public HBaseTestingUtility(Configuration conf) {
214     this.conf = conf;
215 
216     // a hbase checksum verification failure will cause unit tests to fail
217     ChecksumUtil.generateExceptionForChecksumFailureForTest(true);
218   }
219 
220   /**
221    * Returns this classes's instance of {@link Configuration}.  Be careful how
222    * you use the returned Configuration since {@link HConnection} instances
223    * can be shared.  The Map of HConnections is keyed by the Configuration.  If
224    * say, a Connection was being used against a cluster that had been shutdown,
225    * see {@link #shutdownMiniCluster()}, then the Connection will no longer
226    * be wholesome.  Rather than use the return direct, its usually best to
227    * make a copy and use that.  Do
228    * <code>Configuration c = new Configuration(INSTANCE.getConfiguration());</code>
229    * @return Instance of Configuration.
230    */
231   public Configuration getConfiguration() {
232     return this.conf;
233   }
234 
235   public void setHBaseCluster(HBaseCluster hbaseCluster) {
236     this.hbaseCluster = hbaseCluster;
237   }
238 
239   /**
240    * Home our data in a dir under {@link #DEFAULT_BASE_TEST_DIRECTORY}.
241    * Give it a random name so can have many concurrent tests running if
242    * we need to.  It needs to amend the {@link #TEST_DIRECTORY_KEY}
243    * System property, as it's what minidfscluster bases
244    * it data dir on.  Moding a System property is not the way to do concurrent
245    * instances -- another instance could grab the temporary
246    * value unintentionally -- but not anything can do about it at moment;
247    * single instance only is how the minidfscluster works.
248    *
249    * We also create the underlying directory for
250    *  hadoop.log.dir, mapred.local.dir and hadoop.tmp.dir, and set the values
251    *  in the conf, and as a system property for hadoop.tmp.dir
252    *
253    * @return The calculated data test build directory, if newly-created.
254    */
255   @Override
256   protected Path setupDataTestDir() {
257     Path testPath = super.setupDataTestDir();
258     if (null == testPath) {
259       return null;
260     }
261 
262     createSubDirAndSystemProperty(
263       "hadoop.log.dir",
264       testPath, "hadoop-log-dir");
265 
266     // This is defaulted in core-default.xml to /tmp/hadoop-${user.name}, but
267     //  we want our own value to ensure uniqueness on the same machine
268     createSubDirAndSystemProperty(
269       "hadoop.tmp.dir",
270       testPath, "hadoop-tmp-dir");
271 
272     // Read and modified in org.apache.hadoop.mapred.MiniMRCluster
273     createSubDir(
274       "mapred.local.dir",
275       testPath, "mapred-local-dir");
276 
277     createSubDir(
278       "hbase.local.dir",
279       testPath, "hbase-local-dir");
280     return testPath;
281   }
282 
283   private void createSubDir(String propertyName, Path parent, String subDirName){
284     Path newPath= new Path(parent, subDirName);
285     File newDir = new File(newPath.toString()).getAbsoluteFile();
286     newDir.deleteOnExit();
287     conf.set(propertyName, newDir.getAbsolutePath());
288   }
289 
290   private void createSubDirAndSystemProperty(
291     String propertyName, Path parent, String subDirName){
292 
293     String sysValue = System.getProperty(propertyName);
294 
295     if (sysValue != null) {
296       // There is already a value set. So we do nothing but hope
297       //  that there will be no conflicts
298       LOG.info("System.getProperty(\""+propertyName+"\") already set to: "+
299         sysValue + " so I do NOT create it in " + parent);
300       String confValue = conf.get(propertyName);
301       if (confValue != null && !confValue.endsWith(sysValue)){
302        LOG.warn(
303          propertyName + " property value differs in configuration and system: "+
304          "Configuration="+confValue+" while System="+sysValue+
305          " Erasing configuration value by system value."
306        );
307       }
308       conf.set(propertyName, sysValue);
309     } else {
310       // Ok, it's not set, so we create it as a subdirectory
311       createSubDir(propertyName, parent, subDirName);
312       System.setProperty(propertyName, conf.get(propertyName));
313     }
314   }
315 
316   /**
317    * @return Where to write test data on the test filesystem; Returns working directory
318    * for the test filesystem by default
319    * @see #setupDataTestDirOnTestFS()
320    * @see #getTestFileSystem()
321    */
322   private Path getBaseTestDirOnTestFS() throws IOException {
323     FileSystem fs = getTestFileSystem();
324     return new Path(fs.getWorkingDirectory(), "test-data");
325   }
326 
327   /**
328    * @return Where the DFS cluster will write data on the local subsystem.
329    * Creates it if it does not exist already.  A subdir of {@link #getBaseTestDir()}
330    * @see #getTestFileSystem()
331    */
332   Path getClusterTestDir() {
333     if (clusterTestDir == null){
334       setupClusterTestDir();
335     }
336     return new Path(clusterTestDir.getAbsolutePath());
337   }
338 
339   /**
340    * Creates a directory for the DFS cluster, under the test data
341    */
342   private void setupClusterTestDir() {
343     if (clusterTestDir != null) {
344       return;
345     }
346 
347     // Using randomUUID ensures that multiple clusters can be launched by
348     //  a same test, if it stops & starts them
349     Path testDir = getDataTestDir("dfscluster_" + UUID.randomUUID().toString());
350     clusterTestDir = new File(testDir.toString()).getAbsoluteFile();
351     // Have it cleaned up on exit
352     clusterTestDir.deleteOnExit();
353     conf.set(TEST_DIRECTORY_KEY, clusterTestDir.getPath());
354     LOG.info("Created new mini-cluster data directory: " + clusterTestDir);
355   }
356 
357   /**
358    * Returns a Path in the test filesystem, obtained from {@link #getTestFileSystem()}
359    * to write temporary test data. Call this method after setting up the mini dfs cluster
360    * if the test relies on it.
361    * @return a unique path in the test filesystem
362    */
363   public Path getDataTestDirOnTestFS() throws IOException {
364     if (dataTestDirOnTestFS == null) {
365       setupDataTestDirOnTestFS();
366     }
367 
368     return dataTestDirOnTestFS;
369   }
370 
371   /**
372    * Returns a Path in the test filesystem, obtained from {@link #getTestFileSystem()}
373    * to write temporary test data. Call this method after setting up the mini dfs cluster
374    * if the test relies on it.
375    * @return a unique path in the test filesystem
376    * @param subdirName name of the subdir to create under the base test dir
377    */
378   public Path getDataTestDirOnTestFS(final String subdirName) throws IOException {
379     return new Path(getDataTestDirOnTestFS(), subdirName);
380   }
381 
382   /**
383    * Sets up a path in test filesystem to be used by tests
384    */
385   private void setupDataTestDirOnTestFS() throws IOException {
386     if (dataTestDirOnTestFS != null) {
387       LOG.warn("Data test on test fs dir already setup in "
388           + dataTestDirOnTestFS.toString());
389       return;
390     }
391 
392     //The file system can be either local, mini dfs, or if the configuration
393     //is supplied externally, it can be an external cluster FS. If it is a local
394     //file system, the tests should use getBaseTestDir, otherwise, we can use
395     //the working directory, and create a unique sub dir there
396     FileSystem fs = getTestFileSystem();
397     if (fs.getUri().getScheme().equals(FileSystem.getLocal(conf).getUri().getScheme())) {
398       File dataTestDir = new File(getDataTestDir().toString());
399       dataTestDir.deleteOnExit();
400       dataTestDirOnTestFS = new Path(dataTestDir.getAbsolutePath());
401     } else {
402       Path base = getBaseTestDirOnTestFS();
403       String randomStr = UUID.randomUUID().toString();
404       dataTestDirOnTestFS = new Path(base, randomStr);
405       fs.deleteOnExit(dataTestDirOnTestFS);
406     }
407   }
408 
409   /**
410    * Cleans the test data directory on the test filesystem.
411    * @return True if we removed the test dirs
412    * @throws IOException
413    */
414   public boolean cleanupDataTestDirOnTestFS() throws IOException {
415     boolean ret = getTestFileSystem().delete(dataTestDirOnTestFS, true);
416     if (ret)
417       dataTestDirOnTestFS = null;
418     return ret;
419   }
420 
421   /**
422    * Cleans a subdirectory under the test data directory on the test filesystem.
423    * @return True if we removed child
424    * @throws IOException
425    */
426   public boolean cleanupDataTestDirOnTestFS(String subdirName) throws IOException {
427     Path cpath = getDataTestDirOnTestFS(subdirName);
428     return getTestFileSystem().delete(cpath, true);
429   }
430 
431   /**
432    * Start a minidfscluster.
433    * @param servers How many DNs to start.
434    * @throws Exception
435    * @see {@link #shutdownMiniDFSCluster()}
436    * @return The mini dfs cluster created.
437    */
438   public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception {
439     return startMiniDFSCluster(servers, null);
440   }
441 
442   /**
443    * Start a minidfscluster.
444    * This is useful if you want to run datanode on distinct hosts for things
445    * like HDFS block location verification.
446    * If you start MiniDFSCluster without host names, all instances of the
447    * datanodes will have the same host name.
448    * @param hosts hostnames DNs to run on.
449    * @throws Exception
450    * @see {@link #shutdownMiniDFSCluster()}
451    * @return The mini dfs cluster created.
452    */
453   public MiniDFSCluster startMiniDFSCluster(final String hosts[])
454   throws Exception {
455     if ( hosts != null && hosts.length != 0) {
456       return startMiniDFSCluster(hosts.length, hosts);
457     } else {
458       return startMiniDFSCluster(1, null);
459     }
460   }
461 
462   /**
463    * Start a minidfscluster.
464    * Can only create one.
465    * @param servers How many DNs to start.
466    * @param hosts hostnames DNs to run on.
467    * @throws Exception
468    * @see {@link #shutdownMiniDFSCluster()}
469    * @return The mini dfs cluster created.
470    */
471   public MiniDFSCluster startMiniDFSCluster(int servers, final String hosts[])
472   throws Exception {
473     createDirsAndSetProperties();
474 
475     // Error level to skip some warnings specific to the minicluster. See HBASE-4709
476     org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.util.MBeans.class).
477         setLevel(org.apache.log4j.Level.ERROR);
478     org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class).
479         setLevel(org.apache.log4j.Level.ERROR);
480 
481 
482     this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
483       true, null, null, hosts, null);
484 
485     // Set this just-started cluster as our filesystem.
486     FileSystem fs = this.dfsCluster.getFileSystem();
487     FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
488 
489     // Wait for the cluster to be totally up
490     this.dfsCluster.waitClusterUp();
491 
492     //reset the test directory for test file system
493     dataTestDirOnTestFS = null;
494 
495     return this.dfsCluster;
496   }
497 
498 
499   public MiniDFSCluster startMiniDFSCluster(int servers, final  String racks[], String hosts[])
500       throws Exception {
501     createDirsAndSetProperties();
502     this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
503         true, null, racks, hosts, null);
504 
505     // Set this just-started cluster as our filesystem.
506     FileSystem fs = this.dfsCluster.getFileSystem();
507     FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
508 
509     // Wait for the cluster to be totally up
510     this.dfsCluster.waitClusterUp();
511 
512     //reset the test directory for test file system
513     dataTestDirOnTestFS = null;
514 
515     return this.dfsCluster;
516   }
517 
518   public MiniDFSCluster startMiniDFSClusterForTestHLog(int namenodePort) throws IOException {
519     createDirsAndSetProperties();
520     dfsCluster = new MiniDFSCluster(namenodePort, conf, 5, false, true, true, null,
521         null, null, null);
522     return dfsCluster;
523   }
524 
525   /** This is used before starting HDFS and map-reduce mini-clusters */
526   private void createDirsAndSetProperties() throws IOException {
527     setupClusterTestDir();
528     System.setProperty(TEST_DIRECTORY_KEY, clusterTestDir.getPath());
529     createDirAndSetProperty("cache_data", "test.cache.data");
530     createDirAndSetProperty("hadoop_tmp", "hadoop.tmp.dir");
531     hadoopLogDir = createDirAndSetProperty("hadoop_logs", "hadoop.log.dir");
532     createDirAndSetProperty("mapred_local", "mapred.local.dir");
533     createDirAndSetProperty("mapred_temp", "mapred.temp.dir");
534     enableShortCircuit();
535 
536     Path root = getDataTestDirOnTestFS("hadoop");
537     conf.set(MapreduceTestingShim.getMROutputDirProp(),
538       new Path(root, "mapred-output-dir").toString());
539     conf.set("mapred.system.dir", new Path(root, "mapred-system-dir").toString());
540     conf.set("mapreduce.jobtracker.staging.root.dir",
541       new Path(root, "mapreduce-jobtracker-staging-root-dir").toString());
542     conf.set("mapred.working.dir", new Path(root, "mapred-working-dir").toString());
543   }
544 
545 
546   /**
547    *  Get the HBase setting for dfs.client.read.shortcircuit from the conf or a system property.
548    *  This allows to specify this parameter on the command line.
549    *   If not set, default is true.
550    */
551   public boolean isReadShortCircuitOn(){
552     final String propName = "hbase.tests.use.shortcircuit.reads";
553     String readOnProp = System.getProperty(propName);
554     if (readOnProp != null){
555       return  Boolean.parseBoolean(readOnProp);
556     } else {
557       return conf.getBoolean(propName, false);
558     }
559   }
560 
561   /** Enable the short circuit read, unless configured differently.
562    * Set both HBase and HDFS settings, including skipping the hdfs checksum checks.
563    */
564   private void enableShortCircuit() {
565     if (isReadShortCircuitOn()) {
566       String curUser = System.getProperty("user.name");
567       LOG.info("read short circuit is ON for user " + curUser);
568       // read short circuit, for hdfs
569       conf.set("dfs.block.local-path-access.user", curUser);
570       // read short circuit, for hbase
571       conf.setBoolean("dfs.client.read.shortcircuit", true);
572       // Skip checking checksum, for the hdfs client and the datanode
573       conf.setBoolean("dfs.client.read.shortcircuit.skip.checksum", true);
574     } else {
575       LOG.info("read short circuit is OFF");
576     }
577   }
578 
579   private String createDirAndSetProperty(final String relPath, String property) {
580     String path = getDataTestDir(relPath).toString();
581     System.setProperty(property, path);
582     conf.set(property, path);
583     new File(path).mkdirs();
584     LOG.info("Setting " + property + " to " + path + " in system properties and HBase conf");
585     return path;
586   }
587 
588   /**
589    * Shuts down instance created by call to {@link #startMiniDFSCluster(int)}
590    * or does nothing.
591    * @throws IOException
592    */
593   public void shutdownMiniDFSCluster() throws IOException {
594     if (this.dfsCluster != null) {
595       // The below throws an exception per dn, AsynchronousCloseException.
596       this.dfsCluster.shutdown();
597       dfsCluster = null;
598       dataTestDirOnTestFS = null;
599       FSUtils.setFsDefault(this.conf, new Path("file:///"));
600     }
601   }
602 
603   /**
604    * Call this if you only want a zk cluster.
605    * @see #startMiniZKCluster() if you want zk + dfs + hbase mini cluster.
606    * @throws Exception
607    * @see #shutdownMiniZKCluster()
608    * @return zk cluster started.
609    */
610   public MiniZooKeeperCluster startMiniZKCluster() throws Exception {
611     return startMiniZKCluster(1);
612   }
613 
614   /**
615    * Call this if you only want a zk cluster.
616    * @param zooKeeperServerNum
617    * @see #startMiniZKCluster() if you want zk + dfs + hbase mini cluster.
618    * @throws Exception
619    * @see #shutdownMiniZKCluster()
620    * @return zk cluster started.
621    */
622   public MiniZooKeeperCluster startMiniZKCluster(int zooKeeperServerNum)
623       throws Exception {
624     setupClusterTestDir();
625     return startMiniZKCluster(clusterTestDir, zooKeeperServerNum);
626   }
627 
628   private MiniZooKeeperCluster startMiniZKCluster(final File dir)
629     throws Exception {
630     return startMiniZKCluster(dir,1);
631   }
632 
633   /**
634    * Start a mini ZK cluster. If the property "test.hbase.zookeeper.property.clientPort" is set
635    *  the port mentionned is used as the default port for ZooKeeper.
636    */
637   private MiniZooKeeperCluster startMiniZKCluster(final File dir,
638       int zooKeeperServerNum)
639   throws Exception {
640     if (this.zkCluster != null) {
641       throw new IOException("Cluster already running at " + dir);
642     }
643     this.passedZkCluster = false;
644     this.zkCluster = new MiniZooKeeperCluster(this.getConfiguration());
645     final int defPort = this.conf.getInt("test.hbase.zookeeper.property.clientPort", 0);
646     if (defPort > 0){
647       // If there is a port in the config file, we use it.
648       this.zkCluster.setDefaultClientPort(defPort);
649     }
650     int clientPort =   this.zkCluster.startup(dir,zooKeeperServerNum);
651     this.conf.set(HConstants.ZOOKEEPER_CLIENT_PORT,
652       Integer.toString(clientPort));
653     return this.zkCluster;
654   }
655 
656   /**
657    * Shuts down zk cluster created by call to {@link #startMiniZKCluster(File)}
658    * or does nothing.
659    * @throws IOException
660    * @see #startMiniZKCluster()
661    */
662   public void shutdownMiniZKCluster() throws IOException {
663     if (this.zkCluster != null) {
664       this.zkCluster.shutdown();
665       this.zkCluster = null;
666     }
667   }
668 
669   /**
670    * Start up a minicluster of hbase, dfs, and zookeeper.
671    * @throws Exception
672    * @return Mini hbase cluster instance created.
673    * @see {@link #shutdownMiniDFSCluster()}
674    */
675   public MiniHBaseCluster startMiniCluster() throws Exception {
676     return startMiniCluster(1, 1);
677   }
678 
679   /**
680    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
681    * Modifies Configuration.  Homes the cluster data directory under a random
682    * subdirectory in a directory under System property test.build.data.
683    * Directory is cleaned up on exit.
684    * @param numSlaves Number of slaves to start up.  We'll start this many
685    * datanodes and regionservers.  If numSlaves is > 1, then make sure
686    * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
687    * bind errors.
688    * @throws Exception
689    * @see {@link #shutdownMiniCluster()}
690    * @return Mini hbase cluster instance created.
691    */
692   public MiniHBaseCluster startMiniCluster(final int numSlaves)
693   throws Exception {
694     return startMiniCluster(1, numSlaves);
695   }
696 
697 
698   /**
699    * start minicluster
700    * @throws Exception
701    * @see {@link #shutdownMiniCluster()}
702    * @return Mini hbase cluster instance created.
703    */
704   public MiniHBaseCluster startMiniCluster(final int numMasters,
705     final int numSlaves)
706   throws Exception {
707     return startMiniCluster(numMasters, numSlaves, null);
708   }
709 
710   /**
711    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
712    * Modifies Configuration.  Homes the cluster data directory under a random
713    * subdirectory in a directory under System property test.build.data.
714    * Directory is cleaned up on exit.
715    * @param numMasters Number of masters to start up.  We'll start this many
716    * hbase masters.  If numMasters > 1, you can find the active/primary master
717    * with {@link MiniHBaseCluster#getMaster()}.
718    * @param numSlaves Number of slaves to start up.  We'll start this many
719    * regionservers. If dataNodeHosts == null, this also indicates the number of
720    * datanodes to start. If dataNodeHosts != null, the number of datanodes is
721    * based on dataNodeHosts.length.
722    * If numSlaves is > 1, then make sure
723    * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
724    * bind errors.
725    * @param dataNodeHosts hostnames DNs to run on.
726    * This is useful if you want to run datanode on distinct hosts for things
727    * like HDFS block location verification.
728    * If you start MiniDFSCluster without host names,
729    * all instances of the datanodes will have the same host name.
730    * @throws Exception
731    * @see {@link #shutdownMiniCluster()}
732    * @return Mini hbase cluster instance created.
733    */
734   public MiniHBaseCluster startMiniCluster(final int numMasters,
735       final int numSlaves, final String[] dataNodeHosts) throws Exception {
736     return startMiniCluster(numMasters, numSlaves, numSlaves, dataNodeHosts, null, null);
737   }
738 
739   /**
740    * Same as {@link #startMiniCluster(int, int)}, but with custom number of datanodes.
741    * @param numDataNodes Number of data nodes.
742    */
743   public MiniHBaseCluster startMiniCluster(final int numMasters,
744       final int numSlaves, final int numDataNodes) throws Exception {
745     return startMiniCluster(numMasters, numSlaves, numDataNodes, null, null, null);
746   }
747 
748   /**
749    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
750    * Modifies Configuration.  Homes the cluster data directory under a random
751    * subdirectory in a directory under System property test.build.data.
752    * Directory is cleaned up on exit.
753    * @param numMasters Number of masters to start up.  We'll start this many
754    * hbase masters.  If numMasters > 1, you can find the active/primary master
755    * with {@link MiniHBaseCluster#getMaster()}.
756    * @param numSlaves Number of slaves to start up.  We'll start this many
757    * regionservers. If dataNodeHosts == null, this also indicates the number of
758    * datanodes to start. If dataNodeHosts != null, the number of datanodes is
759    * based on dataNodeHosts.length.
760    * If numSlaves is > 1, then make sure
761    * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
762    * bind errors.
763    * @param dataNodeHosts hostnames DNs to run on.
764    * This is useful if you want to run datanode on distinct hosts for things
765    * like HDFS block location verification.
766    * If you start MiniDFSCluster without host names,
767    * all instances of the datanodes will have the same host name.
768    * @param masterClass The class to use as HMaster, or null for default
769    * @param regionserverClass The class to use as HRegionServer, or null for
770    * default
771    * @throws Exception
772    * @see {@link #shutdownMiniCluster()}
773    * @return Mini hbase cluster instance created.
774    */
775   public MiniHBaseCluster startMiniCluster(final int numMasters,
776       final int numSlaves, final String[] dataNodeHosts, Class<? extends HMaster> masterClass,
777       Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
778           throws Exception {
779     return startMiniCluster(
780         numMasters, numSlaves, numSlaves, dataNodeHosts, masterClass, regionserverClass);
781   }
782 
783   /**
784    * Same as {@link #startMiniCluster(int, int, String[], Class, Class)}, but with custom
785    * number of datanodes.
786    * @param numDataNodes Number of data nodes.
787    */
788   public MiniHBaseCluster startMiniCluster(final int numMasters,
789     final int numSlaves, int numDataNodes, final String[] dataNodeHosts,
790     Class<? extends HMaster> masterClass,
791     Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
792   throws Exception {
793     if (dataNodeHosts != null && dataNodeHosts.length != 0) {
794       numDataNodes = dataNodeHosts.length;
795     }
796 
797     LOG.info("Starting up minicluster with " + numMasters + " master(s) and " +
798         numSlaves + " regionserver(s) and " + numDataNodes + " datanode(s)");
799 
800     // If we already put up a cluster, fail.
801     if (miniClusterRunning) {
802       throw new IllegalStateException("A mini-cluster is already running");
803     }
804     miniClusterRunning = true;
805 
806     setupClusterTestDir();
807     System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestDir.getPath());
808 
809     // Bring up mini dfs cluster. This spews a bunch of warnings about missing
810     // scheme. Complaints are 'Scheme is undefined for build/test/data/dfs/name1'.
811     startMiniDFSCluster(numDataNodes, dataNodeHosts);
812 
813     // Start up a zk cluster.
814     if (this.zkCluster == null) {
815       startMiniZKCluster(clusterTestDir);
816     }
817 
818     // Start the MiniHBaseCluster
819     return startMiniHBaseCluster(numMasters, numSlaves, masterClass, regionserverClass);
820   }
821 
822   public MiniHBaseCluster startMiniHBaseCluster(final int numMasters, final int numSlaves)
823       throws IOException, InterruptedException{
824     return startMiniHBaseCluster(numMasters, numSlaves, null, null);
825   }
826 
827   /**
828    * Starts up mini hbase cluster.  Usually used after call to
829    * {@link #startMiniCluster(int, int)} when doing stepped startup of clusters.
830    * Usually you won't want this.  You'll usually want {@link #startMiniCluster()}.
831    * @param numMasters
832    * @param numSlaves
833    * @return Reference to the hbase mini hbase cluster.
834    * @throws IOException
835    * @throws InterruptedException
836    * @see {@link #startMiniCluster()}
837    */
838   public MiniHBaseCluster startMiniHBaseCluster(final int numMasters,
839         final int numSlaves, Class<? extends HMaster> masterClass,
840         Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
841   throws IOException, InterruptedException {
842     // Now do the mini hbase cluster.  Set the hbase.rootdir in config.
843     createRootDir();
844 
845     // These settings will make the server waits until this exact number of
846     // regions servers are connected.
847     if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1) == -1) {
848       conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, numSlaves);
849     }
850     if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1) == -1) {
851       conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, numSlaves);
852     }
853 
854     Configuration c = new Configuration(this.conf);
855     this.hbaseCluster =
856         new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass);
857     // Don't leave here till we've done a successful scan of the .META.
858     HTable t = new HTable(c, TableName.META_TABLE_NAME);
859     ResultScanner s = t.getScanner(new Scan());
860     while (s.next() != null) {
861       continue;
862     }
863     s.close();
864     t.close();
865 
866     getHBaseAdmin(); // create immediately the hbaseAdmin
867     LOG.info("Minicluster is up");
868     return (MiniHBaseCluster)this.hbaseCluster;
869   }
870 
871   /**
872    * Starts the hbase cluster up again after shutting it down previously in a
873    * test.  Use this if you want to keep dfs/zk up and just stop/start hbase.
874    * @param servers number of region servers
875    * @throws IOException
876    */
877   public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
878     this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
879     // Don't leave here till we've done a successful scan of the .META.
880     HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
881     ResultScanner s = t.getScanner(new Scan());
882     while (s.next() != null) {
883       // do nothing
884     }
885     LOG.info("HBase has been restarted");
886     s.close();
887     t.close();
888   }
889 
890   /**
891    * @return Current mini hbase cluster. Only has something in it after a call
892    * to {@link #startMiniCluster()}.
893    * @see #startMiniCluster()
894    */
895   public MiniHBaseCluster getMiniHBaseCluster() {
896     if (this.hbaseCluster instanceof MiniHBaseCluster) {
897       return (MiniHBaseCluster)this.hbaseCluster;
898     }
899     throw new RuntimeException(hbaseCluster + " not an instance of " +
900                                MiniHBaseCluster.class.getName());
901   }
902 
903   /**
904    * Stops mini hbase, zk, and hdfs clusters.
905    * @throws IOException
906    * @see {@link #startMiniCluster(int)}
907    */
908   public void shutdownMiniCluster() throws Exception {
909     LOG.info("Shutting down minicluster");
910     shutdownMiniHBaseCluster();
911     if (!this.passedZkCluster){
912       shutdownMiniZKCluster();
913     }
914     shutdownMiniDFSCluster();
915 
916     cleanupTestDir();
917     miniClusterRunning = false;
918     LOG.info("Minicluster is down");
919   }
920 
921   /**
922    * Shutdown HBase mini cluster.  Does not shutdown zk or dfs if running.
923    * @throws IOException
924    */
925   public void shutdownMiniHBaseCluster() throws IOException {
926     if (hbaseAdmin != null) {
927       hbaseAdmin.close();
928       hbaseAdmin = null;
929     }
930 
931     if (zooKeeperWatcher != null) {
932       zooKeeperWatcher.close();
933       zooKeeperWatcher = null;
934     }
935 
936     // unset the configuration for MIN and MAX RS to start
937     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
938     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1);
939     if (this.hbaseCluster != null) {
940       this.hbaseCluster.shutdown();
941       // Wait till hbase is down before going on to shutdown zk.
942       this.hbaseCluster.waitUntilShutDown();
943       this.hbaseCluster = null;
944     }
945   }
946 
947   /**
948    * Returns the path to the default root dir the minicluster uses.
949    * Note: this does not cause the root dir to be created.
950    * @return Fully qualified path for the default hbase root dir
951    * @throws IOException
952    */
953   public Path getDefaultRootDirPath() throws IOException {
954 	FileSystem fs = FileSystem.get(this.conf);
955 	return new Path(fs.makeQualified(fs.getHomeDirectory()),"hbase");
956   }
957 
958   /**
959    * Creates an hbase rootdir in user home directory.  Also creates hbase
960    * version file.  Normally you won't make use of this method.  Root hbasedir
961    * is created for you as part of mini cluster startup.  You'd only use this
962    * method if you were doing manual operation.
963    * @return Fully qualified path to hbase root dir
964    * @throws IOException
965    */
966   public Path createRootDir() throws IOException {
967     FileSystem fs = FileSystem.get(this.conf);
968     Path hbaseRootdir = getDefaultRootDirPath();
969     FSUtils.setRootDir(this.conf, hbaseRootdir);
970     fs.mkdirs(hbaseRootdir);
971     FSUtils.setVersion(fs, hbaseRootdir);
972     return hbaseRootdir;
973   }
974 
975   /**
976    * Flushes all caches in the mini hbase cluster
977    * @throws IOException
978    */
979   public void flush() throws IOException {
980     getMiniHBaseCluster().flushcache();
981   }
982 
983   /**
984    * Flushes all caches in the mini hbase cluster
985    * @throws IOException
986    */
987   public void flush(TableName tableName) throws IOException {
988     getMiniHBaseCluster().flushcache(tableName);
989   }
990 
991   /**
992    * Compact all regions in the mini hbase cluster
993    * @throws IOException
994    */
995   public void compact(boolean major) throws IOException {
996     getMiniHBaseCluster().compact(major);
997   }
998 
999   /**
1000    * Compact all of a table's reagion in the mini hbase cluster
1001    * @throws IOException
1002    */
1003   public void compact(TableName tableName, boolean major) throws IOException {
1004     getMiniHBaseCluster().compact(tableName, major);
1005   }
1006 
1007   /**
1008    * Create a table.
1009    * @param tableName
1010    * @param family
1011    * @return An HTable instance for the created table.
1012    * @throws IOException
1013    */
1014   public HTable createTable(String tableName, String family)
1015   throws IOException{
1016     return createTable(TableName.valueOf(tableName), new String[]{family});
1017   }
1018 
1019   /**
1020    * Create a table.
1021    * @param tableName
1022    * @param family
1023    * @return An HTable instance for the created table.
1024    * @throws IOException
1025    */
1026   public HTable createTable(byte[] tableName, byte[] family)
1027   throws IOException{
1028     return createTable(TableName.valueOf(tableName), new byte[][]{family});
1029   }
1030 
1031   /**
1032    * Create a table.
1033    * @param tableName
1034    * @param families
1035    * @return An HTable instance for the created table.
1036    * @throws IOException
1037    */
1038   public HTable createTable(String tableName, String[] families)
1039   throws IOException {
1040     return createTable(tableName, families);
1041   }
1042 
1043   /**
1044    * Create a table.
1045    * @param tableName
1046    * @param families
1047    * @return An HTable instance for the created table.
1048    * @throws IOException
1049    */
1050   public HTable createTable(TableName tableName, String[] families)
1051   throws IOException {
1052     List<byte[]> fams = new ArrayList<byte[]>(families.length);
1053     for (String family : families) {
1054       fams.add(Bytes.toBytes(family));
1055     }
1056     return createTable(tableName, fams.toArray(new byte[0][]));
1057   }
1058 
1059   /**
1060    * Create a table.
1061    * @param tableName
1062    * @param family
1063    * @return An HTable instance for the created table.
1064    * @throws IOException
1065    */
1066   public HTable createTable(TableName tableName, byte[] family)
1067   throws IOException{
1068     return createTable(tableName, new byte[][]{family});
1069   }
1070 
1071 
1072   /**
1073    * Create a table.
1074    * @param tableName
1075    * @param families
1076    * @return An HTable instance for the created table.
1077    * @throws IOException
1078    */
1079   public HTable createTable(byte[] tableName, byte[][] families)
1080   throws IOException {
1081     return createTable(tableName, families,
1082         new Configuration(getConfiguration()));
1083   }
1084 
1085   /**
1086    * Create a table.
1087    * @param tableName
1088    * @param families
1089    * @return An HTable instance for the created table.
1090    * @throws IOException
1091    */
1092   public HTable createTable(TableName tableName, byte[][] families)
1093   throws IOException {
1094     return createTable(tableName, families,
1095         new Configuration(getConfiguration()));
1096   }
1097 
1098   public HTable createTable(byte[] tableName, byte[][] families,
1099       int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
1100     return createTable(TableName.valueOf(tableName), families, numVersions,
1101         startKey, endKey, numRegions);
1102   }
1103 
1104   public HTable createTable(String tableName, byte[][] families,
1105       int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
1106     return createTable(TableName.valueOf(tableName), families, numVersions,
1107         startKey, endKey, numRegions);
1108   }
1109 
1110   public HTable createTable(TableName tableName, byte[][] families,
1111       int numVersions, byte[] startKey, byte[] endKey, int numRegions)
1112   throws IOException{
1113     HTableDescriptor desc = new HTableDescriptor(tableName);
1114     for (byte[] family : families) {
1115       HColumnDescriptor hcd = new HColumnDescriptor(family)
1116           .setMaxVersions(numVersions);
1117       desc.addFamily(hcd);
1118     }
1119     getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
1120     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
1121     waitUntilAllRegionsAssigned(tableName);
1122     return new HTable(getConfiguration(), tableName);
1123   }
1124 
1125   /**
1126    * Create a table.
1127    * @param tableName
1128    * @param families
1129    * @param c Configuration to use
1130    * @return An HTable instance for the created table.
1131    * @throws IOException
1132    */
1133   public HTable createTable(TableName tableName, byte[][] families,
1134       final Configuration c)
1135   throws IOException {
1136     HTableDescriptor desc = new HTableDescriptor(tableName);
1137     for(byte[] family : families) {
1138       HColumnDescriptor hcd = new HColumnDescriptor(family);
1139       // Disable blooms (they are on by default as of 0.95) but we disable them here because
1140       // tests have hard coded counts of what to expect in block cache, etc., and blooms being
1141       // on is interfering.
1142       hcd.setBloomFilterType(BloomType.NONE);
1143       desc.addFamily(hcd);
1144     }
1145     getHBaseAdmin().createTable(desc);
1146     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
1147     waitUntilAllRegionsAssigned(tableName);
1148     return new HTable(c, tableName);
1149   }
1150 
1151   /**
1152    * Create a table.
1153    * @param tableName
1154    * @param families
1155    * @param c Configuration to use
1156    * @return An HTable instance for the created table.
1157    * @throws IOException
1158    */
1159   public HTable createTable(byte[] tableName, byte[][] families,
1160       final Configuration c)
1161   throws IOException {
1162     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
1163     for(byte[] family : families) {
1164       HColumnDescriptor hcd = new HColumnDescriptor(family);
1165       // Disable blooms (they are on by default as of 0.95) but we disable them here because
1166       // tests have hard coded counts of what to expect in block cache, etc., and blooms being
1167       // on is interfering.
1168       hcd.setBloomFilterType(BloomType.NONE);
1169       desc.addFamily(hcd);
1170     }
1171     getHBaseAdmin().createTable(desc);
1172     return new HTable(c, tableName);
1173   }
1174 
1175   /**
1176    * Create a table.
1177    * @param tableName
1178    * @param families
1179    * @param c Configuration to use
1180    * @param numVersions
1181    * @return An HTable instance for the created table.
1182    * @throws IOException
1183    */
1184   public HTable createTable(TableName tableName, byte[][] families,
1185       final Configuration c, int numVersions)
1186   throws IOException {
1187     HTableDescriptor desc = new HTableDescriptor(tableName);
1188     for(byte[] family : families) {
1189       HColumnDescriptor hcd = new HColumnDescriptor(family)
1190           .setMaxVersions(numVersions);
1191       desc.addFamily(hcd);
1192     }
1193     getHBaseAdmin().createTable(desc);
1194     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
1195     waitUntilAllRegionsAssigned(tableName);
1196     return new HTable(c, tableName);
1197   }
1198 
1199   /**
1200    * Create a table.
1201    * @param tableName
1202    * @param families
1203    * @param c Configuration to use
1204    * @param numVersions
1205    * @return An HTable instance for the created table.
1206    * @throws IOException
1207    */
1208   public HTable createTable(byte[] tableName, byte[][] families,
1209       final Configuration c, int numVersions)
1210   throws IOException {
1211     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
1212     for(byte[] family : families) {
1213       HColumnDescriptor hcd = new HColumnDescriptor(family)
1214           .setMaxVersions(numVersions);
1215       desc.addFamily(hcd);
1216     }
1217     getHBaseAdmin().createTable(desc);
1218     return new HTable(c, tableName);
1219   }
1220 
1221   /**
1222    * Create a table.
1223    * @param tableName
1224    * @param family
1225    * @param numVersions
1226    * @return An HTable instance for the created table.
1227    * @throws IOException
1228    */
1229   public HTable createTable(byte[] tableName, byte[] family, int numVersions)
1230   throws IOException {
1231     return createTable(tableName, new byte[][]{family}, numVersions);
1232   }
1233 
1234   /**
1235    * Create a table.
1236    * @param tableName
1237    * @param family
1238    * @param numVersions
1239    * @return An HTable instance for the created table.
1240    * @throws IOException
1241    */
1242   public HTable createTable(TableName tableName, byte[] family, int numVersions)
1243   throws IOException {
1244     return createTable(tableName, new byte[][]{family}, numVersions);
1245   }
1246 
1247   /**
1248    * Create a table.
1249    * @param tableName
1250    * @param families
1251    * @param numVersions
1252    * @return An HTable instance for the created table.
1253    * @throws IOException
1254    */
1255   public HTable createTable(byte[] tableName, byte[][] families,
1256       int numVersions)
1257   throws IOException {
1258     return createTable(TableName.valueOf(tableName), families, numVersions);
1259   }
1260 
1261   /**
1262    * Create a table.
1263    * @param tableName
1264    * @param families
1265    * @param numVersions
1266    * @return An HTable instance for the created table.
1267    * @throws IOException
1268    */
1269   public HTable createTable(TableName tableName, byte[][] families,
1270       int numVersions)
1271   throws IOException {
1272     HTableDescriptor desc = new HTableDescriptor(tableName);
1273     for (byte[] family : families) {
1274       HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions);
1275       desc.addFamily(hcd);
1276     }
1277     getHBaseAdmin().createTable(desc);
1278     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
1279     waitUntilAllRegionsAssigned(tableName);
1280     return new HTable(new Configuration(getConfiguration()), tableName);
1281   }
1282 
1283   /**
1284    * Create a table.
1285    * @param tableName
1286    * @param families
1287    * @param numVersions
1288    * @return An HTable instance for the created table.
1289    * @throws IOException
1290    */
1291   public HTable createTable(byte[] tableName, byte[][] families,
1292     int numVersions, int blockSize) throws IOException {
1293     return createTable(TableName.valueOf(tableName),
1294         families, numVersions, blockSize);
1295   }
1296 
1297   /**
1298    * Create a table.
1299    * @param tableName
1300    * @param families
1301    * @param numVersions
1302    * @return An HTable instance for the created table.
1303    * @throws IOException
1304    */
1305   public HTable createTable(TableName tableName, byte[][] families,
1306     int numVersions, int blockSize) throws IOException {
1307     HTableDescriptor desc = new HTableDescriptor(tableName);
1308     for (byte[] family : families) {
1309       HColumnDescriptor hcd = new HColumnDescriptor(family)
1310           .setMaxVersions(numVersions)
1311           .setBlocksize(blockSize);
1312       desc.addFamily(hcd);
1313     }
1314     getHBaseAdmin().createTable(desc);
1315     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
1316     waitUntilAllRegionsAssigned(tableName);
1317     return new HTable(new Configuration(getConfiguration()), tableName);
1318   }
1319 
1320   /**
1321    * Create a table.
1322    * @param tableName
1323    * @param families
1324    * @param numVersions
1325    * @return An HTable instance for the created table.
1326    * @throws IOException
1327    */
1328   public HTable createTable(byte[] tableName, byte[][] families,
1329       int[] numVersions)
1330   throws IOException {
1331     return createTable(TableName.valueOf(tableName), families, numVersions);
1332   }
1333 
1334   /**
1335    * Create a table.
1336    * @param tableName
1337    * @param families
1338    * @param numVersions
1339    * @return An HTable instance for the created table.
1340    * @throws IOException
1341    */
1342   public HTable createTable(TableName tableName, byte[][] families,
1343       int[] numVersions)
1344   throws IOException {
1345     HTableDescriptor desc = new HTableDescriptor(tableName);
1346     int i = 0;
1347     for (byte[] family : families) {
1348       HColumnDescriptor hcd = new HColumnDescriptor(family)
1349           .setMaxVersions(numVersions[i]);
1350       desc.addFamily(hcd);
1351       i++;
1352     }
1353     getHBaseAdmin().createTable(desc);
1354     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
1355     waitUntilAllRegionsAssigned(tableName);
1356     return new HTable(new Configuration(getConfiguration()), tableName);
1357   }
1358 
1359   /**
1360    * Create a table.
1361    * @param tableName
1362    * @param family
1363    * @param splitRows
1364    * @return An HTable instance for the created table.
1365    * @throws IOException
1366    */
1367   public HTable createTable(byte[] tableName, byte[] family, byte[][] splitRows)
1368     throws IOException{
1369     return createTable(TableName.valueOf(tableName), family, splitRows);
1370   }
1371 
1372   /**
1373    * Create a table.
1374    * @param tableName
1375    * @param family
1376    * @param splitRows
1377    * @return An HTable instance for the created table.
1378    * @throws IOException
1379    */
1380   public HTable createTable(TableName tableName, byte[] family, byte[][] splitRows)
1381       throws IOException {
1382     HTableDescriptor desc = new HTableDescriptor(tableName);
1383     HColumnDescriptor hcd = new HColumnDescriptor(family);
1384     desc.addFamily(hcd);
1385     getHBaseAdmin().createTable(desc, splitRows);
1386     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
1387     waitUntilAllRegionsAssigned(tableName);
1388     return new HTable(getConfiguration(), tableName);
1389   }
1390 
1391   /**
1392    * Create a table.
1393    * @param tableName
1394    * @param families
1395    * @param splitRows
1396    * @return An HTable instance for the created table.
1397    * @throws IOException
1398    */
1399   public HTable createTable(byte[] tableName, byte[][] families, byte[][] splitRows)
1400       throws IOException {
1401     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
1402     for(byte[] family:families) {
1403       HColumnDescriptor hcd = new HColumnDescriptor(family);
1404       desc.addFamily(hcd);
1405     }
1406     getHBaseAdmin().createTable(desc, splitRows);
1407     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
1408     waitUntilAllRegionsAssigned(TableName.valueOf(tableName));
1409     return new HTable(getConfiguration(), tableName);
1410   }
1411 
1412   /**
1413    * Drop an existing table
1414    * @param tableName existing table
1415    */
1416   public void deleteTable(String tableName) throws IOException {
1417     deleteTable(TableName.valueOf(tableName));
1418   }
1419 
1420   /**
1421    * Drop an existing table
1422    * @param tableName existing table
1423    */
1424   public void deleteTable(byte[] tableName) throws IOException {
1425     deleteTable(TableName.valueOf(tableName));
1426   }
1427 
1428   /**
1429    * Drop an existing table
1430    * @param tableName existing table
1431    */
1432   public void deleteTable(TableName tableName) throws IOException {
1433     try {
1434       getHBaseAdmin().disableTable(tableName);
1435     } catch (TableNotEnabledException e) {
1436       LOG.debug("Table: " + tableName + " already disabled, so just deleting it.");
1437     }
1438     getHBaseAdmin().deleteTable(tableName);
1439   }
1440 
1441 
1442   /**
1443    * Provide an existing table name to truncate
1444    * @param tableName existing table
1445    * @return HTable to that new table
1446    * @throws IOException
1447    */
1448   public HTable truncateTable(byte[] tableName) throws IOException {
1449     return truncateTable(TableName.valueOf(tableName));
1450   }
1451 
1452   /**
1453    * Provide an existing table name to truncate
1454    * @param tableName existing table
1455    * @return HTable to that new table
1456    * @throws IOException
1457    */
1458   public HTable truncateTable(TableName tableName) throws IOException {
1459     HTable table = new HTable(getConfiguration(), tableName);
1460     Scan scan = new Scan();
1461     ResultScanner resScan = table.getScanner(scan);
1462     for(Result res : resScan) {
1463       Delete del = new Delete(res.getRow());
1464       table.delete(del);
1465     }
1466     resScan = table.getScanner(scan);
1467     resScan.close();
1468     return table;
1469   }
1470 
1471   /**
1472    * Load table with rows from 'aaa' to 'zzz'.
1473    * @param t Table
1474    * @param f Family
1475    * @return Count of rows loaded.
1476    * @throws IOException
1477    */
1478   public int loadTable(final HTable t, final byte[] f) throws IOException {
1479     t.setAutoFlush(false);
1480     byte[] k = new byte[3];
1481     int rowCount = 0;
1482     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1483       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1484         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1485           k[0] = b1;
1486           k[1] = b2;
1487           k[2] = b3;
1488           Put put = new Put(k);
1489           put.add(f, null, k);
1490           t.put(put);
1491           rowCount++;
1492         }
1493       }
1494     }
1495     t.flushCommits();
1496     return rowCount;
1497   }
1498 
1499   /**
1500    * Load table of multiple column families with rows from 'aaa' to 'zzz'.
1501    * @param t Table
1502    * @param f Array of Families to load
1503    * @return Count of rows loaded.
1504    * @throws IOException
1505    */
1506   public int loadTable(final HTable t, final byte[][] f) throws IOException {
1507     t.setAutoFlush(false);
1508     byte[] k = new byte[3];
1509     int rowCount = 0;
1510     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1511       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1512         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1513           k[0] = b1;
1514           k[1] = b2;
1515           k[2] = b3;
1516           Put put = new Put(k);
1517           for (int i = 0; i < f.length; i++) {
1518             put.add(f[i], null, k);
1519           }
1520           t.put(put);
1521           rowCount++;
1522         }
1523       }
1524     }
1525     t.flushCommits();
1526     return rowCount;
1527   }
1528 
1529   public int loadRegion(final HRegion r, final byte[] f) throws IOException {
1530     return loadRegion(r, f, false);
1531   }
1532 
1533   /**
1534    * Load region with rows from 'aaa' to 'zzz'.
1535    * @param r Region
1536    * @param f Family
1537    * @param flush flush the cache if true
1538    * @return Count of rows loaded.
1539    * @throws IOException
1540    */
1541   public int loadRegion(final HRegion r, final byte[] f, final boolean flush)
1542   throws IOException {
1543     byte[] k = new byte[3];
1544     int rowCount = 0;
1545     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
1546       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
1547         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
1548           k[0] = b1;
1549           k[1] = b2;
1550           k[2] = b3;
1551           Put put = new Put(k);
1552           put.add(f, null, k);
1553           if (r.getLog() == null) put.setDurability(Durability.SKIP_WAL);
1554           r.put(put);
1555           rowCount++;
1556         }
1557       }
1558       if (flush) {
1559         r.flushcache();
1560       }
1561     }
1562     return rowCount;
1563   }
1564 
1565   public void loadNumericRows(final HTable t, final byte[] f, int startRow, int endRow) throws IOException {
1566     for (int i = startRow; i < endRow; i++) {
1567       byte[] data = Bytes.toBytes(String.valueOf(i));
1568       Put put = new Put(data);
1569       put.add(f, null, data);
1570       t.put(put);
1571     }
1572   }
1573 
1574   /**
1575    * Return the number of rows in the given table.
1576    */
1577   public int countRows(final HTable table) throws IOException {
1578     Scan scan = new Scan();
1579     ResultScanner results = table.getScanner(scan);
1580     int count = 0;
1581     for (@SuppressWarnings("unused") Result res : results) {
1582       count++;
1583     }
1584     results.close();
1585     return count;
1586   }
1587 
1588   public int countRows(final HTable table, final byte[]... families) throws IOException {
1589     Scan scan = new Scan();
1590     for (byte[] family: families) {
1591       scan.addFamily(family);
1592     }
1593     ResultScanner results = table.getScanner(scan);
1594     int count = 0;
1595     for (@SuppressWarnings("unused") Result res : results) {
1596       count++;
1597     }
1598     results.close();
1599     return count;
1600   }
1601 
1602   /**
1603    * Return an md5 digest of the entire contents of a table.
1604    */
1605   public String checksumRows(final HTable table) throws Exception {
1606     Scan scan = new Scan();
1607     ResultScanner results = table.getScanner(scan);
1608     MessageDigest digest = MessageDigest.getInstance("MD5");
1609     for (Result res : results) {
1610       digest.update(res.getRow());
1611     }
1612     results.close();
1613     return digest.toString();
1614   }
1615 
1616   /**
1617    * Creates many regions names "aaa" to "zzz".
1618    *
1619    * @param table  The table to use for the data.
1620    * @param columnFamily  The family to insert the data into.
1621    * @return count of regions created.
1622    * @throws IOException When creating the regions fails.
1623    */
1624   public int createMultiRegions(HTable table, byte[] columnFamily)
1625   throws IOException {
1626     return createMultiRegions(getConfiguration(), table, columnFamily);
1627   }
1628 
1629   public static final byte[][] KEYS = {
1630     HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"),
1631     Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
1632     Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
1633     Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
1634     Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
1635     Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
1636     Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
1637     Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
1638     Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
1639   };
1640 
1641   public static final byte[][] KEYS_FOR_HBA_CREATE_TABLE = {
1642       Bytes.toBytes("bbb"),
1643       Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
1644       Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
1645       Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
1646       Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
1647       Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
1648       Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
1649       Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
1650       Bytes.toBytes("xxx"), Bytes.toBytes("yyy"), Bytes.toBytes("zzz")
1651   };
1652 
1653   /**
1654    * Creates many regions names "aaa" to "zzz".
1655    * @param c Configuration to use.
1656    * @param table  The table to use for the data.
1657    * @param columnFamily  The family to insert the data into.
1658    * @return count of regions created.
1659    * @throws IOException When creating the regions fails.
1660    */
1661   public int createMultiRegions(final Configuration c, final HTable table,
1662       final byte[] columnFamily)
1663   throws IOException {
1664     return createMultiRegions(c, table, columnFamily, KEYS);
1665   }
1666 
1667   /**
1668    * Creates the specified number of regions in the specified table.
1669    * @param c
1670    * @param table
1671    * @param family
1672    * @param numRegions
1673    * @return
1674    * @throws IOException
1675    */
1676   public int createMultiRegions(final Configuration c, final HTable table,
1677       final byte [] family, int numRegions)
1678   throws IOException {
1679     if (numRegions < 3) throw new IOException("Must create at least 3 regions");
1680     byte [] startKey = Bytes.toBytes("aaaaa");
1681     byte [] endKey = Bytes.toBytes("zzzzz");
1682     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
1683     byte [][] regionStartKeys = new byte[splitKeys.length+1][];
1684     for (int i=0;i<splitKeys.length;i++) {
1685       regionStartKeys[i+1] = splitKeys[i];
1686     }
1687     regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
1688     return createMultiRegions(c, table, family, regionStartKeys);
1689   }
1690 
1691   public int createMultiRegions(final Configuration c, final HTable table,
1692       final byte[] columnFamily, byte [][] startKeys)
1693   throws IOException {
1694     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
1695     HTable meta = new HTable(c, TableName.META_TABLE_NAME);
1696     HTableDescriptor htd = table.getTableDescriptor();
1697     if(!htd.hasFamily(columnFamily)) {
1698       HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
1699       htd.addFamily(hcd);
1700     }
1701     // remove empty region - this is tricky as the mini cluster during the test
1702     // setup already has the "<tablename>,,123456789" row with an empty start
1703     // and end key. Adding the custom regions below adds those blindly,
1704     // including the new start region from empty to "bbb". lg
1705     List<byte[]> rows = getMetaTableRows(htd.getTableName());
1706     String regionToDeleteInFS = table
1707         .getRegionsInRange(Bytes.toBytes(""), Bytes.toBytes("")).get(0)
1708         .getRegionInfo().getEncodedName();
1709     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
1710     // add custom ones
1711     int count = 0;
1712     for (int i = 0; i < startKeys.length; i++) {
1713       int j = (i + 1) % startKeys.length;
1714       HRegionInfo hri = new HRegionInfo(table.getName(),
1715         startKeys[i], startKeys[j]);
1716       MetaEditor.addRegionToMeta(meta, hri);
1717       newRegions.add(hri);
1718       count++;
1719     }
1720     // see comment above, remove "old" (or previous) single region
1721     for (byte[] row : rows) {
1722       LOG.info("createMultiRegions: deleting meta row -> " +
1723         Bytes.toStringBinary(row));
1724       meta.delete(new Delete(row));
1725     }
1726     // remove the "old" region from FS
1727     Path tableDir = new Path(getDefaultRootDirPath().toString()
1728         + System.getProperty("file.separator") + htd.getTableName()
1729         + System.getProperty("file.separator") + regionToDeleteInFS);
1730     FileSystem.get(c).delete(tableDir);
1731     // flush cache of regions
1732     HConnection conn = table.getConnection();
1733     conn.clearRegionCache();
1734     // assign all the new regions IF table is enabled.
1735     HBaseAdmin admin = getHBaseAdmin();
1736     if (admin.isTableEnabled(table.getTableName())) {
1737       for(HRegionInfo hri : newRegions) {
1738         admin.assign(hri.getRegionName());
1739       }
1740     }
1741 
1742     meta.close();
1743 
1744     return count;
1745   }
1746 
1747   /**
1748    * Create rows in META for regions of the specified table with the specified
1749    * start keys.  The first startKey should be a 0 length byte array if you
1750    * want to form a proper range of regions.
1751    * @param conf
1752    * @param htd
1753    * @param startKeys
1754    * @return list of region info for regions added to meta
1755    * @throws IOException
1756    */
1757   public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
1758       final HTableDescriptor htd, byte [][] startKeys)
1759   throws IOException {
1760     HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
1761     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
1762     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
1763     // add custom ones
1764     for (int i = 0; i < startKeys.length; i++) {
1765       int j = (i + 1) % startKeys.length;
1766       HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKeys[i],
1767           startKeys[j]);
1768       MetaEditor.addRegionToMeta(meta, hri);
1769       newRegions.add(hri);
1770     }
1771 
1772     meta.close();
1773     return newRegions;
1774   }
1775 
1776   /**
1777    * Returns all rows from the .META. table.
1778    *
1779    * @throws IOException When reading the rows fails.
1780    */
1781   public List<byte[]> getMetaTableRows() throws IOException {
1782     // TODO: Redo using MetaReader class
1783     HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
1784     List<byte[]> rows = new ArrayList<byte[]>();
1785     ResultScanner s = t.getScanner(new Scan());
1786     for (Result result : s) {
1787       LOG.info("getMetaTableRows: row -> " +
1788         Bytes.toStringBinary(result.getRow()));
1789       rows.add(result.getRow());
1790     }
1791     s.close();
1792     t.close();
1793     return rows;
1794   }
1795 
1796   /**
1797    * Returns all rows from the .META. table for a given user table
1798    *
1799    * @throws IOException When reading the rows fails.
1800    */
1801   public List<byte[]> getMetaTableRows(TableName tableName) throws IOException {
1802     // TODO: Redo using MetaReader.
1803     HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
1804     List<byte[]> rows = new ArrayList<byte[]>();
1805     ResultScanner s = t.getScanner(new Scan());
1806     for (Result result : s) {
1807       HRegionInfo info = HRegionInfo.getHRegionInfo(result);
1808       if (info == null) {
1809         LOG.error("No region info for row " + Bytes.toString(result.getRow()));
1810         // TODO figure out what to do for this new hosed case.
1811         continue;
1812       }
1813 
1814       if (info.getTableName().equals(tableName)) {
1815         LOG.info("getMetaTableRows: row -> " +
1816             Bytes.toStringBinary(result.getRow()) + info);
1817         rows.add(result.getRow());
1818       }
1819     }
1820     s.close();
1821     t.close();
1822     return rows;
1823   }
1824 
1825   /**
1826    * Tool to get the reference to the region server object that holds the
1827    * region of the specified user table.
1828    * It first searches for the meta rows that contain the region of the
1829    * specified table, then gets the index of that RS, and finally retrieves
1830    * the RS's reference.
1831    * @param tableName user table to lookup in .META.
1832    * @return region server that holds it, null if the row doesn't exist
1833    * @throws IOException
1834    * @throws InterruptedException
1835    */
1836   public HRegionServer getRSForFirstRegionInTable(byte[] tableName)
1837       throws IOException, InterruptedException {
1838     return getRSForFirstRegionInTable(TableName.valueOf(tableName));
1839   }
1840   /**
1841    * Tool to get the reference to the region server object that holds the
1842    * region of the specified user table.
1843    * It first searches for the meta rows that contain the region of the
1844    * specified table, then gets the index of that RS, and finally retrieves
1845    * the RS's reference.
1846    * @param tableName user table to lookup in .META.
1847    * @return region server that holds it, null if the row doesn't exist
1848    * @throws IOException
1849    */
1850   public HRegionServer getRSForFirstRegionInTable(TableName tableName)
1851       throws IOException, InterruptedException {
1852     List<byte[]> metaRows = getMetaTableRows(tableName);
1853     if (metaRows == null || metaRows.isEmpty()) {
1854       return null;
1855     }
1856     LOG.debug("Found " + metaRows.size() + " rows for table " +
1857       tableName);
1858     byte [] firstrow = metaRows.get(0);
1859     LOG.debug("FirstRow=" + Bytes.toString(firstrow));
1860     long pause = getConfiguration().getLong(HConstants.HBASE_CLIENT_PAUSE,
1861       HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
1862     int numRetries = getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
1863       HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
1864     RetryCounter retrier = new RetryCounter(numRetries, (int)pause, TimeUnit.MICROSECONDS);
1865     while(retrier.shouldRetry()) {
1866       int index = getMiniHBaseCluster().getServerWith(firstrow);
1867       if (index != -1) {
1868         return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer();
1869       }
1870       // Came back -1.  Region may not be online yet.  Sleep a while.
1871       retrier.sleepUntilNextRetry();
1872     }
1873     return null;
1874   }
1875 
1876   /**
1877    * Starts a <code>MiniMRCluster</code> with a default number of
1878    * <code>TaskTracker</code>'s.
1879    *
1880    * @throws IOException When starting the cluster fails.
1881    */
1882   public MiniMRCluster startMiniMapReduceCluster() throws IOException {
1883     startMiniMapReduceCluster(2);
1884     return mrCluster;
1885   }
1886 
1887   /**
1888    * Tasktracker has a bug where changing the hadoop.log.dir system property
1889    * will not change its internal static LOG_DIR variable.
1890    */
1891   private void forceChangeTaskLogDir() {
1892     Field logDirField;
1893     try {
1894       logDirField = TaskLog.class.getDeclaredField("LOG_DIR");
1895       logDirField.setAccessible(true);
1896 
1897       Field modifiersField = Field.class.getDeclaredField("modifiers");
1898       modifiersField.setAccessible(true);
1899       modifiersField.setInt(logDirField, logDirField.getModifiers() & ~Modifier.FINAL);
1900 
1901       logDirField.set(null, new File(hadoopLogDir, "userlogs"));
1902     } catch (SecurityException e) {
1903       throw new RuntimeException(e);
1904     } catch (NoSuchFieldException e) {
1905       // TODO Auto-generated catch block
1906       throw new RuntimeException(e);
1907     } catch (IllegalArgumentException e) {
1908       throw new RuntimeException(e);
1909     } catch (IllegalAccessException e) {
1910       throw new RuntimeException(e);
1911     }
1912   }
1913 
1914   /**
1915    * Starts a <code>MiniMRCluster</code>. Call {@link #setFileSystemURI(String)} to use a different
1916    * filesystem.
1917    * @param servers  The number of <code>TaskTracker</code>'s to start.
1918    * @throws IOException When starting the cluster fails.
1919    */
1920   private void startMiniMapReduceCluster(final int servers) throws IOException {
1921     if (mrCluster != null) {
1922       throw new IllegalStateException("MiniMRCluster is already running");
1923     }
1924     LOG.info("Starting mini mapreduce cluster...");
1925     setupClusterTestDir();
1926     createDirsAndSetProperties();
1927 
1928     forceChangeTaskLogDir();
1929 
1930     //// hadoop2 specific settings
1931     // Tests were failing because this process used 6GB of virtual memory and was getting killed.
1932     // we up the VM usable so that processes don't get killed.
1933     conf.setFloat("yarn.nodemanager.vmem-pmem-ratio", 8.0f);
1934 
1935     // Tests were failing due to MAPREDUCE-4880 / MAPREDUCE-4607 against hadoop 2.0.2-alpha and
1936     // this avoids the problem by disabling speculative task execution in tests.
1937     conf.setBoolean("mapreduce.map.speculative", false);
1938     conf.setBoolean("mapreduce.reduce.speculative", false);
1939     ////
1940 
1941     // Allow the user to override FS URI for this map-reduce cluster to use.
1942     mrCluster = new MiniMRCluster(servers,
1943       FS_URI != null ? FS_URI : FileSystem.get(conf).getUri().toString(), 1,
1944       null, null, new JobConf(this.conf));
1945     JobConf jobConf = MapreduceTestingShim.getJobConf(mrCluster);
1946     if (jobConf == null) {
1947       jobConf = mrCluster.createJobConf();
1948     }
1949 
1950     jobConf.set("mapred.local.dir",
1951       conf.get("mapred.local.dir")); //Hadoop MiniMR overwrites this while it should not
1952     LOG.info("Mini mapreduce cluster started");
1953 
1954     // In hadoop2, YARN/MR2 starts a mini cluster with its own conf instance and updates settings.
1955     // Our HBase MR jobs need several of these settings in order to properly run.  So we copy the
1956     // necessary config properties here.  YARN-129 required adding a few properties.
1957     conf.set("mapred.job.tracker", jobConf.get("mapred.job.tracker"));
1958     // this for mrv2 support; mr1 ignores this
1959     conf.set("mapreduce.framework.name", "yarn");
1960     conf.setBoolean("yarn.is.minicluster", true);
1961     String rmAddress = jobConf.get("yarn.resourcemanager.address");
1962     if (rmAddress != null) {
1963       conf.set("yarn.resourcemanager.address", rmAddress);
1964     }
1965     String schedulerAddress =
1966       jobConf.get("yarn.resourcemanager.scheduler.address");
1967     if (schedulerAddress != null) {
1968       conf.set("yarn.resourcemanager.scheduler.address", schedulerAddress);
1969     }
1970   }
1971 
1972   /**
1973    * Stops the previously started <code>MiniMRCluster</code>.
1974    */
1975   public void shutdownMiniMapReduceCluster() {
1976     LOG.info("Stopping mini mapreduce cluster...");
1977     if (mrCluster != null) {
1978       mrCluster.shutdown();
1979       mrCluster = null;
1980     }
1981     // Restore configuration to point to local jobtracker
1982     conf.set("mapred.job.tracker", "local");
1983     LOG.info("Mini mapreduce cluster stopped");
1984   }
1985 
1986   /**
1987    * Switches the logger for the given class to DEBUG level.
1988    *
1989    * @param clazz  The class for which to switch to debug logging.
1990    */
1991   public void enableDebug(Class<?> clazz) {
1992     Log l = LogFactory.getLog(clazz);
1993     if (l instanceof Log4JLogger) {
1994       ((Log4JLogger) l).getLogger().setLevel(org.apache.log4j.Level.DEBUG);
1995     } else if (l instanceof Jdk14Logger) {
1996       ((Jdk14Logger) l).getLogger().setLevel(java.util.logging.Level.ALL);
1997     }
1998   }
1999 
2000   /**
2001    * Expire the Master's session
2002    * @throws Exception
2003    */
2004   public void expireMasterSession() throws Exception {
2005     HMaster master = getMiniHBaseCluster().getMaster();
2006     expireSession(master.getZooKeeper(), false);
2007   }
2008 
2009   /**
2010    * Expire a region server's session
2011    * @param index which RS
2012    * @throws Exception
2013    */
2014   public void expireRegionServerSession(int index) throws Exception {
2015     HRegionServer rs = getMiniHBaseCluster().getRegionServer(index);
2016     expireSession(rs.getZooKeeper(), false);
2017     decrementMinRegionServerCount();
2018   }
2019 
2020   private void decrementMinRegionServerCount() {
2021     // decrement the count for this.conf, for newly spwaned master
2022     // this.hbaseCluster shares this configuration too
2023     decrementMinRegionServerCount(getConfiguration());
2024 
2025     // each master thread keeps a copy of configuration
2026     for (MasterThread master : getHBaseCluster().getMasterThreads()) {
2027       decrementMinRegionServerCount(master.getMaster().getConfiguration());
2028     }
2029   }
2030 
2031   private void decrementMinRegionServerCount(Configuration conf) {
2032     int currentCount = conf.getInt(
2033         ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
2034     if (currentCount != -1) {
2035       conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
2036           Math.max(currentCount - 1, 1));
2037     }
2038   }
2039 
2040   public void expireSession(ZooKeeperWatcher nodeZK) throws Exception {
2041    expireSession(nodeZK, false);
2042   }
2043 
2044   @Deprecated
2045   public void expireSession(ZooKeeperWatcher nodeZK, Server server)
2046     throws Exception {
2047     expireSession(nodeZK, false);
2048   }
2049 
2050   /**
2051    * Expire a ZooKeeper session as recommended in ZooKeeper documentation
2052    * http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A4
2053    * There are issues when doing this:
2054    * [1] http://www.mail-archive.com/dev@zookeeper.apache.org/msg01942.html
2055    * [2] https://issues.apache.org/jira/browse/ZOOKEEPER-1105
2056    *
2057    * @param nodeZK - the ZK watcher to expire
2058    * @param checkStatus - true to check if we can create an HTable with the
2059    *                    current configuration.
2060    */
2061   public void expireSession(ZooKeeperWatcher nodeZK, boolean checkStatus)
2062     throws Exception {
2063     Configuration c = new Configuration(this.conf);
2064     String quorumServers = ZKConfig.getZKQuorumServersString(c);
2065     ZooKeeper zk = nodeZK.getRecoverableZooKeeper().getZooKeeper();
2066     byte[] password = zk.getSessionPasswd();
2067     long sessionID = zk.getSessionId();
2068 
2069     // Expiry seems to be asynchronous (see comment from P. Hunt in [1]),
2070     //  so we create a first watcher to be sure that the
2071     //  event was sent. We expect that if our watcher receives the event
2072     //  other watchers on the same machine will get is as well.
2073     // When we ask to close the connection, ZK does not close it before
2074     //  we receive all the events, so don't have to capture the event, just
2075     //  closing the connection should be enough.
2076     ZooKeeper monitor = new ZooKeeper(quorumServers,
2077       1000, new org.apache.zookeeper.Watcher(){
2078       @Override
2079       public void process(WatchedEvent watchedEvent) {
2080         LOG.info("Monitor ZKW received event="+watchedEvent);
2081       }
2082     } , sessionID, password);
2083 
2084     // Making it expire
2085     ZooKeeper newZK = new ZooKeeper(quorumServers,
2086         1000, EmptyWatcher.instance, sessionID, password);
2087 
2088     //ensure that we have connection to the server before closing down, otherwise
2089     //the close session event will be eaten out before we start CONNECTING state
2090     long start = System.currentTimeMillis();
2091     while (newZK.getState() != States.CONNECTED
2092          && System.currentTimeMillis() - start < 1000) {
2093        Thread.sleep(1);
2094     }
2095     newZK.close();
2096     LOG.info("ZK Closed Session 0x" + Long.toHexString(sessionID));
2097 
2098     // Now closing & waiting to be sure that the clients get it.
2099     monitor.close();
2100 
2101     if (checkStatus) {
2102       new HTable(new Configuration(conf), TableName.META_TABLE_NAME).close();
2103     }
2104   }
2105 
2106   /**
2107    * Get the Mini HBase cluster.
2108    *
2109    * @return hbase cluster
2110    * @see #getHBaseClusterInterface()
2111    */
2112   public MiniHBaseCluster getHBaseCluster() {
2113     return getMiniHBaseCluster();
2114   }
2115 
2116   /**
2117    * Returns the HBaseCluster instance.
2118    * <p>Returned object can be any of the subclasses of HBaseCluster, and the
2119    * tests referring this should not assume that the cluster is a mini cluster or a
2120    * distributed one. If the test only works on a mini cluster, then specific
2121    * method {@link #getMiniHBaseCluster()} can be used instead w/o the
2122    * need to type-cast.
2123    */
2124   public HBaseCluster getHBaseClusterInterface() {
2125     //implementation note: we should rename this method as #getHBaseCluster(),
2126     //but this would require refactoring 90+ calls.
2127     return hbaseCluster;
2128   }
2129 
2130   /**
2131    * Returns a HBaseAdmin instance.
2132    * This instance is shared between HBaseTestingUtility instance users.
2133    * Don't close it, it will be closed automatically when the
2134    * cluster shutdowns
2135    *
2136    * @return The HBaseAdmin instance.
2137    * @throws IOException
2138    */
2139   public synchronized HBaseAdmin getHBaseAdmin()
2140   throws IOException {
2141     if (hbaseAdmin == null){
2142       hbaseAdmin = new HBaseAdmin(getConfiguration());
2143     }
2144     return hbaseAdmin;
2145   }
2146   private HBaseAdmin hbaseAdmin = null;
2147 
2148   /**
2149    * Returns a ZooKeeperWatcher instance.
2150    * This instance is shared between HBaseTestingUtility instance users.
2151    * Don't close it, it will be closed automatically when the
2152    * cluster shutdowns
2153    *
2154    * @return The ZooKeeperWatcher instance.
2155    * @throws IOException
2156    */
2157   public synchronized ZooKeeperWatcher getZooKeeperWatcher()
2158     throws IOException {
2159     if (zooKeeperWatcher == null) {
2160       zooKeeperWatcher = new ZooKeeperWatcher(conf, "testing utility",
2161         new Abortable() {
2162         @Override public void abort(String why, Throwable e) {
2163           throw new RuntimeException("Unexpected abort in HBaseTestingUtility:"+why, e);
2164         }
2165         @Override public boolean isAborted() {return false;}
2166       });
2167     }
2168     return zooKeeperWatcher;
2169   }
2170   private ZooKeeperWatcher zooKeeperWatcher;
2171 
2172 
2173 
2174   /**
2175    * Closes the named region.
2176    *
2177    * @param regionName  The region to close.
2178    * @throws IOException
2179    */
2180   public void closeRegion(String regionName) throws IOException {
2181     closeRegion(Bytes.toBytes(regionName));
2182   }
2183 
2184   /**
2185    * Closes the named region.
2186    *
2187    * @param regionName  The region to close.
2188    * @throws IOException
2189    */
2190   public void closeRegion(byte[] regionName) throws IOException {
2191     getHBaseAdmin().closeRegion(regionName, null);
2192   }
2193 
2194   /**
2195    * Closes the region containing the given row.
2196    *
2197    * @param row  The row to find the containing region.
2198    * @param table  The table to find the region.
2199    * @throws IOException
2200    */
2201   public void closeRegionByRow(String row, HTable table) throws IOException {
2202     closeRegionByRow(Bytes.toBytes(row), table);
2203   }
2204 
2205   /**
2206    * Closes the region containing the given row.
2207    *
2208    * @param row  The row to find the containing region.
2209    * @param table  The table to find the region.
2210    * @throws IOException
2211    */
2212   public void closeRegionByRow(byte[] row, HTable table) throws IOException {
2213     HRegionLocation hrl = table.getRegionLocation(row);
2214     closeRegion(hrl.getRegionInfo().getRegionName());
2215   }
2216 
2217   /*
2218    * Retrieves a splittable region randomly from tableName
2219    *
2220    * @param tableName name of table
2221    * @param maxAttempts maximum number of attempts, unlimited for value of -1
2222    * @return the HRegion chosen, null if none was found within limit of maxAttempts
2223    */
2224   public HRegion getSplittableRegion(TableName tableName, int maxAttempts) {
2225     List<HRegion> regions = getHBaseCluster().getRegions(tableName);
2226     int regCount = regions.size();
2227     Set<Integer> attempted = new HashSet<Integer>();
2228     int idx;
2229     int attempts = 0;
2230     do {
2231       regions = getHBaseCluster().getRegions(tableName);
2232       if (regCount != regions.size()) {
2233         // if there was region movement, clear attempted Set
2234         attempted.clear();
2235       }
2236       regCount = regions.size();
2237       // There are chances that before we get the region for the table from an RS the region may
2238       // be going for CLOSE.  This may be because online schema change is enabled
2239       if (regCount > 0) {
2240         idx = random.nextInt(regCount);
2241         // if we have just tried this region, there is no need to try again
2242         if (attempted.contains(idx))
2243           continue;
2244         try {
2245           regions.get(idx).checkSplit();
2246           return regions.get(idx);
2247         } catch (Exception ex) {
2248           LOG.warn("Caught exception", ex);
2249           attempted.add(idx);
2250         }
2251       }
2252       attempts++;
2253     } while (maxAttempts == -1 || attempts < maxAttempts);
2254     return null;
2255   }
2256 
2257   public MiniZooKeeperCluster getZkCluster() {
2258     return zkCluster;
2259   }
2260 
2261   public void setZkCluster(MiniZooKeeperCluster zkCluster) {
2262     this.passedZkCluster = true;
2263     this.zkCluster = zkCluster;
2264     conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkCluster.getClientPort());
2265   }
2266 
2267   public MiniDFSCluster getDFSCluster() {
2268     return dfsCluster;
2269   }
2270 
2271   public void setDFSCluster(MiniDFSCluster cluster) throws IOException {
2272     if (dfsCluster != null && dfsCluster.isClusterUp()) {
2273       throw new IOException("DFSCluster is already running! Shut it down first.");
2274     }
2275     this.dfsCluster = cluster;
2276   }
2277 
2278   public FileSystem getTestFileSystem() throws IOException {
2279     return HFileSystem.get(conf);
2280   }
2281 
2282   /**
2283    * Wait until all regions in a table have been assigned.  Waits default timeout before giving up
2284    * (30 seconds).
2285    * @param table Table to wait on.
2286    * @throws InterruptedException
2287    * @throws IOException
2288    */
2289   public void waitTableAvailable(byte[] table)
2290       throws InterruptedException, IOException {
2291     waitTableAvailable(getHBaseAdmin(), table, 30000);
2292   }
2293 
2294   public void waitTableAvailable(HBaseAdmin admin, byte[] table)
2295       throws InterruptedException, IOException {
2296     waitTableAvailable(admin, table, 30000);
2297   }
2298 
2299   /**
2300    * Wait until all regions in a table have been assigned
2301    * @param table Table to wait on.
2302    * @param timeoutMillis Timeout.
2303    * @throws InterruptedException
2304    * @throws IOException
2305    */
2306   public void waitTableAvailable(byte[] table, long timeoutMillis)
2307   throws InterruptedException, IOException {
2308     waitTableAvailable(getHBaseAdmin(), table, timeoutMillis);
2309   }
2310 
2311   public void waitTableAvailable(HBaseAdmin admin, byte[] table, long timeoutMillis)
2312   throws InterruptedException, IOException {
2313     long startWait = System.currentTimeMillis();
2314     while (!admin.isTableAvailable(table)) {
2315       assertTrue("Timed out waiting for table to become available " +
2316         Bytes.toStringBinary(table),
2317         System.currentTimeMillis() - startWait < timeoutMillis);
2318       Thread.sleep(200);
2319     }
2320     // Finally make sure all regions are fully open and online out on the cluster. Regions may be
2321     // in the .META. table and almost open on all regionservers but there setting the region
2322     // online in the regionserver is the very last thing done and can take a little while to happen.
2323     // Below we do a get.  The get will retry if a NotServeringRegionException or a
2324     // RegionOpeningException.  It is crass but when done all will be online.
2325     try {
2326       Canary.sniff(admin, TableName.valueOf(table));
2327     } catch (Exception e) {
2328       throw new IOException(e);
2329     }
2330   }
2331 
2332   /**
2333    * Waits for a table to be 'enabled'.  Enabled means that table is set as 'enabled' and the
2334    * regions have been all assigned.  Will timeout after default period (30 seconds)
2335    * @see #waitTableAvailable(byte[])
2336    * @param table Table to wait on.
2337    * @param table
2338    * @throws InterruptedException
2339    * @throws IOException
2340    */
2341   public void waitTableEnabled(byte[] table)
2342       throws InterruptedException, IOException {
2343     waitTableEnabled(getHBaseAdmin(), table, 30000);
2344   }
2345 
2346   public void waitTableEnabled(HBaseAdmin admin, byte[] table)
2347       throws InterruptedException, IOException {
2348     waitTableEnabled(admin, table, 30000);
2349   }
2350 
2351   /**
2352    * Waits for a table to be 'enabled'.  Enabled means that table is set as 'enabled' and the
2353    * regions have been all assigned.
2354    * @see #waitTableAvailable(byte[])
2355    * @param table Table to wait on.
2356    * @param timeoutMillis Time to wait on it being marked enabled.
2357    * @throws InterruptedException
2358    * @throws IOException
2359    */
2360   public void waitTableEnabled(byte[] table, long timeoutMillis)
2361   throws InterruptedException, IOException {
2362     waitTableEnabled(getHBaseAdmin(), table, timeoutMillis);
2363   }
2364 
2365   public void waitTableEnabled(HBaseAdmin admin, byte[] table, long timeoutMillis)
2366   throws InterruptedException, IOException {
2367     long startWait = System.currentTimeMillis();
2368     waitTableAvailable(admin, table, timeoutMillis);
2369     long remainder = System.currentTimeMillis() - startWait;
2370     while (!admin.isTableEnabled(table)) {
2371       assertTrue("Timed out waiting for table to become available and enabled " +
2372          Bytes.toStringBinary(table),
2373          System.currentTimeMillis() - remainder < timeoutMillis);
2374       Thread.sleep(200);
2375     }
2376     LOG.debug("REMOVE AFTER table=" + Bytes.toString(table) + ", isTableAvailable=" +
2377         admin.isTableAvailable(table) +
2378         ", isTableEnabled=" + admin.isTableEnabled(table));
2379   }
2380 
2381   /**
2382    * Make sure that at least the specified number of region servers
2383    * are running
2384    * @param num minimum number of region servers that should be running
2385    * @return true if we started some servers
2386    * @throws IOException
2387    */
2388   public boolean ensureSomeRegionServersAvailable(final int num)
2389       throws IOException {
2390     boolean startedServer = false;
2391     MiniHBaseCluster hbaseCluster = getMiniHBaseCluster();
2392     for (int i=hbaseCluster.getLiveRegionServerThreads().size(); i<num; ++i) {
2393       LOG.info("Started new server=" + hbaseCluster.startRegionServer());
2394       startedServer = true;
2395     }
2396 
2397     return startedServer;
2398   }
2399 
2400 
2401   /**
2402    * Make sure that at least the specified number of region servers
2403    * are running. We don't count the ones that are currently stopping or are
2404    * stopped.
2405    * @param num minimum number of region servers that should be running
2406    * @return true if we started some servers
2407    * @throws IOException
2408    */
2409   public boolean ensureSomeNonStoppedRegionServersAvailable(final int num)
2410     throws IOException {
2411     boolean startedServer = ensureSomeRegionServersAvailable(num);
2412 
2413     int nonStoppedServers = 0;
2414     for (JVMClusterUtil.RegionServerThread rst :
2415       getMiniHBaseCluster().getRegionServerThreads()) {
2416 
2417       HRegionServer hrs = rst.getRegionServer();
2418       if (hrs.isStopping() || hrs.isStopped()) {
2419         LOG.info("A region server is stopped or stopping:"+hrs);
2420       } else {
2421         nonStoppedServers++;
2422       }
2423     }
2424     for (int i=nonStoppedServers; i<num; ++i) {
2425       LOG.info("Started new server=" + getMiniHBaseCluster().startRegionServer());
2426       startedServer = true;
2427     }
2428     return startedServer;
2429   }
2430 
2431 
2432   /**
2433    * This method clones the passed <code>c</code> configuration setting a new
2434    * user into the clone.  Use it getting new instances of FileSystem.  Only
2435    * works for DistributedFileSystem.
2436    * @param c Initial configuration
2437    * @param differentiatingSuffix Suffix to differentiate this user from others.
2438    * @return A new configuration instance with a different user set into it.
2439    * @throws IOException
2440    */
2441   public static User getDifferentUser(final Configuration c,
2442     final String differentiatingSuffix)
2443   throws IOException {
2444     FileSystem currentfs = FileSystem.get(c);
2445     if (!(currentfs instanceof DistributedFileSystem)) {
2446       return User.getCurrent();
2447     }
2448     // Else distributed filesystem.  Make a new instance per daemon.  Below
2449     // code is taken from the AppendTestUtil over in hdfs.
2450     String username = User.getCurrent().getName() +
2451       differentiatingSuffix;
2452     User user = User.createUserForTesting(c, username,
2453         new String[]{"supergroup"});
2454     return user;
2455   }
2456 
2457   /**
2458    * Set maxRecoveryErrorCount in DFSClient.  In 0.20 pre-append its hard-coded to 5 and
2459    * makes tests linger.  Here is the exception you'll see:
2460    * <pre>
2461    * 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...
2462    * </pre>
2463    * @param stream A DFSClient.DFSOutputStream.
2464    * @param max
2465    * @throws NoSuchFieldException
2466    * @throws SecurityException
2467    * @throws IllegalAccessException
2468    * @throws IllegalArgumentException
2469    */
2470   public static void setMaxRecoveryErrorCount(final OutputStream stream,
2471       final int max) {
2472     try {
2473       Class<?> [] clazzes = DFSClient.class.getDeclaredClasses();
2474       for (Class<?> clazz: clazzes) {
2475         String className = clazz.getSimpleName();
2476         if (className.equals("DFSOutputStream")) {
2477           if (clazz.isInstance(stream)) {
2478             Field maxRecoveryErrorCountField =
2479               stream.getClass().getDeclaredField("maxRecoveryErrorCount");
2480             maxRecoveryErrorCountField.setAccessible(true);
2481             maxRecoveryErrorCountField.setInt(stream, max);
2482             break;
2483           }
2484         }
2485       }
2486     } catch (Exception e) {
2487       LOG.info("Could not set max recovery field", e);
2488     }
2489   }
2490 
2491   /**
2492    * Wait until all regions for a table in .META. have a non-empty
2493    * info:server, up to 60 seconds. This means all regions have been deployed,
2494    * master has been informed and updated .META. with the regions deployed
2495    * server.
2496    * @param tableName the table name
2497    * @throws IOException
2498    */
2499   public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOException {
2500     waitUntilAllRegionsAssigned(tableName, 60000);
2501   }
2502 
2503   /**
2504    * Wait until all regions for a table in .META. have a non-empty
2505    * info:server, or until timeout.  This means all regions have been deployed,
2506    * master has been informed and updated .META. with the regions deployed
2507    * server.
2508    * @param tableName the table name
2509    * @param timeout timeout, in milliseconds
2510    * @throws IOException
2511    */
2512   public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout)
2513       throws IOException {
2514     final HTable meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME);
2515     try {
2516       waitFor(timeout, 200, true, new Predicate<IOException>() {
2517         @Override
2518         public boolean evaluate() throws IOException {
2519           boolean allRegionsAssigned = true;
2520           Scan scan = new Scan();
2521           scan.addFamily(HConstants.CATALOG_FAMILY);
2522           ResultScanner s = meta.getScanner(scan);
2523           try {
2524             Result r;
2525             while ((r = s.next()) != null) {
2526               byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
2527               HRegionInfo info = HRegionInfo.parseFromOrNull(b);
2528               if (info != null && info.getTableName().equals(tableName)) {
2529                 b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
2530                 allRegionsAssigned &= (b != null);
2531               }
2532             }
2533           } finally {
2534             s.close();
2535           }
2536           return allRegionsAssigned;
2537         }
2538       });
2539     } finally {
2540       meta.close();
2541     }
2542   }
2543 
2544   /**
2545    * Do a small get/scan against one store. This is required because store
2546    * has no actual methods of querying itself, and relies on StoreScanner.
2547    */
2548   public static List<KeyValue> getFromStoreFile(HStore store,
2549                                                 Get get) throws IOException {
2550     MultiVersionConsistencyControl.resetThreadReadPoint();
2551     Scan scan = new Scan(get);
2552     InternalScanner scanner = (InternalScanner) store.getScanner(scan,
2553         scan.getFamilyMap().get(store.getFamily().getName()));
2554 
2555     List<KeyValue> result = new ArrayList<KeyValue>();
2556     scanner.next(result);
2557     if (!result.isEmpty()) {
2558       // verify that we are on the row we want:
2559       KeyValue kv = result.get(0);
2560       if (!Bytes.equals(kv.getRow(), get.getRow())) {
2561         result.clear();
2562       }
2563     }
2564     scanner.close();
2565     return result;
2566   }
2567 
2568   /**
2569    * Create region split keys between startkey and endKey
2570    *
2571    * @param startKey
2572    * @param endKey
2573    * @param numRegions the number of regions to be created. it has to be greater than 3.
2574    * @return
2575    */
2576   public byte[][] getRegionSplitStartKeys(byte[] startKey, byte[] endKey, int numRegions){
2577     assertTrue(numRegions>3);
2578     byte [][] tmpSplitKeys = Bytes.split(startKey, endKey, numRegions - 3);
2579     byte [][] result = new byte[tmpSplitKeys.length+1][];
2580     for (int i=0;i<tmpSplitKeys.length;i++) {
2581       result[i+1] = tmpSplitKeys[i];
2582     }
2583     result[0] = HConstants.EMPTY_BYTE_ARRAY;
2584     return result;
2585   }
2586 
2587   /**
2588    * Do a small get/scan against one store. This is required because store
2589    * has no actual methods of querying itself, and relies on StoreScanner.
2590    */
2591   public static List<KeyValue> getFromStoreFile(HStore store,
2592                                                 byte [] row,
2593                                                 NavigableSet<byte[]> columns
2594                                                 ) throws IOException {
2595     Get get = new Get(row);
2596     Map<byte[], NavigableSet<byte[]>> s = get.getFamilyMap();
2597     s.put(store.getFamily().getName(), columns);
2598 
2599     return getFromStoreFile(store,get);
2600   }
2601 
2602   /**
2603    * Gets a ZooKeeperWatcher.
2604    * @param TEST_UTIL
2605    */
2606   public static ZooKeeperWatcher getZooKeeperWatcher(
2607       HBaseTestingUtility TEST_UTIL) throws ZooKeeperConnectionException,
2608       IOException {
2609     ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
2610         "unittest", new Abortable() {
2611           boolean aborted = false;
2612 
2613           @Override
2614           public void abort(String why, Throwable e) {
2615             aborted = true;
2616             throw new RuntimeException("Fatal ZK error, why=" + why, e);
2617           }
2618 
2619           @Override
2620           public boolean isAborted() {
2621             return aborted;
2622           }
2623         });
2624     return zkw;
2625   }
2626 
2627   /**
2628    * Creates a znode with OPENED state.
2629    * @param TEST_UTIL
2630    * @param region
2631    * @param serverName
2632    * @return
2633    * @throws IOException
2634    * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
2635    * @throws KeeperException
2636    * @throws NodeExistsException
2637    */
2638   public static ZooKeeperWatcher createAndForceNodeToOpenedState(
2639       HBaseTestingUtility TEST_UTIL, HRegion region,
2640       ServerName serverName) throws ZooKeeperConnectionException,
2641       IOException, KeeperException, NodeExistsException {
2642     ZooKeeperWatcher zkw = getZooKeeperWatcher(TEST_UTIL);
2643     ZKAssign.createNodeOffline(zkw, region.getRegionInfo(), serverName);
2644     int version = ZKAssign.transitionNodeOpening(zkw, region
2645         .getRegionInfo(), serverName);
2646     ZKAssign.transitionNodeOpened(zkw, region.getRegionInfo(), serverName,
2647         version);
2648     return zkw;
2649   }
2650 
2651   public static void assertKVListsEqual(String additionalMsg,
2652       final List<KeyValue> expected,
2653       final List<KeyValue> actual) {
2654     final int eLen = expected.size();
2655     final int aLen = actual.size();
2656     final int minLen = Math.min(eLen, aLen);
2657 
2658     int i;
2659     for (i = 0; i < minLen
2660         && KeyValue.COMPARATOR.compare(expected.get(i), actual.get(i)) == 0;
2661         ++i) {}
2662 
2663     if (additionalMsg == null) {
2664       additionalMsg = "";
2665     }
2666     if (!additionalMsg.isEmpty()) {
2667       additionalMsg = ". " + additionalMsg;
2668     }
2669 
2670     if (eLen != aLen || i != minLen) {
2671       throw new AssertionError(
2672           "Expected and actual KV arrays differ at position " + i + ": " +
2673           safeGetAsStr(expected, i) + " (length " + eLen +") vs. " +
2674           safeGetAsStr(actual, i) + " (length " + aLen + ")" + additionalMsg);
2675     }
2676   }
2677 
2678   private static <T> String safeGetAsStr(List<T> lst, int i) {
2679     if (0 <= i && i < lst.size()) {
2680       return lst.get(i).toString();
2681     } else {
2682       return "<out_of_range>";
2683     }
2684   }
2685 
2686   public String getClusterKey() {
2687     return conf.get(HConstants.ZOOKEEPER_QUORUM) + ":"
2688         + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":"
2689         + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
2690             HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
2691   }
2692 
2693   /** Creates a random table with the given parameters */
2694   public HTable createRandomTable(String tableName,
2695       final Collection<String> families,
2696       final int maxVersions,
2697       final int numColsPerRow,
2698       final int numFlushes,
2699       final int numRegions,
2700       final int numRowsPerFlush)
2701       throws IOException, InterruptedException {
2702 
2703     LOG.info("\n\nCreating random table " + tableName + " with " + numRegions +
2704         " regions, " + numFlushes + " storefiles per region, " +
2705         numRowsPerFlush + " rows per flush, maxVersions=" +  maxVersions +
2706         "\n");
2707 
2708     final Random rand = new Random(tableName.hashCode() * 17L + 12938197137L);
2709     final int numCF = families.size();
2710     final byte[][] cfBytes = new byte[numCF][];
2711     {
2712       int cfIndex = 0;
2713       for (String cf : families) {
2714         cfBytes[cfIndex++] = Bytes.toBytes(cf);
2715       }
2716     }
2717 
2718     final int actualStartKey = 0;
2719     final int actualEndKey = Integer.MAX_VALUE;
2720     final int keysPerRegion = (actualEndKey - actualStartKey) / numRegions;
2721     final int splitStartKey = actualStartKey + keysPerRegion;
2722     final int splitEndKey = actualEndKey - keysPerRegion;
2723     final String keyFormat = "%08x";
2724     final HTable table = createTable(tableName, cfBytes,
2725         maxVersions,
2726         Bytes.toBytes(String.format(keyFormat, splitStartKey)),
2727         Bytes.toBytes(String.format(keyFormat, splitEndKey)),
2728         numRegions);
2729 
2730     if (hbaseCluster != null) {
2731       getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME);
2732     }
2733 
2734     for (int iFlush = 0; iFlush < numFlushes; ++iFlush) {
2735       for (int iRow = 0; iRow < numRowsPerFlush; ++iRow) {
2736         final byte[] row = Bytes.toBytes(String.format(keyFormat,
2737             actualStartKey + rand.nextInt(actualEndKey - actualStartKey)));
2738 
2739         Put put = new Put(row);
2740         Delete del = new Delete(row);
2741         for (int iCol = 0; iCol < numColsPerRow; ++iCol) {
2742           final byte[] cf = cfBytes[rand.nextInt(numCF)];
2743           final long ts = rand.nextInt();
2744           final byte[] qual = Bytes.toBytes("col" + iCol);
2745           if (rand.nextBoolean()) {
2746             final byte[] value = Bytes.toBytes("value_for_row_" + iRow +
2747                 "_cf_" + Bytes.toStringBinary(cf) + "_col_" + iCol + "_ts_" +
2748                 ts + "_random_" + rand.nextLong());
2749             put.add(cf, qual, ts, value);
2750           } else if (rand.nextDouble() < 0.8) {
2751             del.deleteColumn(cf, qual, ts);
2752           } else {
2753             del.deleteColumns(cf, qual, ts);
2754           }
2755         }
2756 
2757         if (!put.isEmpty()) {
2758           table.put(put);
2759         }
2760 
2761         if (!del.isEmpty()) {
2762           table.delete(del);
2763         }
2764       }
2765       LOG.info("Initiating flush #" + iFlush + " for table " + tableName);
2766       table.flushCommits();
2767       if (hbaseCluster != null) {
2768         getMiniHBaseCluster().flushcache(table.getName());
2769       }
2770     }
2771 
2772     return table;
2773   }
2774 
2775   private static final int MIN_RANDOM_PORT = 0xc000;
2776   private static final int MAX_RANDOM_PORT = 0xfffe;
2777   private static Random random = new Random();
2778 
2779   /**
2780    * Returns a random port. These ports cannot be registered with IANA and are
2781    * intended for dynamic allocation (see http://bit.ly/dynports).
2782    */
2783   public static int randomPort() {
2784     return MIN_RANDOM_PORT
2785         + random.nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT);
2786   }
2787 
2788   /**
2789    * Returns a random free port and marks that port as taken. Not thread-safe. Expected to be
2790    * called from single-threaded test setup code/
2791    */
2792   public static int randomFreePort() {
2793     int port = 0;
2794     do {
2795       port = randomPort();
2796       if (takenRandomPorts.contains(port)) {
2797         continue;
2798       }
2799       takenRandomPorts.add(port);
2800 
2801       try {
2802         ServerSocket sock = new ServerSocket(port);
2803         sock.close();
2804       } catch (IOException ex) {
2805         port = 0;
2806       }
2807     } while (port == 0);
2808     return port;
2809   }
2810 
2811 
2812   public static String randomMultiCastAddress() {
2813     return "226.1.1." + random.nextInt(254);
2814   }
2815 
2816 
2817 
2818   public static void waitForHostPort(String host, int port)
2819       throws IOException {
2820     final int maxTimeMs = 10000;
2821     final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
2822     IOException savedException = null;
2823     LOG.info("Waiting for server at " + host + ":" + port);
2824     for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
2825       try {
2826         Socket sock = new Socket(InetAddress.getByName(host), port);
2827         sock.close();
2828         savedException = null;
2829         LOG.info("Server at " + host + ":" + port + " is available");
2830         break;
2831       } catch (UnknownHostException e) {
2832         throw new IOException("Failed to look up " + host, e);
2833       } catch (IOException e) {
2834         savedException = e;
2835       }
2836       Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
2837     }
2838 
2839     if (savedException != null) {
2840       throw savedException;
2841     }
2842   }
2843 
2844   /**
2845    * Creates a pre-split table for load testing. If the table already exists,
2846    * logs a warning and continues.
2847    * @return the number of regions the table was split into
2848    */
2849   public static int createPreSplitLoadTestTable(Configuration conf,
2850       TableName tableName, byte[] columnFamily, Algorithm compression,
2851       DataBlockEncoding dataBlockEncoding) throws IOException {
2852     HTableDescriptor desc = new HTableDescriptor(tableName);
2853     HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
2854     hcd.setDataBlockEncoding(dataBlockEncoding);
2855     hcd.setCompressionType(compression);
2856     return createPreSplitLoadTestTable(conf, desc, hcd);
2857   }
2858 
2859   /**
2860    * Creates a pre-split table for load testing. If the table already exists,
2861    * logs a warning and continues.
2862    * @return the number of regions the table was split into
2863    */
2864   public static int createPreSplitLoadTestTable(Configuration conf,
2865       HTableDescriptor desc, HColumnDescriptor hcd) throws IOException {
2866     if (!desc.hasFamily(hcd.getName())) {
2867       desc.addFamily(hcd);
2868     }
2869 
2870     int totalNumberOfRegions = 0;
2871     HBaseAdmin admin = new HBaseAdmin(conf);
2872     try {
2873       // create a table a pre-splits regions.
2874       // The number of splits is set as:
2875       //    region servers * regions per region server).
2876       int numberOfServers = admin.getClusterStatus().getServers().size();
2877       if (numberOfServers == 0) {
2878         throw new IllegalStateException("No live regionservers");
2879       }
2880 
2881       totalNumberOfRegions = numberOfServers * DEFAULT_REGIONS_PER_SERVER;
2882       LOG.info("Number of live regionservers: " + numberOfServers + ", " +
2883           "pre-splitting table into " + totalNumberOfRegions + " regions " +
2884           "(default regions per server: " + DEFAULT_REGIONS_PER_SERVER + ")");
2885 
2886       byte[][] splits = new RegionSplitter.HexStringSplit().split(
2887           totalNumberOfRegions);
2888 
2889       admin.createTable(desc, splits);
2890       admin.close();
2891     } catch (MasterNotRunningException e) {
2892       LOG.error("Master not running", e);
2893       throw new IOException(e);
2894     } catch (TableExistsException e) {
2895       LOG.warn("Table " + desc.getTableName() +
2896           " already exists, continuing");
2897     } finally {
2898       admin.close();
2899     }
2900     return totalNumberOfRegions;
2901   }
2902 
2903   public static int getMetaRSPort(Configuration conf) throws IOException {
2904     HTable table = new HTable(conf, TableName.META_TABLE_NAME);
2905     HRegionLocation hloc = table.getRegionLocation(Bytes.toBytes(""));
2906     table.close();
2907     return hloc.getPort();
2908   }
2909 
2910   /**
2911    *  Due to async racing issue, a region may not be in
2912    *  the online region list of a region server yet, after
2913    *  the assignment znode is deleted and the new assignment
2914    *  is recorded in master.
2915    */
2916   public void assertRegionOnServer(
2917       final HRegionInfo hri, final ServerName server,
2918       final long timeout) throws IOException, InterruptedException {
2919     long timeoutTime = System.currentTimeMillis() + timeout;
2920     while (true) {
2921       List<HRegionInfo> regions = getHBaseAdmin().getOnlineRegions(server);
2922       if (regions.contains(hri)) return;
2923       long now = System.currentTimeMillis();
2924       if (now > timeoutTime) break;
2925       Thread.sleep(10);
2926     }
2927     fail("Could not find region " + hri.getRegionNameAsString()
2928       + " on server " + server);
2929   }
2930 
2931   public HRegion createTestRegion(String tableName, HColumnDescriptor hcd)
2932       throws IOException {
2933     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
2934     htd.addFamily(hcd);
2935     HRegionInfo info =
2936         new HRegionInfo(TableName.valueOf(tableName), null, null, false);
2937     HRegion region =
2938         HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), htd);
2939     return region;
2940   }
2941 
2942   public void setFileSystemURI(String fsURI) {
2943     FS_URI = fsURI;
2944   }
2945 
2946   /**
2947    * Wrapper method for {@link Waiter#waitFor(Configuration, long, Predicate)}.
2948    */
2949   public <E extends Exception> long waitFor(long timeout, Predicate<E> predicate)
2950       throws E {
2951     return Waiter.waitFor(this.conf, timeout, predicate);
2952   }
2953 
2954   /**
2955    * Wrapper method for {@link Waiter#waitFor(Configuration, long, long, Predicate)}.
2956    */
2957   public <E extends Exception> long waitFor(long timeout, long interval, Predicate<E> predicate)
2958       throws E {
2959     return Waiter.waitFor(this.conf, timeout, interval, predicate);
2960   }
2961 
2962   /**
2963    * Wrapper method for {@link Waiter#waitFor(Configuration, long, long, boolean, Predicate)}.
2964    */
2965   public <E extends Exception> long waitFor(long timeout, long interval,
2966       boolean failIfTimeout, Predicate<E> predicate) throws E {
2967     return Waiter.waitFor(this.conf, timeout, interval, failIfTimeout, predicate);
2968   }
2969 
2970   /**
2971    * Returns a {@link Predicate} for checking that there is no regions in transition in master
2972    */
2973   public Waiter.Predicate<Exception> predicateNoRegionsInTransition() {
2974     return new Waiter.Predicate<Exception>() {
2975       @Override
2976       public boolean evaluate() throws Exception {
2977         final RegionStates regionStates = getMiniHBaseCluster().getMaster()
2978             .getAssignmentManager().getRegionStates();
2979         return !regionStates.isRegionsInTransition();
2980       }
2981     };
2982   }
2983 
2984 }