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.master.HMaster;
53  import org.apache.hadoop.hbase.regionserver.HRegion;
54  import org.apache.hadoop.hbase.regionserver.HRegionServer;
55  import org.apache.hadoop.hbase.regionserver.InternalScanner;
56  import org.apache.hadoop.hbase.regionserver.ReadWriteConsistencyControl;
57  import org.apache.hadoop.hbase.regionserver.Store;
58  import org.apache.hadoop.hbase.util.Bytes;
59  import org.apache.hadoop.hbase.util.FSUtils;
60  import org.apache.hadoop.hbase.util.Threads;
61  import org.apache.hadoop.hbase.util.Writables;
62  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
63  import org.apache.hadoop.hdfs.DFSClient;
64  import org.apache.hadoop.hdfs.DistributedFileSystem;
65  import org.apache.hadoop.hdfs.MiniDFSCluster;
66  import org.apache.hadoop.hdfs.server.namenode.NameNode;
67  import org.apache.hadoop.mapred.MiniMRCluster;
68  import org.apache.hadoop.security.UserGroupInformation;
69  import org.apache.zookeeper.ZooKeeper;
70  
71  import com.google.common.base.Preconditions;
72  
73  /**
74   * Facility for testing HBase. Added as tool to abet junit4 testing.  Replaces
75   * old HBaseTestCase and HBaseCluserTestCase functionality.
76   * Create an instance and keep it around doing HBase testing.  This class is
77   * meant to be your one-stop shop for anything you might need testing.  Manages
78   * one cluster at a time only.  Depends on log4j being on classpath and
79   * hbase-site.xml for logging and test-run configuration.  It does not set
80   * logging levels nor make changes to configuration parameters.
81   */
82  public class HBaseTestingUtility {
83    private final static Log LOG = LogFactory.getLog(HBaseTestingUtility.class);
84    private final Configuration conf;
85    private MiniZooKeeperCluster zkCluster = null;
86    private MiniDFSCluster dfsCluster = null;
87    private MiniHBaseCluster hbaseCluster = null;
88    private MiniMRCluster mrCluster = null;
89    // If non-null, then already a cluster running.
90    private File clusterTestBuildDir = null;
91    private HBaseAdmin hbaseAdmin = null;
92  
93    /**
94     * System property key to get test directory value.
95     */
96    public static final String TEST_DIRECTORY_KEY = "test.build.data";
97  
98    /**
99     * Default parent direccounttory for test output.
100    */
101   public static final String DEFAULT_TEST_DIRECTORY = "target/build/data";
102 
103   public HBaseTestingUtility() {
104     this(HBaseConfiguration.create());
105   }
106 
107   public HBaseTestingUtility(Configuration conf) {
108     this.conf = conf;
109   }
110 
111   /**
112    * @return Instance of Configuration.
113    */
114   public Configuration getConfiguration() {
115     return this.conf;
116   }
117 
118   /**
119    * @return Where to write test data on local filesystem; usually
120    * {@link #DEFAULT_TEST_DIRECTORY}
121    * @see #setupClusterTestBuildDir()
122    */
123   public static Path getTestDir() {
124     return new Path(System.getProperty(TEST_DIRECTORY_KEY,
125       DEFAULT_TEST_DIRECTORY));
126   }
127 
128   /**
129    * @param subdirName
130    * @return Path to a subdirectory named <code>subdirName</code> under
131    * {@link #getTestDir()}.
132    * @see #setupClusterTestBuildDir()
133    */
134   public static Path getTestDir(final String subdirName) {
135     return new Path(getTestDir(), subdirName);
136   }
137 
138   /**
139    * Home our cluster in a dir under target/test.  Give it a random name
140    * so can have many concurrent clusters running if we need to.  Need to
141    * amend the test.build.data System property.  Its what minidfscluster bases
142    * it data dir on.  Moding a System property is not the way to do concurrent
143    * instances -- another instance could grab the temporary
144    * value unintentionally -- but not anything can do about it at moment;
145    * single instance only is how the minidfscluster works.
146    * @return The calculated cluster test build directory.
147    */
148   File setupClusterTestBuildDir() {
149     String randomStr = UUID.randomUUID().toString();
150     String dirStr = getTestDir(randomStr).toString();
151     File dir = new File(dirStr).getAbsoluteFile();
152     // Have it cleaned up on exit
153     dir.deleteOnExit();
154     return dir;
155   }
156 
157   /**
158    * @throws IOException If a cluster -- zk, dfs, or hbase -- already running.
159    */
160   void isRunningCluster() throws IOException {
161     if (this.clusterTestBuildDir == null) return;
162     throw new IOException("Cluster already running at " +
163       this.clusterTestBuildDir);
164   }
165 
166   /**
167    * Start a minidfscluster.
168    * @param servers How many DNs to start.
169    * @throws Exception
170    * @see {@link #shutdownMiniDFSCluster()}
171    * @return The mini dfs cluster created.
172    */
173   public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception {
174     return startMiniDFSCluster(servers, null);
175   }
176 
177   /**
178    * Start a minidfscluster.
179    * Can only create one.
180    * @param dir Where to home your dfs cluster.
181    * @param servers How many DNs to start.
182    * @throws Exception
183    * @see {@link #shutdownMiniDFSCluster()}
184    * @return The mini dfs cluster created.
185    */
186   public MiniDFSCluster startMiniDFSCluster(int servers, final File dir)
187   throws Exception {
188     // This does the following to home the minidfscluster
189     //     base_dir = new File(System.getProperty("test.build.data", "build/test/data"), "dfs/");
190     // Some tests also do this:
191     //  System.getProperty("test.cache.data", "build/test/cache");
192     if (dir == null) this.clusterTestBuildDir = setupClusterTestBuildDir();
193     else this.clusterTestBuildDir = dir;
194     System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestBuildDir.toString());
195     System.setProperty("test.cache.data", this.clusterTestBuildDir.toString());
196     this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
197       true, null, null, null, null);
198     return this.dfsCluster;
199   }
200 
201   /**
202    * Shuts down instance created by call to {@link #startMiniDFSCluster(int, File)}
203    * or does nothing.
204    * @throws Exception
205    */
206   public void shutdownMiniDFSCluster() throws Exception {
207     if (this.dfsCluster != null) {
208       // The below throws an exception per dn, AsynchronousCloseException.
209       this.dfsCluster.shutdown();
210     }
211   }
212 
213   /**
214    * Call this if you only want a zk cluster.
215    * @see #startMiniZKCluster() if you want zk + dfs + hbase mini cluster.
216    * @throws Exception
217    * @see #shutdownMiniZKCluster()
218    * @return zk cluster started.
219    */
220   public MiniZooKeeperCluster startMiniZKCluster() throws Exception {
221     return startMiniZKCluster(setupClusterTestBuildDir());
222 
223   }
224 
225   private MiniZooKeeperCluster startMiniZKCluster(final File dir)
226   throws Exception {
227     if (this.zkCluster != null) {
228       throw new IOException("Cluster already running at " + dir);
229     }
230     this.zkCluster = new MiniZooKeeperCluster();
231     int clientPort = this.zkCluster.startup(dir);
232     this.conf.set("hbase.zookeeper.property.clientPort",
233       Integer.toString(clientPort));
234     return this.zkCluster;
235   }
236 
237   /**
238    * Shuts down zk cluster created by call to {@link #startMiniZKCluster(File)}
239    * or does nothing.
240    * @throws IOException
241    * @see #startMiniZKCluster()
242    */
243   public void shutdownMiniZKCluster() throws IOException {
244     if (this.zkCluster != null) this.zkCluster.shutdown();
245   }
246 
247   /**
248    * Start up a minicluster of hbase, dfs, and zookeeper.
249    * @throws Exception
250    * @return Mini hbase cluster instance created.
251    * @see {@link #shutdownMiniDFSCluster()}
252    */
253   public MiniHBaseCluster startMiniCluster() throws Exception {
254     return startMiniCluster(1);
255   }
256 
257   /**
258    * Start up a minicluster of hbase, optionally dfs, and zookeeper.
259    * Modifies Configuration.  Homes the cluster data directory under a random
260    * subdirectory in a directory under System property test.build.data.
261    * Directory is cleaned up on exit.
262    * @param servers Number of servers to start up.  We'll start this many
263    * datanodes and regionservers.  If servers is > 1, then make sure
264    * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise
265    * bind errors.
266    * @throws Exception
267    * @see {@link #shutdownMiniCluster()}
268    * @return Mini hbase cluster instance created.
269    */
270   public MiniHBaseCluster startMiniCluster(final int servers)
271   throws Exception {
272     LOG.info("Starting up minicluster");
273     // If we already put up a cluster, fail.
274     isRunningCluster();
275     // Make a new random dir to home everything in.  Set it as system property.
276     // minidfs reads home from system property.
277     this.clusterTestBuildDir = setupClusterTestBuildDir();
278     System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestBuildDir.getPath());
279     // Bring up mini dfs cluster. This spews a bunch of warnings about missing
280     // scheme. Complaints are 'Scheme is undefined for build/test/data/dfs/name1'.
281     startMiniDFSCluster(servers, this.clusterTestBuildDir);
282 
283     // Mangle conf so fs parameter points to minidfs we just started up
284     FileSystem fs = this.dfsCluster.getFileSystem();
285     this.conf.set("fs.defaultFS", fs.getUri().toString());
286     // Do old style too just to be safe.
287     this.conf.set("fs.default.name", fs.getUri().toString());
288     this.dfsCluster.waitClusterUp();
289 
290     // Start up a zk cluster.
291     if (this.zkCluster == null) {
292       startMiniZKCluster(this.clusterTestBuildDir);
293     }
294 
295     // Now do the mini hbase cluster.  Set the hbase.rootdir in config.
296     Path hbaseRootdir = fs.makeQualified(fs.getHomeDirectory());
297     this.conf.set(HConstants.HBASE_DIR, hbaseRootdir.toString());
298     fs.mkdirs(hbaseRootdir);
299     FSUtils.setVersion(fs, hbaseRootdir);
300     this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
301     // Don't leave here till we've done a successful scan of the .META.
302     HTable t = new HTable(this.conf, HConstants.META_TABLE_NAME);
303     ResultScanner s = t.getScanner(new Scan());
304     while (s.next() != null) continue;
305     LOG.info("Minicluster is up");
306     return this.hbaseCluster;
307   }
308 
309   /**
310    * @return Current mini hbase cluster. Only has something in it after a call
311    * to {@link #startMiniCluster()}.
312    * @see #startMiniCluster()
313    */
314   public MiniHBaseCluster getMiniHBaseCluster() {
315     return this.hbaseCluster;
316   }
317 
318   /**
319    * @throws IOException
320    * @see {@link #startMiniCluster(int)}
321    */
322   public void shutdownMiniCluster() throws IOException {
323     LOG.info("Shutting down minicluster");
324     if (this.hbaseCluster != null) {
325       this.hbaseCluster.shutdown();
326       // Wait till hbase is down before going on to shutdown zk.
327       this.hbaseCluster.join();
328     }
329     shutdownMiniZKCluster();
330     if (this.dfsCluster != null) {
331       // The below throws an exception per dn, AsynchronousCloseException.
332       this.dfsCluster.shutdown();
333     }
334     // Clean up our directory.
335     if (this.clusterTestBuildDir != null && this.clusterTestBuildDir.exists()) {
336       // Need to use deleteDirectory because File.delete required dir is empty.
337       if (!FSUtils.deleteDirectory(FileSystem.getLocal(this.conf),
338           new Path(this.clusterTestBuildDir.toString()))) {
339         LOG.warn("Failed delete of " + this.clusterTestBuildDir.toString());
340       }
341     }
342     LOG.info("Minicluster is down");
343   }
344 
345   /**
346    * Flushes all caches in the mini hbase cluster
347    * @throws IOException
348    */
349   public void flush() throws IOException {
350     this.hbaseCluster.flushcache();
351   }
352 
353   /**
354    * Flushes all caches in the mini hbase cluster
355    * @throws IOException
356    */
357   public void flush(byte [] tableName) throws IOException {
358     this.hbaseCluster.flushcache(tableName);
359   }
360 
361 
362   /**
363    * Create a table.
364    * @param tableName
365    * @param family
366    * @return An HTable instance for the created table.
367    * @throws IOException
368    */
369   public HTable createTable(byte[] tableName, byte[] family)
370   throws IOException{
371     return createTable(tableName, new byte[][]{family});
372   }
373 
374   /**
375    * Create a table.
376    * @param tableName
377    * @param families
378    * @return An HTable instance for the created table.
379    * @throws IOException
380    */
381   public HTable createTable(byte[] tableName, byte[][] families)
382   throws IOException {
383     HTableDescriptor desc = new HTableDescriptor(tableName);
384     for(byte[] family : families) {
385       desc.addFamily(new HColumnDescriptor(family));
386     }
387     (new HBaseAdmin(getConfiguration())).createTable(desc);
388     return new HTable(getConfiguration(), tableName);
389   }
390 
391   /**
392    * Create a table.
393    * @param tableName
394    * @param family
395    * @param numVersions
396    * @return An HTable instance for the created table.
397    * @throws IOException
398    */
399   public HTable createTable(byte[] tableName, byte[] family, int numVersions)
400   throws IOException {
401     return createTable(tableName, new byte[][]{family}, numVersions);
402   }
403 
404   /**
405    * Create a table.
406    * @param tableName
407    * @param families
408    * @param numVersions
409    * @return An HTable instance for the created table.
410    * @throws IOException
411    */
412   public HTable createTable(byte[] tableName, byte[][] families,
413       int numVersions)
414   throws IOException {
415     HTableDescriptor desc = new HTableDescriptor(tableName);
416     for (byte[] family : families) {
417       HColumnDescriptor hcd = new HColumnDescriptor(family, numVersions,
418           HColumnDescriptor.DEFAULT_COMPRESSION,
419           HColumnDescriptor.DEFAULT_IN_MEMORY,
420           HColumnDescriptor.DEFAULT_BLOCKCACHE,
421           Integer.MAX_VALUE, HColumnDescriptor.DEFAULT_TTL,
422           HColumnDescriptor.DEFAULT_BLOOMFILTER,
423           HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
424       desc.addFamily(hcd);
425     }
426     (new HBaseAdmin(getConfiguration())).createTable(desc);
427     return new HTable(getConfiguration(), tableName);
428   }
429 
430   /**
431    * Create a table.
432    * @param tableName
433    * @param families
434    * @param numVersions
435    * @return An HTable instance for the created table.
436    * @throws IOException
437    */
438   public HTable createTable(byte[] tableName, byte[][] families,
439       int[] numVersions)
440   throws IOException {
441     HTableDescriptor desc = new HTableDescriptor(tableName);
442     int i = 0;
443     for (byte[] family : families) {
444       HColumnDescriptor hcd = new HColumnDescriptor(family, numVersions[i],
445           HColumnDescriptor.DEFAULT_COMPRESSION,
446           HColumnDescriptor.DEFAULT_IN_MEMORY,
447           HColumnDescriptor.DEFAULT_BLOCKCACHE,
448           Integer.MAX_VALUE, HColumnDescriptor.DEFAULT_TTL,
449           HColumnDescriptor.DEFAULT_BLOOMFILTER,
450           HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
451       desc.addFamily(hcd);
452       i++;
453     }
454     (new HBaseAdmin(getConfiguration())).createTable(desc);
455     return new HTable(getConfiguration(), tableName);
456   }
457 
458   /**
459    * Provide an existing table name to truncate
460    * @param tableName existing table
461    * @return HTable to that new table
462    * @throws IOException
463    */
464   public HTable truncateTable(byte [] tableName) throws IOException {
465     HTable table = new HTable(getConfiguration(), tableName);
466     Scan scan = new Scan();
467     ResultScanner resScan = table.getScanner(scan);
468     for(Result res : resScan) {
469       Delete del = new Delete(res.getRow());
470       table.delete(del);
471     }
472     return table;
473   }
474 
475   /**
476    * Load table with rows from 'aaa' to 'zzz'.
477    * @param t Table
478    * @param f Family
479    * @return Count of rows loaded.
480    * @throws IOException
481    */
482   public int loadTable(final HTable t, final byte[] f) throws IOException {
483     t.setAutoFlush(false);
484     byte[] k = new byte[3];
485     int rowCount = 0;
486     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
487       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
488         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
489           k[0] = b1;
490           k[1] = b2;
491           k[2] = b3;
492           Put put = new Put(k);
493           put.add(f, null, k);
494           t.put(put);
495           rowCount++;
496         }
497       }
498     }
499     t.flushCommits();
500     return rowCount;
501   }
502   /**
503    * Load region with rows from 'aaa' to 'zzz'.
504    * @param r Region
505    * @param f Family
506    * @return Count of rows loaded.
507    * @throws IOException
508    */
509   public int loadRegion(final HRegion r, final byte[] f)
510   throws IOException {
511     byte[] k = new byte[3];
512     int rowCount = 0;
513     for (byte b1 = 'a'; b1 <= 'z'; b1++) {
514       for (byte b2 = 'a'; b2 <= 'z'; b2++) {
515         for (byte b3 = 'a'; b3 <= 'z'; b3++) {
516           k[0] = b1;
517           k[1] = b2;
518           k[2] = b3;
519           Put put = new Put(k);
520           put.add(f, null, k);
521           if (r.getLog() == null) put.setWriteToWAL(false);
522           r.put(put);
523           rowCount++;
524         }
525       }
526     }
527     return rowCount;
528   }
529 
530   /**
531    * Return the number of rows in the given table.
532    */
533   public int countRows(final HTable table) throws IOException {
534     Scan scan = new Scan();
535     ResultScanner results = table.getScanner(scan);
536     int count = 0;
537     for (@SuppressWarnings("unused") Result res : results) {
538       count++;
539     }
540     results.close();
541     return count;
542   }
543 
544   /**
545    * Return an md5 digest of the entire contents of a table.
546    */
547   public String checksumRows(final HTable table) throws Exception {
548     Scan scan = new Scan();
549     ResultScanner results = table.getScanner(scan);
550     MessageDigest digest = MessageDigest.getInstance("MD5");
551     for (Result res : results) {
552       digest.update(res.getRow());
553     }
554     results.close();
555     return digest.toString();
556   }
557 
558   /**
559    * Creates many regions names "aaa" to "zzz".
560    *
561    * @param table  The table to use for the data.
562    * @param columnFamily  The family to insert the data into.
563    * @return count of regions created.
564    * @throws IOException When creating the regions fails.
565    */
566   public int createMultiRegions(HTable table, byte[] columnFamily)
567   throws IOException {
568     return createMultiRegions(getConfiguration(), table, columnFamily);
569   }
570 
571   /**
572    * Creates many regions names "aaa" to "zzz".
573    * @param c Configuration to use.
574    * @param table  The table to use for the data.
575    * @param columnFamily  The family to insert the data into.
576    * @return count of regions created.
577    * @throws IOException When creating the regions fails.
578    */
579   public int createMultiRegions(final Configuration c, final HTable table,
580       final byte[] columnFamily)
581   throws IOException {
582     byte[][] KEYS = {
583       HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"),
584       Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
585       Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
586       Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
587       Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
588       Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
589       Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
590       Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
591       Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
592     };
593     return createMultiRegions(c, table, columnFamily, KEYS);
594   }
595 
596   public int createMultiRegions(final Configuration c, final HTable table,
597       final byte[] columnFamily, byte [][] startKeys)
598   throws IOException {
599     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
600     HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
601     HTableDescriptor htd = table.getTableDescriptor();
602     if(!htd.hasFamily(columnFamily)) {
603       HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
604       htd.addFamily(hcd);
605     }
606     // remove empty region - this is tricky as the mini cluster during the test
607     // setup already has the "<tablename>,,123456789" row with an empty start
608     // and end key. Adding the custom regions below adds those blindly,
609     // including the new start region from empty to "bbb". lg
610     List<byte[]> rows = getMetaTableRows(htd.getName());
611     // add custom ones
612     int count = 0;
613     for (int i = 0; i < startKeys.length; i++) {
614       int j = (i + 1) % startKeys.length;
615       HRegionInfo hri = new HRegionInfo(table.getTableDescriptor(),
616         startKeys[i], startKeys[j]);
617       Put put = new Put(hri.getRegionName());
618       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
619         Writables.getBytes(hri));
620       meta.put(put);
621       LOG.info("createMultiRegions: inserted " + hri.toString());
622       count++;
623     }
624     // see comment above, remove "old" (or previous) single region
625     for (byte[] row : rows) {
626       LOG.info("createMultiRegions: deleting meta row -> " +
627         Bytes.toStringBinary(row));
628       meta.delete(new Delete(row));
629     }
630     // flush cache of regions
631     HConnection conn = table.getConnection();
632     conn.clearRegionCache();
633     return count;
634   }
635 
636   /**
637    * Returns all rows from the .META. table.
638    *
639    * @throws IOException When reading the rows fails.
640    */
641   public List<byte[]> getMetaTableRows() throws IOException {
642     HTable t = new HTable(this.conf, HConstants.META_TABLE_NAME);
643     List<byte[]> rows = new ArrayList<byte[]>();
644     ResultScanner s = t.getScanner(new Scan());
645     for (Result result : s) {
646       LOG.info("getMetaTableRows: row -> " +
647         Bytes.toStringBinary(result.getRow()));
648       rows.add(result.getRow());
649     }
650     s.close();
651     return rows;
652   }
653 
654   /**
655    * Returns all rows from the .META. table for a given user table
656    *
657    * @throws IOException When reading the rows fails.
658    */
659   public List<byte[]> getMetaTableRows(byte[] tableName) throws IOException {
660     HTable t = new HTable(this.conf, HConstants.META_TABLE_NAME);
661     List<byte[]> rows = new ArrayList<byte[]>();
662     ResultScanner s = t.getScanner(new Scan());
663     for (Result result : s) {
664       byte[] value = result.getValue(
665           HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
666       if (value == null) continue;
667       HRegionInfo info = Writables.getHRegionInfo(value);
668       HTableDescriptor desc = info.getTableDesc();
669       if (Bytes.compareTo(desc.getName(), tableName) == 0) {
670         LOG.info("getMetaTableRows: row -> " +
671             Bytes.toStringBinary(result.getRow()));
672         rows.add(result.getRow());
673       }
674     }
675     s.close();
676     return rows;
677   }
678 
679   /**
680    * Tool to get the reference to the region server object that holds the
681    * region of the specified user table.
682    * It first searches for the meta rows that contain the region of the
683    * specified table, then gets the index of that RS, and finally retrieves
684    * the RS's reference.
685    * @param tableName user table to lookup in .META.
686    * @return region server that holds it, null if the row doesn't exist
687    * @throws IOException
688    */
689   public HRegionServer getRSForFirstRegionInTable(byte[] tableName)
690       throws IOException {
691     List<byte[]> metaRows = getMetaTableRows(tableName);
692     if (metaRows == null || metaRows.size() == 0) {
693       return null;
694     }
695     int index = hbaseCluster.getServerWith(metaRows.get(0));
696     return hbaseCluster.getRegionServerThreads().get(index).getRegionServer();
697   }
698 
699   /**
700    * Starts a <code>MiniMRCluster</code> with a default number of
701    * <code>TaskTracker</code>'s.
702    *
703    * @throws IOException When starting the cluster fails.
704    */
705   public void startMiniMapReduceCluster() throws IOException {
706     startMiniMapReduceCluster(2);
707   }
708 
709   /**
710    * Starts a <code>MiniMRCluster</code>.
711    *
712    * @param servers  The number of <code>TaskTracker</code>'s to start.
713    * @throws IOException When starting the cluster fails.
714    */
715   public void startMiniMapReduceCluster(final int servers) throws IOException {
716     LOG.info("Starting mini mapreduce cluster...");
717     // These are needed for the new and improved Map/Reduce framework
718     Configuration c = getConfiguration();
719     System.setProperty("hadoop.log.dir", c.get("hadoop.log.dir"));
720     c.set("mapred.output.dir", c.get("hadoop.tmp.dir"));
721     mrCluster = new MiniMRCluster(servers,
722       FileSystem.get(c).getUri().toString(), 1);
723     LOG.info("Mini mapreduce cluster started");
724     c.set("mapred.job.tracker",
725         mrCluster.createJobConf().get("mapred.job.tracker"));
726   }
727 
728   /**
729    * Stops the previously started <code>MiniMRCluster</code>.
730    */
731   public void shutdownMiniMapReduceCluster() {
732     LOG.info("Stopping mini mapreduce cluster...");
733     if (mrCluster != null) {
734       mrCluster.shutdown();
735     }
736     // Restore configuration to point to local jobtracker
737     conf.set("mapred.job.tracker", "local");
738     LOG.info("Mini mapreduce cluster stopped");
739   }
740 
741   /**
742    * Switches the logger for the given class to DEBUG level.
743    *
744    * @param clazz  The class for which to switch to debug logging.
745    */
746   public void enableDebug(Class<?> clazz) {
747     Log l = LogFactory.getLog(clazz);
748     if (l instanceof Log4JLogger) {
749       ((Log4JLogger) l).getLogger().setLevel(org.apache.log4j.Level.DEBUG);
750     } else if (l instanceof Jdk14Logger) {
751       ((Jdk14Logger) l).getLogger().setLevel(java.util.logging.Level.ALL);
752     }
753   }
754 
755   /**
756    * Expire the Master's session
757    * @throws Exception
758    */
759   public void expireMasterSession() throws Exception {
760     HMaster master = hbaseCluster.getMaster();
761     expireSession(master.getZooKeeperWrapper());
762   }
763 
764   /**
765    * Expire a region server's session
766    * @param index which RS
767    * @throws Exception
768    */
769   public void expireRegionServerSession(int index) throws Exception {
770     HRegionServer rs = hbaseCluster.getRegionServer(index);
771     expireSession(rs.getZooKeeperWrapper());
772   }
773 
774   public void expireSession(ZooKeeperWrapper nodeZK) throws Exception{
775     ZooKeeperWrapper zkw =
776         ZooKeeperWrapper.createInstance(conf,
777             ZooKeeperWrapper.class.getName());
778     zkw.registerListener(EmptyWatcher.instance);
779     String quorumServers = zkw.getQuorumServers();
780     int sessionTimeout = 5 * 1000; // 5 seconds
781 
782     byte[] password = nodeZK.getSessionPassword();
783     long sessionID = nodeZK.getSessionID();
784 
785     ZooKeeper zk = new ZooKeeper(quorumServers,
786         sessionTimeout, EmptyWatcher.instance, sessionID, password);
787     zk.close();
788     final long sleep = sessionTimeout * 5L;
789     LOG.info("ZK Closed; sleeping=" + sleep);
790 
791     Thread.sleep(sleep);
792 
793     new HTable(conf, HConstants.META_TABLE_NAME);
794   }
795 
796   /**
797    * Get the HBase cluster.
798    *
799    * @return hbase cluster
800    */
801   public MiniHBaseCluster getHBaseCluster() {
802     return hbaseCluster;
803   }
804 
805   /**
806    * Returns a HBaseAdmin instance.
807    *
808    * @return The HBaseAdmin instance.
809    * @throws MasterNotRunningException
810    */
811   public HBaseAdmin getHBaseAdmin() throws MasterNotRunningException {
812     if (hbaseAdmin == null) {
813       hbaseAdmin = new HBaseAdmin(getConfiguration());
814     }
815     return hbaseAdmin;
816   }
817 
818   /**
819    * Closes the named region.
820    *
821    * @param regionName  The region to close.
822    * @throws IOException
823    */
824   public void closeRegion(String regionName) throws IOException {
825     closeRegion(Bytes.toBytes(regionName));
826   }
827 
828   /**
829    * Closes the named region.
830    *
831    * @param regionName  The region to close.
832    * @throws IOException
833    */
834   public void closeRegion(byte[] regionName) throws IOException {
835     HBaseAdmin admin = getHBaseAdmin();
836     admin.closeRegion(regionName, (Object[]) null);
837   }
838 
839   /**
840    * Closes the region containing the given row.
841    *
842    * @param row  The row to find the containing region.
843    * @param table  The table to find the region.
844    * @throws IOException
845    */
846   public void closeRegionByRow(String row, HTable table) throws IOException {
847     closeRegionByRow(Bytes.toBytes(row), table);
848   }
849 
850   /**
851    * Closes the region containing the given row.
852    *
853    * @param row  The row to find the containing region.
854    * @param table  The table to find the region.
855    * @throws IOException
856    */
857   public void closeRegionByRow(byte[] row, HTable table) throws IOException {
858     HRegionLocation hrl = table.getRegionLocation(row);
859     closeRegion(hrl.getRegionInfo().getRegionName());
860   }
861 
862   public MiniZooKeeperCluster getZkCluster() {
863     return zkCluster;
864   }
865 
866   public void setZkCluster(MiniZooKeeperCluster zkCluster) {
867     this.zkCluster = zkCluster;
868   }
869 
870   public MiniDFSCluster getDFSCluster() {
871     return dfsCluster;
872   }
873 
874   public FileSystem getTestFileSystem() throws IOException {
875     return FileSystem.get(conf);
876   }
877 
878   public void cleanupTestDir() throws IOException {
879     getTestDir().getFileSystem(conf).delete(getTestDir(), true);
880   }
881 
882   public void waitTableAvailable(byte[] table, long timeoutMillis)
883   throws InterruptedException, IOException {
884     HBaseAdmin admin = new HBaseAdmin(conf);
885     long startWait = System.currentTimeMillis();
886     while (!admin.isTableAvailable(table)) {
887       assertTrue("Timed out waiting for table " + Bytes.toStringBinary(table),
888           System.currentTimeMillis() - startWait < timeoutMillis);
889       Thread.sleep(500);
890     }
891   }
892 
893   /**
894    * Make sure that at least the specified number of region servers
895    * are running
896    * @param num minimum number of region servers that should be running
897    * @throws IOException
898    */
899   public void ensureSomeRegionServersAvailable(final int num)
900       throws IOException {
901     if (this.getHBaseCluster().getLiveRegionServerThreads().size() < num) {
902       // Need at least "num" servers.
903       LOG.info("Started new server=" +
904         this.getHBaseCluster().startRegionServer());
905 
906     }
907   }
908 
909   /**
910    * This method clones the passed <code>c</code> configuration setting a new
911    * user into the clone.  Use it getting new instances of FileSystem.  Only
912    * works for DistributedFileSystem.
913    * @param c Initial configuration
914    * @param differentiatingSuffix Suffix to differentiate this user from others.
915    * @return A new configuration instance with a different user set into it.
916    * @throws IOException
917    */
918   public static UserGroupInformation getDifferentUser(final Configuration c,
919     final String differentiatingSuffix)
920   throws IOException {
921     FileSystem currentfs = FileSystem.get(c);
922     Preconditions.checkArgument(currentfs instanceof DistributedFileSystem);
923     // Else distributed filesystem.  Make a new instance per daemon.  Below
924     // code is taken from the AppendTestUtil over in hdfs.
925     String username = UserGroupInformation.getCurrentUser().getUserName() +
926       differentiatingSuffix;
927     return UserGroupInformation.createUserForTesting(username,
928         new String[]{"supergroup"});
929   }
930 
931   /**
932    * Set soft and hard limits in namenode.
933    * You'll get a NPE if you call before you've started a minidfscluster.
934    * @param soft Soft limit
935    * @param hard Hard limit
936    * @throws NoSuchFieldException
937    * @throws SecurityException
938    * @throws IllegalAccessException
939    * @throws IllegalArgumentException
940    */
941   public void setNameNodeNameSystemLeasePeriod(final int soft, final int hard)
942   throws SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException {
943     // TODO: If 0.20 hadoop do one thing, if 0.21 hadoop do another.
944     // Not available in 0.20 hdfs.  Use reflection to make it happen.
945 
946     // private NameNode nameNode;
947     Field field = this.dfsCluster.getClass().getDeclaredField("nameNode");
948     field.setAccessible(true);
949     NameNode nn = (NameNode)field.get(this.dfsCluster);
950     nn.namesystem.leaseManager.setLeasePeriod(100, 50000);
951   }
952 
953   /**
954    * Set maxRecoveryErrorCount in DFSClient.  In 0.20 pre-append its hard-coded to 5 and
955    * makes tests linger.  Here is the exception you'll see:
956    * <pre>
957    * 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...
958    * </pre>
959    * @param stream A DFSClient.DFSOutputStream.
960    * @param max
961    * @throws NoSuchFieldException
962    * @throws SecurityException
963    * @throws IllegalAccessException
964    * @throws IllegalArgumentException
965    */
966   public static void setMaxRecoveryErrorCount(final OutputStream stream,
967       final int max) {
968     try {
969       Class<?> [] clazzes = DFSClient.class.getDeclaredClasses();
970       for (Class<?> clazz: clazzes) {
971         String className = clazz.getSimpleName();
972         if (className.equals("DFSOutputStream")) {
973           if (clazz.isInstance(stream)) {
974             Field maxRecoveryErrorCountField =
975               stream.getClass().getDeclaredField("maxRecoveryErrorCount");
976             maxRecoveryErrorCountField.setAccessible(true);
977             maxRecoveryErrorCountField.setInt(stream, max);
978             break;
979           }
980         }
981       }
982     } catch (Exception e) {
983       LOG.info("Could not set max recovery field", e);
984     }
985   }
986 
987 
988   /**
989    * Wait until <code>countOfRegion</code> in .META. have a non-empty
990    * info:server.  This means all regions have been deployed, master has been
991    * informed and updated .META. with the regions deployed server.
992    * @param conf Configuration
993    * @param countOfRegions How many regions in .META.
994    * @throws IOException
995    */
996   public void waitUntilAllRegionsAssigned(final int countOfRegions)
997   throws IOException {
998     HTable meta = new HTable(getConfiguration(), HConstants.META_TABLE_NAME);
999     while (true) {
1000       int rows = 0;
1001       Scan scan = new Scan();
1002       scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
1003       ResultScanner s = meta.getScanner(scan);
1004       for (Result r = null; (r = s.next()) != null;) {
1005         byte [] b =
1006           r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
1007         if (b == null || b.length <= 0) break;
1008         rows++;
1009       }
1010       s.close();
1011       // If I get to here and all rows have a Server, then all have been assigned.
1012       if (rows == countOfRegions) break;
1013       LOG.info("Found=" + rows);
1014       Threads.sleep(1000);
1015     }
1016   }
1017 
1018   /**
1019    * Do a small get/scan against one store. This is required because store
1020    * has no actual methods of querying itself, and relies on StoreScanner.
1021    */
1022   public static List<KeyValue> getFromStoreFile(Store store,
1023                                                 Get get) throws IOException {
1024     ReadWriteConsistencyControl.resetThreadReadPoint();
1025     Scan scan = new Scan(get);
1026     InternalScanner scanner = (InternalScanner) store.getScanner(scan,
1027         scan.getFamilyMap().get(store.getFamily().getName()));
1028 
1029     List<KeyValue> result = new ArrayList<KeyValue>();
1030     scanner.next(result);
1031     if (!result.isEmpty()) {
1032       // verify that we are on the row we want:
1033       KeyValue kv = result.get(0);
1034       if (!Bytes.equals(kv.getRow(), get.getRow())) {
1035         result.clear();
1036       }
1037     }
1038     return result;
1039   }
1040 
1041   /**
1042    * Do a small get/scan against one store. This is required because store
1043    * has no actual methods of querying itself, and relies on StoreScanner.
1044    */
1045   public static List<KeyValue> getFromStoreFile(Store store,
1046                                                 byte [] row,
1047                                                 NavigableSet<byte[]> columns
1048                                                 ) throws IOException {
1049     Get get = new Get(row);
1050     Map<byte[], NavigableSet<byte[]>> s = get.getFamilyMap();
1051     s.put(store.getFamily().getName(), columns);
1052 
1053     return getFromStoreFile(store,get);
1054   }
1055 }