1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase;
21  
22  import static org.junit.Assert.assertTrue;
23  
24  import java.io.File;
25  import java.io.IOException;
26  import java.io.OutputStream;
27  import java.lang.reflect.Field;
28  import java.security.MessageDigest;
29  import java.util.ArrayList;
30  import java.util.Arrays;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.NavigableSet;
34  import java.util.UUID;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.commons.logging.impl.Jdk14Logger;
39  import org.apache.commons.logging.impl.Log4JLogger;
40  import org.apache.hadoop.conf.Configuration;
41  import org.apache.hadoop.fs.FileSystem;
42  import org.apache.hadoop.fs.Path;
43  import org.apache.hadoop.hbase.client.Delete;
44  import org.apache.hadoop.hbase.client.Get;
45  import org.apache.hadoop.hbase.client.HBaseAdmin;
46  import org.apache.hadoop.hbase.client.HConnection;
47  import org.apache.hadoop.hbase.client.HTable;
48  import org.apache.hadoop.hbase.client.Put;
49  import org.apache.hadoop.hbase.client.Result;
50  import org.apache.hadoop.hbase.client.ResultScanner;
51  import org.apache.hadoop.hbase.client.Scan;
52  import org.apache.hadoop.hbase.io.hfile.Compression;
53  import org.apache.hadoop.hbase.master.HMaster;
54  import org.apache.hadoop.hbase.regionserver.HRegion;
55  import org.apache.hadoop.hbase.regionserver.HRegionServer;
56  import org.apache.hadoop.hbase.regionserver.InternalScanner;
57  import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
58  import org.apache.hadoop.hbase.regionserver.Store;
59  import org.apache.hadoop.hbase.security.User;
60  import org.apache.hadoop.hbase.util.Bytes;
61  import org.apache.hadoop.hbase.util.FSUtils;
62  import org.apache.hadoop.hbase.util.Threads;
63  import org.apache.hadoop.hbase.util.Writables;
64  import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
65  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
66  import org.apache.hadoop.hbase.zookeeper.ZKConfig;
67  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
68  import org.apache.hadoop.hdfs.DFSClient;
69  import org.apache.hadoop.hdfs.DistributedFileSystem;
70  import org.apache.hadoop.hdfs.MiniDFSCluster;
71  import org.apache.hadoop.mapred.MiniMRCluster;
72  import org.apache.zookeeper.KeeperException;
73  import org.apache.zookeeper.ZooKeeper;
74  import org.apache.zookeeper.KeeperException.NodeExistsException;
75  
76  /**
77   * Facility for testing HBase. Replacement for
78   * old HBaseTestCase and HBaseClusterTestCase functionality.
79   * Create an instance and keep it around testing HBase.  This class is
80   * meant to be your one-stop shop for anything you might need testing.  Manages
81   * one cluster at a time only.
82   * Depends on log4j being on classpath and
83   * hbase-site.xml for logging and test-run configuration.  It does not set
84   * logging levels nor make changes to configuration parameters.
85   */
86  public class HBaseTestingUtility {
87    private static final Log LOG = LogFactory.getLog(HBaseTestingUtility.class);
88    private Configuration conf;
89    private MiniZooKeeperCluster zkCluster = null;
90    /**
91     * Set if we were passed a zkCluster.  If so, we won't shutdown zk as
92     * part of general shutdown.
93     */
94    private boolean passedZkCluster = false;
95    private MiniDFSCluster dfsCluster = null;
96  
97    private MiniHBaseCluster hbaseCluster = null;
98    private MiniMRCluster mrCluster = null;
99  
100   // Directory where we put the data for this instance of HBaseTestingUtility
101   private File dataTestDir = null;
102 
103   // Directory (usually a subdirectory of dataTestDir) used by the dfs cluster
104   //  if any
105   private File clusterTestDir = null;
106 
107   /**
108    * System property key to get test directory value.
109    * Name is as it is because mini dfs has hard-codings to put test data here.
110    * It should NOT be used directly in HBase, as it's a property used in
111    *  mini dfs.
112    *  @deprecated can be used only with mini dfs
113    */
114   private static final String TEST_DIRECTORY_KEY = "test.build.data";
115 
116   /**
117    * System property key to get base test directory value
118    */
119   public static final String BASE_TEST_DIRECTORY_KEY =
120     "test.build.data.basedirectory";
121 
122   /**
123    * Default base directory for test output.
124    */
125   public static final String DEFAULT_BASE_TEST_DIRECTORY = "target/test-data";
126 
127   /** Compression algorithms to use in parameterized JUnit 4 tests */
128   public static final List<Object[]> COMPRESSION_ALGORITHMS_PARAMETERIZED =
129     Arrays.asList(new Object[][] {
130       { Compression.Algorithm.NONE },
131       { Compression.Algorithm.GZ }
132     });
133 
134   /** Compression algorithms to use in testing */
135   public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS ={
136       Compression.Algorithm.NONE, Compression.Algorithm.GZ
137     };
138 
139   public HBaseTestingUtility() {
140     this(HBaseConfiguration.create());
141   }
142 
143   public HBaseTestingUtility(Configuration conf) {
144     this.conf = conf;
145   }
146 
147   /**
148    * Returns this classes's instance of {@link Configuration}.  Be careful how
149    * you use the returned Configuration since {@link HConnection} instances
150    * can be shared.  The Map of HConnections is keyed by the Configuration.  If
151    * say, a Connection was being used against a cluster that had been shutdown,
152    * see {@link #shutdownMiniCluster()}, then the Connection will no longer
153    * be wholesome.  Rather than use the return direct, its usually best to
154    * make a copy and use that.  Do
155    * <code>Configuration c = new Configuration(INSTANCE.getConfiguration());</code>
156    * @return Instance of Configuration.
157    */
158   public Configuration getConfiguration() {
159     return this.conf;
160   }
161 
162   /**
163    * @return Where to write test data on local filesystem; usually
164    * {@link #DEFAULT_BASE_TEST_DIRECTORY}
165    * Should not be used by the unit tests, hence its's private.
166    * Unit test will use a subdirectory of this directory.
167    * @see #setupDataTestDir()
168    * @see #getTestFileSystem()
169    */
170   private Path getBaseTestDir() {
171     String PathName = System.getProperty(
172       BASE_TEST_DIRECTORY_KEY, DEFAULT_BASE_TEST_DIRECTORY);
173 
174     return new Path(PathName);
175   }
176 
177   /**
178    * @return Where to write test data on local filesystem, specific to
179    *  the test.  Useful for tests that do not use a cluster.
180    * Creates it if it does not exist already.
181    * @see #getTestFileSystem()
182    */
183   public Path getDataTestDir() {
184     if (dataTestDir == null){
185       setupDataTestDir();
186     }
187     return new Path(dataTestDir.getAbsolutePath());
188   }
189 
190   /**
191    * @return Where the DFS cluster will write data on the local subsystem.
192    * Creates it if it does not exist already.
193    * @see #getTestFileSystem()
194    */
195   public Path getClusterTestDir() {
196     if (clusterTestDir == null){
197       setupClusterTestDir();
198     }
199     return new Path(clusterTestDir.getAbsolutePath());
200   }
201 
202   /**
203    * @param subdirName
204    * @return Path to a subdirectory named <code>subdirName</code> under
205    * {@link #getDataTestDir()}.
206    * Does *NOT* create it if it does not exist.
207    */
208   public Path getDataTestDir(final String subdirName) {
209     return new Path(getDataTestDir(), subdirName);
210   }
211 
212   /**
213    * Home our data in a dir under {@link #DEFAULT_BASE_TEST_DIRECTORY}.
214    * Give it a random name so can have many concurrent tests running if
215    * we need to.  It needs to amend the {@link #TEST_DIRECTORY_KEY}
216    * System property, as it's what minidfscluster bases
217    * it data dir on.  Moding a System property is not the way to do concurrent
218    * instances -- another instance could grab the temporary
219    * value unintentionally -- but not anything can do about it at moment;
220    * single instance only is how the minidfscluster works.
221    * @return The calculated data test build directory.
222    */
223   private void setupDataTestDir() {
224     if (dataTestDir != null) {
225       LOG.warn("Data test dir already setup in " +
226         dataTestDir.getAbsolutePath());
227       return;
228     }
229 
230     String randomStr = UUID.randomUUID().toString();
231     Path testPath= new Path(
232       getBaseTestDir(),
233       randomStr
234     );
235 
236     dataTestDir = new File(testPath.toString()).getAbsoluteFile();
237     dataTestDir.deleteOnExit();
238   }
239 
240   /**
241    * Creates a directory for the DFS cluster, under the test data
242    */
243   private void setupClusterTestDir() {
244     if (clusterTestDir != null) {
245       LOG.warn("Cluster test dir already setup in " +
246         clusterTestDir.getAbsolutePath());
247       return;
248     }
249 
250     // Using randomUUID ensures that multiple clusters can be launched by
251     //  a same test, if it stops & starts them
252     Path testDir = getDataTestDir("dfscluster_" + UUID.randomUUID().toString());
253     clusterTestDir = new File(testDir.toString()).getAbsoluteFile();
254     // Have it cleaned up on exit
255     clusterTestDir.deleteOnExit();
256   }
257 
258   /**
259    * @throws IOException If a cluster -- zk, dfs, or hbase -- already running.
260    */
261   public void isRunningCluster() throws IOException {
262     if (dfsCluster == null) return;
263     throw new IOException("Cluster already running at " +
264       this.clusterTestDir);
265   }
266 
267   /**
268    * Start a minidfscluster.
269    * @param servers How many DNs to start.
270    * @throws Exception
271    * @see {@link #shutdownMiniDFSCluster()}
272    * @return The mini dfs cluster created.
273    */
274   public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception {
275     return startMiniDFSCluster(servers, null);
276   }
277 
278   /**
279    * Start a minidfscluster.
280    * This is useful if you want to run datanode on distinct hosts for things
281    * like HDFS block location verification.
282    * If you start MiniDFSCluster without host names, all instances of the
283    * datanodes will have the same host name.
284    * @param hosts hostnames DNs to run on.
285    * @throws Exception
286    * @see {@link #shutdownMiniDFSCluster()}
287    * @return The mini dfs cluster created.
288    */
289   public MiniDFSCluster startMiniDFSCluster(final String hosts[])
290     throws Exception {
291     if ( hosts != null && hosts.length != 0) {
292       return startMiniDFSCluster(hosts.length, hosts);
293     } else {
294       return startMiniDFSCluster(1, null);
295     }
296   }
297 
298   /**
299    * Start a minidfscluster.
300    * Can only create one.
301    * @param servers How many DNs to start.
302    * @param hosts hostnames DNs to run on.
303    * @throws Exception
304    * @see {@link #shutdownMiniDFSCluster()}
305    * @return The mini dfs cluster created.
306    */
307   public MiniDFSCluster startMiniDFSCluster(int servers, final String hosts[])
308   throws Exception {
309 
310     // Check that there is not already a cluster running
311     isRunningCluster();
312 
313     // Initialize the local directory used by the MiniDFS
314     if (clusterTestDir == null) {
315       setupClusterTestDir();
316     }
317 
318     // We have to set this property as it is used by MiniCluster
319     System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestDir.toString());
320 
321     // Some tests also do this:
322     //  System.getProperty("test.cache.data", "build/test/cache");
323     // It's also deprecated
324     System.setProperty("test.cache.data", this.clusterTestDir.toString());
325 
326     // Ok, now we can start
327     this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
328       true, null, null, hosts, null);
329 
330     // Set this just-started cluster as our filesystem.
331     FileSystem fs = this.dfsCluster.getFileSystem();
332     this.conf.set("fs.defaultFS", fs.getUri().toString());
333     // Do old style too just to be safe.
334     this.conf.set("fs.default.name", fs.getUri().toString());
335 
336     // Wait for the cluster to be totally up
337     this.dfsCluster.waitClusterUp();
338 
339     return this.dfsCluster;
340   }
341 
342   /**
343    * Shuts down instance created by call to {@link #startMiniDFSCluster(int)}
344    * or does nothing.
345    * @throws Exception
346    */
347   public void shutdownMiniDFSCluster() throws Exception {
348     if (this.dfsCluster != null) {
349       // The below throws an exception per dn, AsynchronousCloseException.
350       this.dfsCluster.shutdown();
351       dfsCluster = null;
352     }
353 
354   }
355 
356   /**
357    * Call this if you only want a zk cluster.
358    * @see #startMiniZKCluster() if you want zk + dfs + hbase mini cluster.
359    * @throws Exception
360    * @see #shutdownMiniZKCluster()
361    * @return zk cluster started.
362    */
363   public MiniZooKeeperCluster startMiniZKCluster() throws Exception {
364     return startMiniZKCluster(1);
365   }
366   
367   /**
368    * Call this if you only want a zk cluster.
369    * @param zooKeeperServerNum
370    * @see #startMiniZKCluster() if you want zk + dfs + hbase mini cluster.
371    * @throws Exception
372    * @see #shutdownMiniZKCluster()
373    * @return zk cluster started.
374    */
375   public MiniZooKeeperCluster startMiniZKCluster(int zooKeeperServerNum) 
376       throws Exception {
377     File zkClusterFile = new File(getClusterTestDir().toString());
378     return startMiniZKCluster(zkClusterFile, zooKeeperServerNum);
379   }
380   
381   private MiniZooKeeperCluster startMiniZKCluster(final File dir)
382     throws Exception {
383     return startMiniZKCluster(dir,1);
384   }
385   
386   private MiniZooKeeperCluster startMiniZKCluster(final File dir, 
387       int zooKeeperServerNum)
388   throws Exception {
389     if (this.zkCluster != null) {
390       throw new IOException("Cluster already running at " + dir);
391     }
392     this.passedZkCluster = false;
393     this.zkCluster = new MiniZooKeeperCluster(this.getConfiguration());
394     int clientPort =   this.zkCluster.startup(dir,zooKeeperServerNum);
395     this.conf.set("hbase.zookeeper.property.clientPort",
396       Integer.toString(clientPort));
397     return this.zkCluster;
398   }
399 
400   /**
401    * Shuts down zk cluster created by call to {@link #startMiniZKCluster(File)}
402    * or does nothing.
403    * @throws IOException
404    * @see #startMiniZKCluster()
405    */
406   public void shutdownMiniZKCluster() throws IOException {
407     if (this.zkCluster != null) {
408       this.zkCluster.shutdown();
409       this.zkCluster = null;
410     }
411   }
412 
413   /**
414    * Start up a minicluster of hbase, dfs, and zookeeper.
415    * @throws Exception
416    * @return Mini hbase cluster instance created.
417    * @see {@link #shutdownMiniDFSCluster()}
418    */
419   public MiniHBaseCluster startMiniCluster() throws Exception {
420     return startMiniCluster(1, 1);
421   }
422 
423   /**
424    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
425    * Modifies Configuration.  Homes the cluster data directory under a random
426    * subdirectory in a directory under System property test.build.data.
427    * Directory is cleaned up on exit.
428    * @param numSlaves Number of slaves to start up.  We'll start this many
429    * datanodes and regionservers.  If numSlaves is > 1, then make sure
430    * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
431    * bind errors.
432    * @throws Exception
433    * @see {@link #shutdownMiniCluster()}
434    * @return Mini hbase cluster instance created.
435    */
436   public MiniHBaseCluster startMiniCluster(final int numSlaves)
437   throws Exception {
438     return startMiniCluster(1, numSlaves);
439   }
440 
441   
442   /**
443    * start minicluster
444    * @throws Exception
445    * @see {@link #shutdownMiniCluster()}
446    * @return Mini hbase cluster instance created.
447    */
448   public MiniHBaseCluster startMiniCluster(final int numMasters,
449     final int numSlaves)
450   throws Exception {
451     return startMiniCluster(numMasters, numSlaves, null);
452   }
453   
454   
455   /**
456    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
457    * Modifies Configuration.  Homes the cluster data directory under a random
458    * subdirectory in a directory under System property test.build.data.
459    * Directory is cleaned up on exit.
460    * @param numMasters Number of masters to start up.  We'll start this many
461    * hbase masters.  If numMasters > 1, you can find the active/primary master
462    * with {@link MiniHBaseCluster#getMaster()}.
463    * @param numSlaves Number of slaves to start up.  We'll start this many
464    * regionservers. If dataNodeHosts == null, this also indicates the number of
465    * datanodes to start. If dataNodeHosts != null, the number of datanodes is
466    * based on dataNodeHosts.length.
467    * If numSlaves is > 1, then make sure
468    * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
469    * bind errors.
470    * @param dataNodeHosts hostnames DNs to run on.
471    * This is useful if you want to run datanode on distinct hosts for things
472    * like HDFS block location verification.
473    * If you start MiniDFSCluster without host names,
474    * all instances of the datanodes will have the same host name.
475    * @throws Exception
476    * @see {@link #shutdownMiniCluster()}
477    * @return Mini hbase cluster instance created.
478    */
479   public MiniHBaseCluster startMiniCluster(final int numMasters,
480     final int numSlaves, final String[] dataNodeHosts)
481   throws Exception {
482     int numDataNodes = numSlaves;
483     if ( dataNodeHosts != null && dataNodeHosts.length != 0) {
484       numDataNodes = dataNodeHosts.length;
485     }
486     
487     LOG.info("Starting up minicluster with " + numMasters + " master(s) and " +
488         numSlaves + " regionserver(s) and " + numDataNodes + " datanode(s)");
489 
490     // If we already put up a cluster, fail.
491     isRunningCluster();
492 
493     // Bring up mini dfs cluster. This spews a bunch of warnings about missing
494     // scheme. Complaints are 'Scheme is undefined for build/test/data/dfs/name1'.
495     startMiniDFSCluster(numDataNodes, dataNodeHosts);
496 
497     // Start up a zk cluster.
498     if (this.zkCluster == null) {
499       startMiniZKCluster(clusterTestDir);
500     }
501 
502     // Start the MiniHBaseCluster
503     return startMiniHBaseCluster(numMasters, numSlaves);
504   }
505 
506   /**
507    * Starts up mini hbase cluster.  Usually used after call to
508    * {@link #startMiniCluster(int, int)} when doing stepped startup of clusters.
509    * Usually you won't want this.  You'll usually want {@link #startMiniCluster()}.
510    * @param numMasters
511    * @param numSlaves
512    * @return Reference to the hbase mini hbase cluster.
513    * @throws IOException
514    * @throws InterruptedException
515    * @see {@link #startMiniCluster()}
516    */
517   public MiniHBaseCluster startMiniHBaseCluster(final int numMasters,
518       final int numSlaves)
519   throws IOException, InterruptedException {
520     // Now do the mini hbase cluster.  Set the hbase.rootdir in config.
521     createRootDir();
522     Configuration c = new Configuration(this.conf);
523     this.hbaseCluster = new MiniHBaseCluster(c, numMasters, numSlaves);
524     // Don't leave here till we've done a successful scan of the .META.
525     HTable t = new HTable(c, HConstants.META_TABLE_NAME);
526     ResultScanner s = t.getScanner(new Scan());
527     while (s.next() != null) {
528       continue;
529     }
530     LOG.info("Minicluster is up");
531     return this.hbaseCluster;
532   }
533 
534   /**
535    * Starts the hbase cluster up again after shutting it down previously in a
536    * test.  Use this if you want to keep dfs/zk up and just stop/start hbase.
537    * @param servers number of region servers
538    * @throws IOException
539    */
540   public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
541     this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
542     // Don't leave here till we've done a successful scan of the .META.
543     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
544     ResultScanner s = t.getScanner(new Scan());
545     while (s.next() != null) {
546       // do nothing
547     }
548     LOG.info("HBase has been restarted");
549   }
550 
551   /**
552    * @return Current mini hbase cluster. Only has something in it after a call
553    * to {@link #startMiniCluster()}.
554    * @see #startMiniCluster()
555    */
556   public MiniHBaseCluster getMiniHBaseCluster() {
557     return this.hbaseCluster;
558   }
559 
560   /**
561    * Stops mini hbase, zk, and hdfs clusters.
562    * @throws IOException
563    * @see {@link #startMiniCluster(int)}
564    */
565   public void shutdownMiniCluster() throws Exception {
566     LOG.info("Shutting down minicluster");
567     shutdownMiniHBaseCluster();
568     if (!this.passedZkCluster){
569       shutdownMiniZKCluster();
570     }
571     shutdownMiniDFSCluster();
572 
573     // Clean up our directory.
574     if (this.clusterTestDir != null && this.clusterTestDir.exists()) {
575       // Need to use deleteDirectory because File.delete required dir is empty.
576       if (!FSUtils.deleteDirectory(FileSystem.getLocal(this.conf),
577           new Path(this.clusterTestDir.toString()))) {
578         LOG.warn("Failed delete of " + this.clusterTestDir.toString());
579       }
580       this.clusterTestDir = null;
581     }
582     LOG.info("Minicluster is down");
583   }
584 
585   /**
586    * Shutdown HBase mini cluster.  Does not shutdown zk or dfs if running.
587    * @throws IOException
588    */
589   public void shutdownMiniHBaseCluster() throws IOException {
590     if (this.hbaseCluster != null) {
591       this.hbaseCluster.shutdown();
592       // Wait till hbase is down before going on to shutdown zk.
593       this.hbaseCluster.join();
594       this.hbaseCluster = null;
595     }
596   }
597 
598   /**
599    * Creates an hbase rootdir in user home directory.  Also creates hbase
600    * version file.  Normally you won't make use of this method.  Root hbasedir
601    * is created for you as part of mini cluster startup.  You'd only use this
602    * method if you were doing manual operation.
603    * @return Fully qualified path to hbase root dir
604    * @throws IOException
605    */
606   public Path createRootDir() throws IOException {
607     FileSystem fs = FileSystem.get(this.conf);
608     Path hbaseRootdir = fs.makeQualified(fs.getHomeDirectory());
609     this.conf.set(HConstants.HBASE_DIR, hbaseRootdir.toString());
610     fs.mkdirs(hbaseRootdir);
611     FSUtils.setVersion(fs, hbaseRootdir);
612     return hbaseRootdir;
613   }
614 
615   /**
616    * Flushes all caches in the mini hbase cluster
617    * @throws IOException
618    */
619   public void flush() throws IOException {
620     this.hbaseCluster.flushcache();
621   }
622 
623   /**
624    * Flushes all caches in the mini hbase cluster
625    * @throws IOException
626    */
627   public void flush(byte [] tableName) throws IOException {
628     this.hbaseCluster.flushcache(tableName);
629   }
630 
631 
632   /**
633    * Create a table.
634    * @param tableName
635    * @param family
636    * @return An HTable instance for the created table.
637    * @throws IOException
638    */
639   public HTable createTable(byte[] tableName, byte[] family)
640   throws IOException{
641     return createTable(tableName, new byte[][]{family});
642   }
643 
644   /**
645    * Create a table.
646    * @param tableName
647    * @param families
648    * @return An HTable instance for the created table.
649    * @throws IOException
650    */
651   public HTable createTable(byte[] tableName, byte[][] families)
652   throws IOException {
653     return createTable(tableName, families,
654         new Configuration(getConfiguration()));
655   }
656 
657   /**
658    * Create a table.
659    * @param tableName
660    * @param families
661    * @param c Configuration to use
662    * @return An HTable instance for the created table.
663    * @throws IOException
664    */
665   public HTable createTable(byte[] tableName, byte[][] families,
666       final Configuration c)
667   throws IOException {
668     HTableDescriptor desc = new HTableDescriptor(tableName);
669     for(byte[] family : families) {
670       desc.addFamily(new HColumnDescriptor(family));
671     }
672     getHBaseAdmin().createTable(desc);
673     return new HTable(c, tableName);
674   }
675 
676   /**
677    * Create a table.
678    * @param tableName
679    * @param families
680    * @param c Configuration to use
681    * @param numVersions
682    * @return An HTable instance for the created table.
683    * @throws IOException
684    */
685   public HTable createTable(byte[] tableName, byte[][] families,
686       final Configuration c, int numVersions)
687   throws IOException {
688     HTableDescriptor desc = new HTableDescriptor(tableName);
689     for(byte[] family : families) {
690       HColumnDescriptor hcd = new HColumnDescriptor(family, numVersions,
691           HColumnDescriptor.DEFAULT_COMPRESSION,
692           HColumnDescriptor.DEFAULT_IN_MEMORY,
693           HColumnDescriptor.DEFAULT_BLOCKCACHE,
694           HColumnDescriptor.DEFAULT_BLOCKSIZE, HColumnDescriptor.DEFAULT_TTL,
695           HColumnDescriptor.DEFAULT_BLOOMFILTER,
696           HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
697       desc.addFamily(hcd);
698     }
699     getHBaseAdmin().createTable(desc);
700     return new HTable(c, tableName);
701   }
702 
703   /**
704    * Create a table.
705    * @param tableName
706    * @param family
707    * @param numVersions
708    * @return An HTable instance for the created table.
709    * @throws IOException
710    */
711   public HTable createTable(byte[] tableName, byte[] family, int numVersions)
712   throws IOException {
713     return createTable(tableName, new byte[][]{family}, numVersions);
714   }
715 
716   /**
717    * Create a table.
718    * @param tableName
719    * @param families
720    * @param numVersions
721    * @return An HTable instance for the created table.
722    * @throws IOException
723    */
724   public HTable createTable(byte[] tableName, byte[][] families,
725       int numVersions)
726   throws IOException {
727     HTableDescriptor desc = new HTableDescriptor(tableName);
728     for (byte[] family : families) {
729       HColumnDescriptor hcd = new HColumnDescriptor(family, numVersions,
730           HColumnDescriptor.DEFAULT_COMPRESSION,
731           HColumnDescriptor.DEFAULT_IN_MEMORY,
732           HColumnDescriptor.DEFAULT_BLOCKCACHE,
733           HColumnDescriptor.DEFAULT_BLOCKSIZE, HColumnDescriptor.DEFAULT_TTL,
734           HColumnDescriptor.DEFAULT_BLOOMFILTER,
735           HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
736       desc.addFamily(hcd);
737     }
738     getHBaseAdmin().createTable(desc);
739     return new HTable(new Configuration(getConfiguration()), tableName);
740   }
741 
742   /**
743    * Create a table.
744    * @param tableName
745    * @param families
746    * @param numVersions
747    * @return An HTable instance for the created table.
748    * @throws IOException
749    */
750   public HTable createTable(byte[] tableName, byte[][] families,
751     int numVersions, int blockSize) throws IOException {
752     HTableDescriptor desc = new HTableDescriptor(tableName);
753     for (byte[] family : families) {
754       HColumnDescriptor hcd = new HColumnDescriptor(family, numVersions,
755           HColumnDescriptor.DEFAULT_COMPRESSION,
756           HColumnDescriptor.DEFAULT_IN_MEMORY,
757           HColumnDescriptor.DEFAULT_BLOCKCACHE,
758           blockSize, HColumnDescriptor.DEFAULT_TTL,
759           HColumnDescriptor.DEFAULT_BLOOMFILTER,
760           HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
761       desc.addFamily(hcd);
762     }
763     getHBaseAdmin().createTable(desc);
764     return new HTable(new Configuration(getConfiguration()), tableName);
765   }
766 
767   /**
768    * Create a table.
769    * @param tableName
770    * @param families
771    * @param numVersions
772    * @return An HTable instance for the created table.
773    * @throws IOException
774    */
775   public HTable createTable(byte[] tableName, byte[][] families,
776       int[] numVersions)
777   throws IOException {
778     HTableDescriptor desc = new HTableDescriptor(tableName);
779     int i = 0;
780     for (byte[] family : families) {
781       HColumnDescriptor hcd = new HColumnDescriptor(family, numVersions[i],
782           HColumnDescriptor.DEFAULT_COMPRESSION,
783           HColumnDescriptor.DEFAULT_IN_MEMORY,
784           HColumnDescriptor.DEFAULT_BLOCKCACHE,
785           HColumnDescriptor.DEFAULT_BLOCKSIZE, HColumnDescriptor.DEFAULT_TTL,
786           HColumnDescriptor.DEFAULT_BLOOMFILTER,
787           HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
788       desc.addFamily(hcd);
789       i++;
790     }
791     getHBaseAdmin().createTable(desc);
792     return new HTable(new Configuration(getConfiguration()), tableName);
793   }
794 
795   /**
796    * Drop an existing table
797    * @param tableName existing table
798    */
799   public void deleteTable(byte[] tableName) throws IOException {
800     HBaseAdmin admin = new HBaseAdmin(getConfiguration());
801     admin.disableTable(tableName);
802     admin.deleteTable(tableName);
803   }
804 
805   /**
806    * Provide an existing table name to truncate
807    * @param tableName existing table
808    * @return HTable to that new table
809    * @throws IOException
810    */
811   public HTable truncateTable(byte [] tableName) throws IOException {
812     HTable table = new HTable(getConfiguration(), tableName);
813     Scan scan = new Scan();
814     ResultScanner resScan = table.getScanner(scan);
815     for(Result res : resScan) {
816       Delete del = new Delete(res.getRow());
817       table.delete(del);
818     }
819     resScan = table.getScanner(scan);
820     return table;
821   }
822 
823   /**
824    * Load table with rows from 'aaa' to 'zzz'.
825    * @param t Table
826    * @param f Family
827    * @return Count of rows loaded.
828    * @throws IOException
829    */
830   public int loadTable(final HTable t, final byte[] f) throws IOException {
831     t.setAutoFlush(false);
832     byte[] k = new byte[3];
833     int rowCount = 0;
834     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
835       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
836         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
837           k[0] = b1;
838           k[1] = b2;
839           k[2] = b3;
840           Put put = new Put(k);
841           put.add(f, null, k);
842           t.put(put);
843           rowCount++;
844         }
845       }
846     }
847     t.flushCommits();
848     return rowCount;
849   }
850   /**
851    * Load region with rows from 'aaa' to 'zzz'.
852    * @param r Region
853    * @param f Family
854    * @return Count of rows loaded.
855    * @throws IOException
856    */
857   public int loadRegion(final HRegion r, final byte[] f)
858   throws IOException {
859     byte[] k = new byte[3];
860     int rowCount = 0;
861     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
862       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
863         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
864           k[0] = b1;
865           k[1] = b2;
866           k[2] = b3;
867           Put put = new Put(k);
868           put.add(f, null, k);
869           if (r.getLog() == null) put.setWriteToWAL(false);
870           r.put(put);
871           rowCount++;
872         }
873       }
874     }
875     return rowCount;
876   }
877 
878   /**
879    * Return the number of rows in the given table.
880    */
881   public int countRows(final HTable table) throws IOException {
882     Scan scan = new Scan();
883     ResultScanner results = table.getScanner(scan);
884     int count = 0;
885     for (@SuppressWarnings("unused") Result res : results) {
886       count++;
887     }
888     results.close();
889     return count;
890   }
891 
892   /**
893    * Return an md5 digest of the entire contents of a table.
894    */
895   public String checksumRows(final HTable table) throws Exception {
896     Scan scan = new Scan();
897     ResultScanner results = table.getScanner(scan);
898     MessageDigest digest = MessageDigest.getInstance("MD5");
899     for (Result res : results) {
900       digest.update(res.getRow());
901     }
902     results.close();
903     return digest.toString();
904   }
905 
906   /**
907    * Creates many regions names "aaa" to "zzz".
908    *
909    * @param table  The table to use for the data.
910    * @param columnFamily  The family to insert the data into.
911    * @return count of regions created.
912    * @throws IOException When creating the regions fails.
913    */
914   public int createMultiRegions(HTable table, byte[] columnFamily)
915   throws IOException {
916     return createMultiRegions(getConfiguration(), table, columnFamily);
917   }
918 
919   public static final byte[][] KEYS = {
920     HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"),
921     Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
922     Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
923     Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
924     Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
925     Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
926     Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
927     Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
928     Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
929   };
930 
931   /**
932    * Creates many regions names "aaa" to "zzz".
933    * @param c Configuration to use.
934    * @param table  The table to use for the data.
935    * @param columnFamily  The family to insert the data into.
936    * @return count of regions created.
937    * @throws IOException When creating the regions fails.
938    */
939   public int createMultiRegions(final Configuration c, final HTable table,
940       final byte[] columnFamily)
941   throws IOException {
942     return createMultiRegions(c, table, columnFamily, KEYS);
943   }
944 
945   /**
946    * Creates the specified number of regions in the specified table.
947    * @param c
948    * @param table
949    * @param family
950    * @param numRegions
951    * @return
952    * @throws IOException
953    */
954   public int createMultiRegions(final Configuration c, final HTable table,
955       final byte [] family, int numRegions)
956   throws IOException {
957     if (numRegions < 3) throw new IOException("Must create at least 3 regions");
958     byte [] startKey = Bytes.toBytes("aaaaa");
959     byte [] endKey = Bytes.toBytes("zzzzz");
960     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
961     byte [][] regionStartKeys = new byte[splitKeys.length+1][];
962     for (int i=0;i<splitKeys.length;i++) {
963       regionStartKeys[i+1] = splitKeys[i];
964     }
965     regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
966     return createMultiRegions(c, table, family, regionStartKeys);
967   }
968 
969   public int createMultiRegions(final Configuration c, final HTable table,
970       final byte[] columnFamily, byte [][] startKeys)
971   throws IOException {
972     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
973     HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
974     HTableDescriptor htd = table.getTableDescriptor();
975     if(!htd.hasFamily(columnFamily)) {
976       HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
977       htd.addFamily(hcd);
978     }
979     // remove empty region - this is tricky as the mini cluster during the test
980     // setup already has the "<tablename>,,123456789" row with an empty start
981     // and end key. Adding the custom regions below adds those blindly,
982     // including the new start region from empty to "bbb". lg
983     List<byte[]> rows = getMetaTableRows(htd.getName());
984     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
985     // add custom ones
986     int count = 0;
987     for (int i = 0; i < startKeys.length; i++) {
988       int j = (i + 1) % startKeys.length;
989       HRegionInfo hri = new HRegionInfo(table.getTableName(),
990         startKeys[i], startKeys[j]);
991       Put put = new Put(hri.getRegionName());
992       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
993         Writables.getBytes(hri));
994       meta.put(put);
995       LOG.info("createMultiRegions: inserted " + hri.toString());
996       newRegions.add(hri);
997       count++;
998     }
999     // see comment above, remove "old" (or previous) single region
1000     for (byte[] row : rows) {
1001       LOG.info("createMultiRegions: deleting meta row -> " +
1002         Bytes.toStringBinary(row));
1003       meta.delete(new Delete(row));
1004     }
1005     // flush cache of regions
1006     HConnection conn = table.getConnection();
1007     conn.clearRegionCache();
1008     // assign all the new regions IF table is enabled.
1009     if (getHBaseAdmin().isTableEnabled(table.getTableName())) {
1010       for(HRegionInfo hri : newRegions) {
1011         hbaseCluster.getMaster().assignRegion(hri);
1012       }
1013     }
1014     return count;
1015   }
1016 
1017   /**
1018    * Create rows in META for regions of the specified table with the specified
1019    * start keys.  The first startKey should be a 0 length byte array if you
1020    * want to form a proper range of regions.
1021    * @param conf
1022    * @param htd
1023    * @param startKeys
1024    * @return list of region info for regions added to meta
1025    * @throws IOException
1026    */
1027   public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
1028       final HTableDescriptor htd, byte [][] startKeys)
1029   throws IOException {
1030     HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
1031     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
1032     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
1033     // add custom ones
1034     for (int i = 0; i < startKeys.length; i++) {
1035       int j = (i + 1) % startKeys.length;
1036       HRegionInfo hri = new HRegionInfo(htd.getName(), startKeys[i],
1037           startKeys[j]);
1038       Put put = new Put(hri.getRegionName());
1039       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
1040         Writables.getBytes(hri));
1041       meta.put(put);
1042       LOG.info("createMultiRegionsInMeta: inserted " + hri.toString());
1043       newRegions.add(hri);
1044     }
1045     return newRegions;
1046   }
1047 
1048   /**
1049    * Returns all rows from the .META. table.
1050    *
1051    * @throws IOException When reading the rows fails.
1052    */
1053   public List<byte[]> getMetaTableRows() throws IOException {
1054     // TODO: Redo using MetaReader class
1055     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
1056     List<byte[]> rows = new ArrayList<byte[]>();
1057     ResultScanner s = t.getScanner(new Scan());
1058     for (Result result : s) {
1059       LOG.info("getMetaTableRows: row -> " +
1060         Bytes.toStringBinary(result.getRow()));
1061       rows.add(result.getRow());
1062     }
1063     s.close();
1064     return rows;
1065   }
1066 
1067   /**
1068    * Returns all rows from the .META. table for a given user table
1069    *
1070    * @throws IOException When reading the rows fails.
1071    */
1072   public List<byte[]> getMetaTableRows(byte[] tableName) throws IOException {
1073     // TODO: Redo using MetaReader.
1074     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
1075     List<byte[]> rows = new ArrayList<byte[]>();
1076     ResultScanner s = t.getScanner(new Scan());
1077     for (Result result : s) {
1078       HRegionInfo info = Writables.getHRegionInfo(
1079           result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
1080       if (Bytes.compareTo(info.getTableName(), tableName) == 0) {
1081         LOG.info("getMetaTableRows: row -> " +
1082             Bytes.toStringBinary(result.getRow()));
1083         rows.add(result.getRow());
1084       }
1085     }
1086     s.close();
1087     return rows;
1088   }
1089 
1090   /**
1091    * Tool to get the reference to the region server object that holds the
1092    * region of the specified user table.
1093    * It first searches for the meta rows that contain the region of the
1094    * specified table, then gets the index of that RS, and finally retrieves
1095    * the RS's reference.
1096    * @param tableName user table to lookup in .META.
1097    * @return region server that holds it, null if the row doesn't exist
1098    * @throws IOException
1099    */
1100   public HRegionServer getRSForFirstRegionInTable(byte[] tableName)
1101       throws IOException {
1102     List<byte[]> metaRows = getMetaTableRows(tableName);
1103     if (metaRows == null || metaRows.isEmpty()) {
1104       return null;
1105     }
1106     LOG.debug("Found " + metaRows.size() + " rows for table " +
1107       Bytes.toString(tableName));
1108     byte [] firstrow = metaRows.get(0);
1109     LOG.debug("FirstRow=" + Bytes.toString(firstrow));
1110     int index = hbaseCluster.getServerWith(firstrow);
1111     return hbaseCluster.getRegionServerThreads().get(index).getRegionServer();
1112   }
1113 
1114   /**
1115    * Starts a <code>MiniMRCluster</code> with a default number of
1116    * <code>TaskTracker</code>'s.
1117    *
1118    * @throws IOException When starting the cluster fails.
1119    */
1120   public void startMiniMapReduceCluster() throws IOException {
1121     startMiniMapReduceCluster(2);
1122   }
1123 
1124   /**
1125    * Starts a <code>MiniMRCluster</code>.
1126    *
1127    * @param servers  The number of <code>TaskTracker</code>'s to start.
1128    * @throws IOException When starting the cluster fails.
1129    */
1130   public void startMiniMapReduceCluster(final int servers) throws IOException {
1131     LOG.info("Starting mini mapreduce cluster...");
1132     // These are needed for the new and improved Map/Reduce framework
1133     Configuration c = getConfiguration();
1134     System.setProperty("hadoop.log.dir", c.get("hadoop.log.dir"));
1135     c.set("mapred.output.dir", c.get("hadoop.tmp.dir"));
1136     mrCluster = new MiniMRCluster(servers,
1137       FileSystem.get(c).getUri().toString(), 1);
1138     LOG.info("Mini mapreduce cluster started");
1139     c.set("mapred.job.tracker",
1140         mrCluster.createJobConf().get("mapred.job.tracker"));
1141   }
1142 
1143   /**
1144    * Stops the previously started <code>MiniMRCluster</code>.
1145    */
1146   public void shutdownMiniMapReduceCluster() {
1147     LOG.info("Stopping mini mapreduce cluster...");
1148     if (mrCluster != null) {
1149       mrCluster.shutdown();
1150       mrCluster = null;
1151     }
1152     // Restore configuration to point to local jobtracker
1153     conf.set("mapred.job.tracker", "local");
1154     LOG.info("Mini mapreduce cluster stopped");
1155   }
1156 
1157   /**
1158    * Switches the logger for the given class to DEBUG level.
1159    *
1160    * @param clazz  The class for which to switch to debug logging.
1161    */
1162   public void enableDebug(Class<?> clazz) {
1163     Log l = LogFactory.getLog(clazz);
1164     if (l instanceof Log4JLogger) {
1165       ((Log4JLogger) l).getLogger().setLevel(org.apache.log4j.Level.DEBUG);
1166     } else if (l instanceof Jdk14Logger) {
1167       ((Jdk14Logger) l).getLogger().setLevel(java.util.logging.Level.ALL);
1168     }
1169   }
1170 
1171   /**
1172    * Expire the Master's session
1173    * @throws Exception
1174    */
1175   public void expireMasterSession() throws Exception {
1176     HMaster master = hbaseCluster.getMaster();
1177     expireSession(master.getZooKeeper(), master);
1178   }
1179 
1180   /**
1181    * Expire a region server's session
1182    * @param index which RS
1183    * @throws Exception
1184    */
1185   public void expireRegionServerSession(int index) throws Exception {
1186     HRegionServer rs = hbaseCluster.getRegionServer(index);
1187     expireSession(rs.getZooKeeper(), rs);
1188   }
1189 
1190   public void expireSession(ZooKeeperWatcher nodeZK, Server server)
1191     throws Exception {
1192     expireSession(nodeZK, server, false);
1193   }
1194 
1195   public void expireSession(ZooKeeperWatcher nodeZK, Server server,
1196       boolean checkStatus) throws Exception {
1197     Configuration c = new Configuration(this.conf);
1198     String quorumServers = ZKConfig.getZKQuorumServersString(c);
1199     int sessionTimeout = 5 * 1000; // 5 seconds
1200     ZooKeeper zk = nodeZK.getRecoverableZooKeeper().getZooKeeper();
1201     byte[] password = zk.getSessionPasswd();
1202     long sessionID = zk.getSessionId();
1203 
1204     ZooKeeper newZK = new ZooKeeper(quorumServers,
1205         sessionTimeout, EmptyWatcher.instance, sessionID, password);
1206     newZK.close();
1207     final long sleep = sessionTimeout * 5L;
1208     LOG.info("ZK Closed Session 0x" + Long.toHexString(sessionID) +
1209       "; sleeping=" + sleep);
1210 
1211     Thread.sleep(sleep);
1212 
1213     if (checkStatus) {
1214       new HTable(new Configuration(conf), HConstants.META_TABLE_NAME);
1215     }
1216   }
1217 
1218 
1219   /**
1220    * Get the HBase cluster.
1221    *
1222    * @return hbase cluster
1223    */
1224   public MiniHBaseCluster getHBaseCluster() {
1225     return hbaseCluster;
1226   }
1227 
1228   /**
1229    * Returns a HBaseAdmin instance.
1230    *
1231    * @return The HBaseAdmin instance.
1232    * @throws IOException
1233    */
1234   public HBaseAdmin getHBaseAdmin()
1235   throws IOException {
1236     return new HBaseAdmin(new Configuration(getConfiguration()));
1237   }
1238 
1239   /**
1240    * Closes the named region.
1241    *
1242    * @param regionName  The region to close.
1243    * @throws IOException
1244    */
1245   public void closeRegion(String regionName) throws IOException {
1246     closeRegion(Bytes.toBytes(regionName));
1247   }
1248 
1249   /**
1250    * Closes the named region.
1251    *
1252    * @param regionName  The region to close.
1253    * @throws IOException
1254    */
1255   public void closeRegion(byte[] regionName) throws IOException {
1256     HBaseAdmin admin = getHBaseAdmin();
1257     admin.closeRegion(regionName, null);
1258   }
1259 
1260   /**
1261    * Closes the region containing the given row.
1262    *
1263    * @param row  The row to find the containing region.
1264    * @param table  The table to find the region.
1265    * @throws IOException
1266    */
1267   public void closeRegionByRow(String row, HTable table) throws IOException {
1268     closeRegionByRow(Bytes.toBytes(row), table);
1269   }
1270 
1271   /**
1272    * Closes the region containing the given row.
1273    *
1274    * @param row  The row to find the containing region.
1275    * @param table  The table to find the region.
1276    * @throws IOException
1277    */
1278   public void closeRegionByRow(byte[] row, HTable table) throws IOException {
1279     HRegionLocation hrl = table.getRegionLocation(row);
1280     closeRegion(hrl.getRegionInfo().getRegionName());
1281   }
1282 
1283   public MiniZooKeeperCluster getZkCluster() {
1284     return zkCluster;
1285   }
1286 
1287   public void setZkCluster(MiniZooKeeperCluster zkCluster) {
1288     this.passedZkCluster = true;
1289     this.zkCluster = zkCluster;
1290   }
1291 
1292   public MiniDFSCluster getDFSCluster() {
1293     return dfsCluster;
1294   }
1295 
1296   public FileSystem getTestFileSystem() throws IOException {
1297     return FileSystem.get(conf);
1298   }
1299 
1300   /**
1301    * @return True if we removed the test dir
1302    * @throws IOException
1303    */
1304   public boolean cleanupTestDir() throws IOException {
1305     if (dataTestDir == null ){
1306       return false;
1307     } else {
1308       boolean ret = deleteDir(getDataTestDir());
1309       dataTestDir = null;
1310       return ret;
1311     }
1312   }
1313 
1314   /**
1315    * @param subdir Test subdir name.
1316    * @return True if we removed the test dir
1317    * @throws IOException
1318    */
1319   public boolean cleanupTestDir(final String subdir) throws IOException {
1320     if (dataTestDir == null){
1321       return false;
1322     }
1323     return deleteDir(getDataTestDir(subdir));
1324   }
1325 
1326   /**
1327    * @param dir Directory to delete
1328    * @return True if we deleted it.
1329    * @throws IOException
1330    */
1331   public boolean deleteDir(final Path dir) throws IOException {
1332     FileSystem fs = getTestFileSystem();
1333     if (fs.exists(dir)) {
1334       return fs.delete(getDataTestDir(), true);
1335     }
1336     return false;
1337   }
1338 
1339   public void waitTableAvailable(byte[] table, long timeoutMillis)
1340   throws InterruptedException, IOException {
1341     HBaseAdmin admin = getHBaseAdmin();
1342     long startWait = System.currentTimeMillis();
1343     while (!admin.isTableAvailable(table)) {
1344       assertTrue("Timed out waiting for table " + Bytes.toStringBinary(table),
1345           System.currentTimeMillis() - startWait < timeoutMillis);
1346       Thread.sleep(200);
1347     }
1348   }
1349 
1350   /**
1351    * Make sure that at least the specified number of region servers
1352    * are running
1353    * @param num minimum number of region servers that should be running
1354    * @return True if we started some servers
1355    * @throws IOException
1356    */
1357   public boolean ensureSomeRegionServersAvailable(final int num)
1358       throws IOException {
1359     boolean startedServer = false;
1360 
1361     for (int i=hbaseCluster.getLiveRegionServerThreads().size(); i<num; ++i){
1362       LOG.info("Started new server=" + hbaseCluster.startRegionServer());
1363       startedServer = true;
1364     }
1365 
1366     return startedServer;
1367   }
1368 
1369 
1370 
1371 
1372   /**
1373    * This method clones the passed <code>c</code> configuration setting a new
1374    * user into the clone.  Use it getting new instances of FileSystem.  Only
1375    * works for DistributedFileSystem.
1376    * @param c Initial configuration
1377    * @param differentiatingSuffix Suffix to differentiate this user from others.
1378    * @return A new configuration instance with a different user set into it.
1379    * @throws IOException
1380    */
1381   public static User getDifferentUser(final Configuration c,
1382     final String differentiatingSuffix)
1383   throws IOException {
1384     FileSystem currentfs = FileSystem.get(c);
1385     if (!(currentfs instanceof DistributedFileSystem)) {
1386       return User.getCurrent();
1387     }
1388     // Else distributed filesystem.  Make a new instance per daemon.  Below
1389     // code is taken from the AppendTestUtil over in hdfs.
1390     String username = User.getCurrent().getName() +
1391       differentiatingSuffix;
1392     User user = User.createUserForTesting(c, username,
1393         new String[]{"supergroup"});
1394     return user;
1395   }
1396 
1397   /**
1398    * Set maxRecoveryErrorCount in DFSClient.  In 0.20 pre-append its hard-coded to 5 and
1399    * makes tests linger.  Here is the exception you'll see:
1400    * <pre>
1401    * 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...
1402    * </pre>
1403    * @param stream A DFSClient.DFSOutputStream.
1404    * @param max
1405    * @throws NoSuchFieldException
1406    * @throws SecurityException
1407    * @throws IllegalAccessException
1408    * @throws IllegalArgumentException
1409    */
1410   public static void setMaxRecoveryErrorCount(final OutputStream stream,
1411       final int max) {
1412     try {
1413       Class<?> [] clazzes = DFSClient.class.getDeclaredClasses();
1414       for (Class<?> clazz: clazzes) {
1415         String className = clazz.getSimpleName();
1416         if (className.equals("DFSOutputStream")) {
1417           if (clazz.isInstance(stream)) {
1418             Field maxRecoveryErrorCountField =
1419               stream.getClass().getDeclaredField("maxRecoveryErrorCount");
1420             maxRecoveryErrorCountField.setAccessible(true);
1421             maxRecoveryErrorCountField.setInt(stream, max);
1422             break;
1423           }
1424         }
1425       }
1426     } catch (Exception e) {
1427       LOG.info("Could not set max recovery field", e);
1428     }
1429   }
1430 
1431 
1432   /**
1433    * Wait until <code>countOfRegion</code> in .META. have a non-empty
1434    * info:server.  This means all regions have been deployed, master has been
1435    * informed and updated .META. with the regions deployed server.
1436    * @param countOfRegions How many regions in .META.
1437    * @throws IOException
1438    */
1439   public void waitUntilAllRegionsAssigned(final int countOfRegions)
1440   throws IOException {
1441     HTable meta = new HTable(getConfiguration(), HConstants.META_TABLE_NAME);
1442     while (true) {
1443       int rows = 0;
1444       Scan scan = new Scan();
1445       scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
1446       ResultScanner s = meta.getScanner(scan);
1447       for (Result r = null; (r = s.next()) != null;) {
1448         byte [] b =
1449           r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
1450         if (b == null || b.length <= 0) {
1451           break;
1452         }
1453         rows++;
1454       }
1455       s.close();
1456       // If I get to here and all rows have a Server, then all have been assigned.
1457       if (rows == countOfRegions) {
1458         break;
1459       }
1460       LOG.info("Found=" + rows);
1461       Threads.sleep(200);
1462     }
1463   }
1464 
1465   /**
1466    * Do a small get/scan against one store. This is required because store
1467    * has no actual methods of querying itself, and relies on StoreScanner.
1468    */
1469   public static List<KeyValue> getFromStoreFile(Store store,
1470                                                 Get get) throws IOException {
1471     MultiVersionConsistencyControl.resetThreadReadPoint();
1472     Scan scan = new Scan(get);
1473     InternalScanner scanner = (InternalScanner) store.getScanner(scan,
1474         scan.getFamilyMap().get(store.getFamily().getName()));
1475 
1476     List<KeyValue> result = new ArrayList<KeyValue>();
1477     scanner.next(result);
1478     if (!result.isEmpty()) {
1479       // verify that we are on the row we want:
1480       KeyValue kv = result.get(0);
1481       if (!Bytes.equals(kv.getRow(), get.getRow())) {
1482         result.clear();
1483       }
1484     }
1485     return result;
1486   }
1487 
1488   /**
1489    * Do a small get/scan against one store. This is required because store
1490    * has no actual methods of querying itself, and relies on StoreScanner.
1491    */
1492   public static List<KeyValue> getFromStoreFile(Store store,
1493                                                 byte [] row,
1494                                                 NavigableSet<byte[]> columns
1495                                                 ) throws IOException {
1496     Get get = new Get(row);
1497     Map<byte[], NavigableSet<byte[]>> s = get.getFamilyMap();
1498     s.put(store.getFamily().getName(), columns);
1499 
1500     return getFromStoreFile(store,get);
1501   }
1502   
1503   /**
1504    * Gets a ZooKeeperWatcher.
1505    * @param TEST_UTIL
1506    */
1507   public static ZooKeeperWatcher getZooKeeperWatcher(
1508       HBaseTestingUtility TEST_UTIL) throws ZooKeeperConnectionException,
1509       IOException {
1510     ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
1511         "unittest", new Abortable() {
1512           boolean aborted = false;
1513 
1514           @Override
1515           public void abort(String why, Throwable e) {
1516             aborted = true;
1517             throw new RuntimeException("Fatal ZK error, why=" + why, e);
1518           }
1519 
1520           @Override
1521           public boolean isAborted() {
1522             return aborted;
1523           }
1524         });
1525     return zkw;
1526   }
1527   
1528   /**
1529    * Creates a znode with OPENED state.
1530    * @param TEST_UTIL
1531    * @param region
1532    * @param serverName
1533    * @return
1534    * @throws IOException
1535    * @throws ZooKeeperConnectionException
1536    * @throws KeeperException
1537    * @throws NodeExistsException
1538    */
1539   public static ZooKeeperWatcher createAndForceNodeToOpenedState(
1540       HBaseTestingUtility TEST_UTIL, HRegion region,
1541       ServerName serverName) throws ZooKeeperConnectionException,
1542       IOException, KeeperException, NodeExistsException {
1543     ZooKeeperWatcher zkw = getZooKeeperWatcher(TEST_UTIL);
1544     ZKAssign.createNodeOffline(zkw, region.getRegionInfo(), serverName);
1545     int version = ZKAssign.transitionNodeOpening(zkw, region
1546         .getRegionInfo(), serverName);
1547     ZKAssign.transitionNodeOpened(zkw, region.getRegionInfo(), serverName,
1548         version);
1549     return zkw;
1550   }
1551   
1552 }