1   /**
2    * Copyright 2011 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.client;
21  
22  
23  import static org.junit.Assert.assertEquals;
24  import static org.junit.Assert.assertFalse;
25  import static org.junit.Assert.assertTrue;
26  import static org.junit.Assert.fail;
27  
28  import java.io.IOException;
29  import java.util.ArrayList;
30  import java.util.Collections;
31  import java.util.Comparator;
32  import java.util.HashMap;
33  import java.util.Iterator;
34  import java.util.List;
35  import java.util.Map;
36  import java.util.concurrent.atomic.AtomicBoolean;
37  import java.util.concurrent.atomic.AtomicInteger;
38  
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.hbase.DoNotRetryIOException;
43  import org.apache.hadoop.hbase.HBaseTestingUtility;
44  import org.apache.hadoop.hbase.HColumnDescriptor;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.HRegionInfo;
47  import org.apache.hadoop.hbase.HRegionLocation;
48  import org.apache.hadoop.hbase.HServerAddress;
49  import org.apache.hadoop.hbase.HTableDescriptor;
50  import org.apache.hadoop.hbase.InvalidFamilyOperationException;
51  import org.apache.hadoop.hbase.LargeTests;
52  import org.apache.hadoop.hbase.ServerName;
53  import org.apache.hadoop.hbase.TableExistsException;
54  import org.apache.hadoop.hbase.TableNotDisabledException;
55  import org.apache.hadoop.hbase.TableNotEnabledException;
56  import org.apache.hadoop.hbase.TableNotFoundException;
57  import org.apache.hadoop.hbase.catalog.CatalogTracker;
58  import org.apache.hadoop.hbase.executor.EventHandler;
59  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
60  import org.apache.hadoop.hbase.executor.ExecutorService;
61  import org.apache.hadoop.hbase.master.MasterServices;
62  import org.apache.hadoop.hbase.regionserver.HRegion;
63  import org.apache.hadoop.hbase.regionserver.HRegionServer;
64  import org.apache.hadoop.hbase.regionserver.wal.HLogUtilsForTests;
65  import org.apache.hadoop.hbase.util.Bytes;
66  import org.apache.hadoop.hbase.util.Pair;
67  import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
68  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
69  import org.junit.After;
70  import org.junit.AfterClass;
71  import org.junit.Assert;
72  import org.junit.Before;
73  import org.junit.BeforeClass;
74  import org.junit.Test;
75  import org.junit.experimental.categories.Category;
76  
77  
78  /**
79   * Class to test HBaseAdmin.
80   * Spins up the minicluster once at test start and then takes it down afterward.
81   * Add any testing of HBaseAdmin functionality here.
82   */
83  @Category(LargeTests.class)
84  public class TestAdmin {
85    final Log LOG = LogFactory.getLog(getClass());
86    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
87    private HBaseAdmin admin;
88  
89    @BeforeClass
90    public static void setUpBeforeClass() throws Exception {
91      TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
92      TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
93      TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
94      TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
95      TEST_UTIL.getConfiguration().setBoolean(
96          "hbase.master.enabletable.roundrobin", true);
97      TEST_UTIL.startMiniCluster(3);
98    }
99  
100   @AfterClass
101   public static void tearDownAfterClass() throws Exception {
102     TEST_UTIL.shutdownMiniCluster();
103   }
104 
105   @Before
106   public void setUp() throws Exception {
107     this.admin = TEST_UTIL.getHBaseAdmin();
108   }
109 
110   @After
111   public void tearDown() throws Exception {
112   }
113 
114   @Test
115   public void testSplitFlushCompactUnknownTable() throws InterruptedException {
116     final String unknowntable = "fubar";
117     Exception exception = null;
118     try {
119       this.admin.compact(unknowntable);
120     } catch (IOException e) {
121       exception = e;
122     }
123     assertTrue(exception instanceof TableNotFoundException);
124 
125     exception = null;
126     try {
127       this.admin.flush(unknowntable);
128     } catch (IOException e) {
129       exception = e;
130     }
131     assertTrue(exception instanceof TableNotFoundException);
132 
133     exception = null;
134     try {
135       this.admin.split(unknowntable);
136     } catch (IOException e) {
137       exception = e;
138     }
139     assertTrue(exception instanceof TableNotFoundException);
140   }
141 
142   @Test
143   public void testDeleteEditUnknownColumnFamilyAndOrTable() throws IOException {
144     // Test we get exception if we try to
145     final String nonexistent = "nonexistent";
146     HColumnDescriptor nonexistentHcd = new HColumnDescriptor(nonexistent);
147     Exception exception = null;
148     try {
149       this.admin.addColumn(nonexistent, nonexistentHcd);
150     } catch (IOException e) {
151       exception = e;
152     }
153     assertTrue(exception instanceof TableNotFoundException);
154 
155     exception = null;
156     try {
157       this.admin.deleteTable(nonexistent);
158     } catch (IOException e) {
159       exception = e;
160     }
161     assertTrue(exception instanceof TableNotFoundException);
162 
163     exception = null;
164     try {
165       this.admin.deleteColumn(nonexistent, nonexistent);
166     } catch (IOException e) {
167       exception = e;
168     }
169     assertTrue(exception instanceof TableNotFoundException);
170 
171     exception = null;
172     try {
173       this.admin.disableTable(nonexistent);
174     } catch (IOException e) {
175       exception = e;
176     }
177     assertTrue(exception instanceof TableNotFoundException);
178 
179     exception = null;
180     try {
181       this.admin.enableTable(nonexistent);
182     } catch (IOException e) {
183       exception = e;
184     }
185     assertTrue(exception instanceof TableNotFoundException);
186 
187     exception = null;
188     try {
189       this.admin.modifyColumn(nonexistent, nonexistentHcd);
190     } catch (IOException e) {
191       exception = e;
192     }
193     assertTrue(exception instanceof TableNotFoundException);
194 
195     exception = null;
196     try {
197       HTableDescriptor htd = new HTableDescriptor(nonexistent);
198       this.admin.modifyTable(htd.getName(), htd);
199     } catch (IOException e) {
200       exception = e;
201     }
202     assertTrue(exception instanceof TableNotFoundException);
203 
204     // Now make it so at least the table exists and then do tests against a
205     // nonexistent column family -- see if we get right exceptions.
206     final String tableName = "t";
207     HTableDescriptor htd = new HTableDescriptor(tableName);
208     htd.addFamily(new HColumnDescriptor("cf"));
209     this.admin.createTable(htd);
210     try {
211       exception = null;
212       try {
213         this.admin.deleteColumn(htd.getName(), nonexistentHcd.getName());
214       } catch (IOException e) {
215         exception = e;
216       }
217       assertTrue(exception instanceof InvalidFamilyOperationException);
218 
219       exception = null;
220       try {
221         this.admin.modifyColumn(htd.getName(), nonexistentHcd);
222       } catch (IOException e) {
223         exception = e;
224       }
225       assertTrue(exception instanceof InvalidFamilyOperationException);
226     } finally {
227       this.admin.disableTable(tableName);
228       this.admin.deleteTable(tableName);
229     }
230   }
231 
232   @Test
233   public void testDisableAndEnableTable() throws IOException {
234     final byte [] row = Bytes.toBytes("row");
235     final byte [] qualifier = Bytes.toBytes("qualifier");
236     final byte [] value = Bytes.toBytes("value");
237     final byte [] table = Bytes.toBytes("testDisableAndEnableTable");
238     HTable ht = TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
239     Put put = new Put(row);
240     put.add(HConstants.CATALOG_FAMILY, qualifier, value);
241     ht.put(put);
242     Get get = new Get(row);
243     get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
244     ht.get(get);
245 
246     this.admin.disableTable(table);
247     assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
248         .getMaster().getAssignmentManager().getZKTable().isDisabledTable(
249             Bytes.toString(table)));
250 
251     // Test that table is disabled
252     get = new Get(row);
253     get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
254     boolean ok = false;
255     try {
256       ht.get(get);
257     } catch (DoNotRetryIOException e) {
258       ok = true;
259     }
260     assertTrue(ok);
261     this.admin.enableTable(table);
262     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
263         .getMaster().getAssignmentManager().getZKTable().isEnabledTable(
264             Bytes.toString(table)));
265 
266     // Test that table is enabled
267     try {
268       ht.get(get);
269     } catch (RetriesExhaustedException e) {
270       ok = false;
271     }
272     assertTrue(ok);
273     ht.close();
274   }
275 
276   @Test
277   public void testIsEnabledOnUnknownTable() throws Exception {
278     try {
279       admin.isTableEnabled(Bytes.toBytes("unkownTable"));
280       fail("Test should fail if isTableEnabled called on unknown table.");
281     } catch (IOException e) {
282     }
283   }
284 
285   @Test
286   public void testDisableAndEnableTables() throws IOException {
287     final byte [] row = Bytes.toBytes("row");
288     final byte [] qualifier = Bytes.toBytes("qualifier");
289     final byte [] value = Bytes.toBytes("value");
290     final byte [] table1 = Bytes.toBytes("testDisableAndEnableTable1");
291     final byte [] table2 = Bytes.toBytes("testDisableAndEnableTable2");
292     HTable ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY);
293     HTable ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY);
294     Put put = new Put(row);
295     put.add(HConstants.CATALOG_FAMILY, qualifier, value);
296     ht1.put(put);
297     ht2.put(put);
298     Get get = new Get(row);
299     get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
300     ht1.get(get);
301     ht2.get(get);
302 
303     this.admin.disableTables("testDisableAndEnableTable.*");
304 
305     // Test that tables are disabled
306     get = new Get(row);
307     get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
308     boolean ok = false;
309     try {
310       ht1.get(get);
311       ht2.get(get);
312     } catch (DoNotRetryIOException e) {
313       ok = true;
314     }
315 
316     assertTrue(ok);
317     this.admin.enableTables("testDisableAndEnableTable.*");
318 
319     // Test that tables are enabled
320     try {
321       ht1.get(get);
322     } catch (IOException e) {
323       ok = false;
324     }
325     try {
326       ht2.get(get);
327     } catch (IOException e) {
328       ok = false;
329     }
330     assertTrue(ok);
331 
332     ht1.close();
333     ht2.close();
334   }
335 
336   @Test
337   public void testCreateTable() throws IOException {
338     HTableDescriptor [] tables = admin.listTables();
339     int numTables = tables.length;
340     TEST_UTIL.createTable(Bytes.toBytes("testCreateTable"),
341       HConstants.CATALOG_FAMILY).close();
342     tables = this.admin.listTables();
343     assertEquals(numTables + 1, tables.length);
344     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
345         .getMaster().getAssignmentManager().getZKTable().isEnabledTable(
346             "testCreateTable"));
347   }
348 
349   @Test
350   public void testGetTableDescriptor() throws IOException {
351     HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
352     HColumnDescriptor fam2 = new HColumnDescriptor("fam2");
353     HColumnDescriptor fam3 = new HColumnDescriptor("fam3");
354     HTableDescriptor htd = new HTableDescriptor("myTestTable");
355     htd.addFamily(fam1);
356     htd.addFamily(fam2);
357     htd.addFamily(fam3);
358     this.admin.createTable(htd);
359     HTable table = new HTable(TEST_UTIL.getConfiguration(), "myTestTable");
360     HTableDescriptor confirmedHtd = table.getTableDescriptor();
361     assertEquals(htd.compareTo(confirmedHtd), 0);
362     table.close();
363   }
364 
365   @Test
366   public void testHColumnValidName() {
367        boolean exceptionThrown = false;
368        try {
369        HColumnDescriptor fam1 = new HColumnDescriptor("\\test\\abc");
370        } catch(IllegalArgumentException iae) {
371            exceptionThrown = true;
372            assertTrue(exceptionThrown);
373        }
374    }
375   /**
376    * Verify schema modification takes.
377    * @throws IOException
378    * @throws InterruptedException
379    */
380   @Test
381   public void testOnlineChangeTableSchema() throws IOException, InterruptedException {
382     final byte [] tableName = Bytes.toBytes("changeTableSchemaOnline");
383     TEST_UTIL.getMiniHBaseCluster().getMaster().getConfiguration().setBoolean(
384         "hbase.online.schema.update.enable", true);
385     HTableDescriptor [] tables = admin.listTables();
386     int numTables = tables.length;
387     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
388     tables = this.admin.listTables();
389     assertEquals(numTables + 1, tables.length);
390 
391     // FIRST, do htabledescriptor changes.
392     HTableDescriptor htd = this.admin.getTableDescriptor(tableName);
393     // Make a copy and assert copy is good.
394     HTableDescriptor copy = new HTableDescriptor(htd);
395     assertTrue(htd.equals(copy));
396     // Now amend the copy. Introduce differences.
397     long newFlushSize = htd.getMemStoreFlushSize() / 2;
398     if (newFlushSize <=0) {
399       newFlushSize = HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE / 2;
400     }
401     copy.setMemStoreFlushSize(newFlushSize);
402     final String key = "anyoldkey";
403     assertTrue(htd.getValue(key) == null);
404     copy.setValue(key, key);
405     boolean expectedException = false;
406     try {
407       modifyTable(tableName, copy);
408     } catch (TableNotDisabledException re) {
409       expectedException = true;
410     }
411     assertFalse(expectedException);
412     HTableDescriptor modifiedHtd = this.admin.getTableDescriptor(tableName);
413     assertFalse(htd.equals(modifiedHtd));
414     assertTrue(copy.equals(modifiedHtd));
415     assertEquals(newFlushSize, modifiedHtd.getMemStoreFlushSize());
416     assertEquals(key, modifiedHtd.getValue(key));
417 
418     // Now work on column family changes.
419     htd = this.admin.getTableDescriptor(tableName);
420     int countOfFamilies = modifiedHtd.getFamilies().size();
421     assertTrue(countOfFamilies > 0);
422     HColumnDescriptor hcd = modifiedHtd.getFamilies().iterator().next();
423     int maxversions = hcd.getMaxVersions();
424     final int newMaxVersions = maxversions + 1;
425     hcd.setMaxVersions(newMaxVersions);
426     final byte [] hcdName = hcd.getName();
427     expectedException = false;
428     try {
429       this.admin.modifyColumn(tableName, hcd);
430     } catch (TableNotDisabledException re) {
431       expectedException = true;
432     }
433     assertFalse(expectedException);
434     modifiedHtd = this.admin.getTableDescriptor(tableName);
435     HColumnDescriptor modifiedHcd = modifiedHtd.getFamily(hcdName);
436     assertEquals(newMaxVersions, modifiedHcd.getMaxVersions());
437 
438     // Try adding a column
439     assertFalse(this.admin.isTableDisabled(tableName));
440     final String xtracolName = "xtracol";
441     htd = this.admin.getTableDescriptor(tableName);
442     HColumnDescriptor xtracol = new HColumnDescriptor(xtracolName);
443     xtracol.setValue(xtracolName, xtracolName);
444     expectedException = false;
445     try {
446       this.admin.addColumn(tableName, xtracol);
447     } catch (TableNotDisabledException re) {
448       expectedException = true;
449     }
450     // Add column should work even if the table is enabled
451     assertFalse(expectedException);
452     modifiedHtd = this.admin.getTableDescriptor(tableName);
453     hcd = modifiedHtd.getFamily(xtracol.getName());
454     assertTrue(hcd != null);
455     assertTrue(hcd.getValue(xtracolName).equals(xtracolName));
456 
457     // Delete the just-added column.
458     this.admin.deleteColumn(tableName, xtracol.getName());
459     modifiedHtd = this.admin.getTableDescriptor(tableName);
460     hcd = modifiedHtd.getFamily(xtracol.getName());
461     assertTrue(hcd == null);
462 
463     // Delete the table
464     this.admin.disableTable(tableName);
465     this.admin.deleteTable(tableName);
466     this.admin.listTables();
467     assertFalse(this.admin.tableExists(tableName));
468   }
469 
470   @Test
471   public void testShouldFailOnlineSchemaUpdateIfOnlineSchemaIsNotEnabled()
472       throws Exception {
473     final byte[] tableName = Bytes.toBytes("changeTableSchemaOnlineFailure");
474     TEST_UTIL.getMiniHBaseCluster().getMaster().getConfiguration().setBoolean(
475         "hbase.online.schema.update.enable", false);
476     HTableDescriptor[] tables = admin.listTables();
477     int numTables = tables.length;
478     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
479     tables = this.admin.listTables();
480     assertEquals(numTables + 1, tables.length);
481 
482     // FIRST, do htabledescriptor changes.
483     HTableDescriptor htd = this.admin.getTableDescriptor(tableName);
484     // Make a copy and assert copy is good.
485     HTableDescriptor copy = new HTableDescriptor(htd);
486     assertTrue(htd.equals(copy));
487     // Now amend the copy. Introduce differences.
488     long newFlushSize = htd.getMemStoreFlushSize() / 2;
489     if (newFlushSize <=0) {
490       newFlushSize = HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE / 2;
491     }
492     copy.setMemStoreFlushSize(newFlushSize);
493     final String key = "anyoldkey";
494     assertTrue(htd.getValue(key) == null);
495     copy.setValue(key, key);
496     boolean expectedException = false;
497     try {
498       modifyTable(tableName, copy);
499     } catch (TableNotDisabledException re) {
500       expectedException = true;
501     }
502     assertTrue("Online schema update should not happen.", expectedException);
503   }
504 
505   /**
506    * Modify table is async so wait on completion of the table operation in master.
507    * @param tableName
508    * @param htd
509    * @throws IOException
510    */
511   private void modifyTable(final byte [] tableName, final HTableDescriptor htd)
512   throws IOException {
513     MasterServices services = TEST_UTIL.getMiniHBaseCluster().getMaster();
514     ExecutorService executor = services.getExecutorService();
515     AtomicBoolean done = new AtomicBoolean(false);
516     executor.registerListener(EventType.C_M_MODIFY_TABLE, new DoneListener(done));
517     this.admin.modifyTable(tableName, htd);
518     while (!done.get()) {
519       synchronized (done) {
520         try {
521           done.wait(100);
522         } catch (InterruptedException e) {
523           e.printStackTrace();
524         }
525       }
526     }
527     executor.unregisterListener(EventType.C_M_MODIFY_TABLE);
528   }
529 
530   /**
531    * Listens for when an event is done in Master.
532    */
533   static class DoneListener implements EventHandler.EventHandlerListener {
534     private final AtomicBoolean done;
535 
536     DoneListener(final AtomicBoolean done) {
537       super();
538       this.done = done;
539     }
540 
541     @Override
542     public void afterProcess(EventHandler event) {
543       this.done.set(true);
544       synchronized (this.done) {
545         // Wake anyone waiting on this value to change.
546         this.done.notifyAll();
547       }
548     }
549 
550     @Override
551     public void beforeProcess(EventHandler event) {
552       // continue
553     }
554   }
555 
556   protected void verifyRoundRobinDistribution(HTable ht, int expectedRegions) throws IOException {
557     int numRS = ht.getConnection().getCurrentNrHRS();
558     Map<HRegionInfo,HServerAddress> regions = ht.getRegionsInfo();
559     Map<HServerAddress, List<HRegionInfo>> server2Regions = new HashMap<HServerAddress, List<HRegionInfo>>();
560     for (Map.Entry<HRegionInfo,HServerAddress> entry : regions.entrySet()) {
561       HServerAddress server = entry.getValue();
562       List<HRegionInfo> regs = server2Regions.get(server);
563       if (regs == null) {
564         regs = new ArrayList<HRegionInfo>();
565         server2Regions.put(server, regs);
566       }
567       regs.add(entry.getKey());
568     }
569     float average = (float) expectedRegions/numRS;
570     int min = (int)Math.floor(average);
571     int max = (int)Math.ceil(average);
572     for (List<HRegionInfo> regionList : server2Regions.values()) {
573       assertTrue(regionList.size() == min || regionList.size() == max);
574     }
575   }
576 
577   @Test
578   public void testCreateTableNumberOfRegions() throws IOException, InterruptedException {
579     byte[] tableName = Bytes.toBytes("testCreateTableNumberOfRegions");
580     HTableDescriptor desc = new HTableDescriptor(tableName);
581     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
582     admin.createTable(desc);
583     HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
584     Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
585     assertEquals("Table should have only 1 region", 1, regions.size());
586     ht.close();
587 
588     byte[] TABLE_2 = Bytes.add(tableName, Bytes.toBytes("_2"));
589     desc = new HTableDescriptor(TABLE_2);
590     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
591     admin.createTable(desc, new byte[][] { new byte[] { 42 } });
592     HTable ht2 = new HTable(TEST_UTIL.getConfiguration(), TABLE_2);
593     regions = ht2.getRegionLocations();
594     assertEquals("Table should have only 2 region", 2, regions.size());
595     ht2.close();
596 
597     byte[] TABLE_3 = Bytes.add(tableName, Bytes.toBytes("_3"));
598     desc = new HTableDescriptor(TABLE_3);
599     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
600     admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3);
601     HTable ht3 = new HTable(TEST_UTIL.getConfiguration(), TABLE_3);
602     regions = ht3.getRegionLocations();
603     assertEquals("Table should have only 3 region", 3, regions.size());
604     ht3.close();
605 
606     byte[] TABLE_4 = Bytes.add(tableName, Bytes.toBytes("_4"));
607     desc = new HTableDescriptor(TABLE_4);
608     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
609     try {
610       admin.createTable(desc, "a".getBytes(), "z".getBytes(), 2);
611       fail("Should not be able to create a table with only 2 regions using this API.");
612     } catch (IllegalArgumentException eae) {
613       // Expected
614     }
615 
616     byte[] TABLE_5 = Bytes.add(tableName, Bytes.toBytes("_5"));
617     desc = new HTableDescriptor(TABLE_5);
618     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
619     admin.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
620     HTable ht5 = new HTable(TEST_UTIL.getConfiguration(), TABLE_5);
621     regions = ht5.getRegionLocations();
622     assertEquals("Table should have 16 region", 16, regions.size());
623     ht5.close();
624   }
625 
626   @Test
627   public void testCreateTableWithRegions() throws IOException, InterruptedException {
628 
629     byte[] tableName = Bytes.toBytes("testCreateTableWithRegions");
630 
631     byte [][] splitKeys = {
632         new byte [] { 1, 1, 1 },
633         new byte [] { 2, 2, 2 },
634         new byte [] { 3, 3, 3 },
635         new byte [] { 4, 4, 4 },
636         new byte [] { 5, 5, 5 },
637         new byte [] { 6, 6, 6 },
638         new byte [] { 7, 7, 7 },
639         new byte [] { 8, 8, 8 },
640         new byte [] { 9, 9, 9 },
641     };
642     int expectedRegions = splitKeys.length + 1;
643 
644     HTableDescriptor desc = new HTableDescriptor(tableName);
645     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
646     admin.createTable(desc, splitKeys);
647 
648     HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
649     Map<HRegionInfo,HServerAddress> regions = ht.getRegionsInfo();
650     assertEquals("Tried to create " + expectedRegions + " regions " +
651         "but only found " + regions.size(),
652         expectedRegions, regions.size());
653     System.err.println("Found " + regions.size() + " regions");
654 
655     Iterator<HRegionInfo> hris = regions.keySet().iterator();
656     HRegionInfo hri = hris.next();
657     assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
658     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[0]));
659     hri = hris.next();
660     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[0]));
661     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1]));
662     hri = hris.next();
663     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[1]));
664     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2]));
665     hri = hris.next();
666     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[2]));
667     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3]));
668     hri = hris.next();
669     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[3]));
670     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4]));
671     hri = hris.next();
672     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[4]));
673     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5]));
674     hri = hris.next();
675     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[5]));
676     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6]));
677     hri = hris.next();
678     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[6]));
679     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7]));
680     hri = hris.next();
681     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[7]));
682     assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8]));
683     hri = hris.next();
684     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
685     assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
686 
687     verifyRoundRobinDistribution(ht, expectedRegions);
688     ht.close();
689 
690     // Now test using start/end with a number of regions
691 
692     // Use 80 bit numbers to make sure we aren't limited
693     byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
694     byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
695 
696     // Splitting into 10 regions, we expect (null,1) ... (9, null)
697     // with (1,2) (2,3) (3,4) (4,5) (5,6) (6,7) (7,8) (8,9) in the middle
698 
699     expectedRegions = 10;
700 
701     byte [] TABLE_2 = Bytes.add(tableName, Bytes.toBytes("_2"));
702 
703     desc = new HTableDescriptor(TABLE_2);
704     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
705     admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
706     admin.createTable(desc, startKey, endKey, expectedRegions);
707 
708     HTable ht2 = new HTable(TEST_UTIL.getConfiguration(), TABLE_2);
709     regions = ht2.getRegionsInfo();
710     assertEquals("Tried to create " + expectedRegions + " regions " +
711         "but only found " + regions.size(),
712         expectedRegions, regions.size());
713     System.err.println("Found " + regions.size() + " regions");
714 
715     hris = regions.keySet().iterator();
716     hri = hris.next();
717     assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
718     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {1,1,1,1,1,1,1,1,1,1}));
719     hri = hris.next();
720     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {1,1,1,1,1,1,1,1,1,1}));
721     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {2,2,2,2,2,2,2,2,2,2}));
722     hri = hris.next();
723     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {2,2,2,2,2,2,2,2,2,2}));
724     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {3,3,3,3,3,3,3,3,3,3}));
725     hri = hris.next();
726     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {3,3,3,3,3,3,3,3,3,3}));
727     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {4,4,4,4,4,4,4,4,4,4}));
728     hri = hris.next();
729     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {4,4,4,4,4,4,4,4,4,4}));
730     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {5,5,5,5,5,5,5,5,5,5}));
731     hri = hris.next();
732     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {5,5,5,5,5,5,5,5,5,5}));
733     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {6,6,6,6,6,6,6,6,6,6}));
734     hri = hris.next();
735     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {6,6,6,6,6,6,6,6,6,6}));
736     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {7,7,7,7,7,7,7,7,7,7}));
737     hri = hris.next();
738     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {7,7,7,7,7,7,7,7,7,7}));
739     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {8,8,8,8,8,8,8,8,8,8}));
740     hri = hris.next();
741     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {8,8,8,8,8,8,8,8,8,8}));
742     assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {9,9,9,9,9,9,9,9,9,9}));
743     hri = hris.next();
744     assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {9,9,9,9,9,9,9,9,9,9}));
745     assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
746 
747     verifyRoundRobinDistribution(ht2, expectedRegions);
748     ht2.close();
749 
750     // Try once more with something that divides into something infinite
751 
752     startKey = new byte [] { 0, 0, 0, 0, 0, 0 };
753     endKey = new byte [] { 1, 0, 0, 0, 0, 0 };
754 
755     expectedRegions = 5;
756 
757     byte [] TABLE_3 = Bytes.add(tableName, Bytes.toBytes("_3"));
758 
759     desc = new HTableDescriptor(TABLE_3);
760     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
761     admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
762     admin.createTable(desc, startKey, endKey, expectedRegions);
763 
764 
765     HTable ht3 = new HTable(TEST_UTIL.getConfiguration(), TABLE_3);
766     regions = ht3.getRegionsInfo();
767     assertEquals("Tried to create " + expectedRegions + " regions " +
768         "but only found " + regions.size(),
769         expectedRegions, regions.size());
770     System.err.println("Found " + regions.size() + " regions");
771 
772     verifyRoundRobinDistribution(ht3, expectedRegions);
773     ht3.close();
774 
775 
776     // Try an invalid case where there are duplicate split keys
777     splitKeys = new byte [][] {
778         new byte [] { 1, 1, 1 },
779         new byte [] { 2, 2, 2 },
780         new byte [] { 3, 3, 3 },
781         new byte [] { 2, 2, 2 }
782     };
783 
784     byte [] TABLE_4 = Bytes.add(tableName, Bytes.toBytes("_4"));
785     desc = new HTableDescriptor(TABLE_4);
786     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
787     HBaseAdmin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
788     try {
789       ladmin.createTable(desc, splitKeys);
790       assertTrue("Should not be able to create this table because of " +
791           "duplicate split keys", false);
792     } catch(IllegalArgumentException iae) {
793       // Expected
794     }
795     ladmin.close();
796   }
797 
798 
799   @Test
800   public void testCreateTableWithOnlyEmptyStartRow() throws IOException {
801     byte[] tableName = Bytes.toBytes("testCreateTableWithOnlyEmptyStartRow");
802     byte[][] splitKeys = new byte[1][];
803     splitKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
804     HTableDescriptor desc = new HTableDescriptor(tableName);
805     desc.addFamily(new HColumnDescriptor("col"));
806     try {
807       admin.createTable(desc, splitKeys);
808       fail("Test case should fail as empty split key is passed.");
809     } catch (IllegalArgumentException e) {
810     }
811   }
812 
813   @Test
814   public void testCreateTableWithEmptyRowInTheSplitKeys() throws IOException {
815     byte[] tableName = Bytes
816         .toBytes("testCreateTableWithEmptyRowInTheSplitKeys");
817     byte[][] splitKeys = new byte[3][];
818     splitKeys[0] = "region1".getBytes();
819     splitKeys[1] = HConstants.EMPTY_BYTE_ARRAY;
820     splitKeys[2] = "region2".getBytes();
821     HTableDescriptor desc = new HTableDescriptor(tableName);
822     desc.addFamily(new HColumnDescriptor("col"));
823     try {
824       admin.createTable(desc, splitKeys);
825       fail("Test case should fail as empty split key is passed.");
826     } catch (IllegalArgumentException e) {
827     }
828   }
829 
830   @Test
831   public void testTableExist() throws IOException {
832     final byte [] table = Bytes.toBytes("testTableExist");
833     boolean exist = false;
834     exist = this.admin.tableExists(table);
835     assertEquals(false, exist);
836     TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
837     exist = this.admin.tableExists(table);
838     assertEquals(true, exist);
839   }
840 
841   /**
842    * Tests forcing split from client and having scanners successfully ride over split.
843    * @throws Exception
844    * @throws IOException
845    */
846   @Test
847   public void testForceSplit() throws Exception {
848     byte[][] familyNames = new byte[][] { Bytes.toBytes("cf") };
849     int[] rowCounts = new int[] { 6000 };
850     int numVersions = HColumnDescriptor.DEFAULT_VERSIONS;
851     int blockSize = 256;
852     splitTest(null, familyNames, rowCounts, numVersions, blockSize);
853 
854     byte[] splitKey = Bytes.toBytes(3500);
855     splitTest(splitKey, familyNames, rowCounts, numVersions, blockSize);
856   }
857 
858   /**
859    * Test round-robin assignment on enableTable.
860    *
861    * @throws IOException
862    */
863   @Test
864   public void testEnableTableRoundRobinAssignment() throws IOException {
865     byte[] tableName = Bytes.toBytes("testEnableTableAssignment");
866     byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 },
867         new byte[] { 3, 3, 3 }, new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 },
868         new byte[] { 6, 6, 6 }, new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 },
869         new byte[] { 9, 9, 9 } };
870     int expectedRegions = splitKeys.length + 1;
871     HTableDescriptor desc = new HTableDescriptor(tableName);
872     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
873     admin.createTable(desc, splitKeys);
874     HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
875     Map<HRegionInfo, HServerAddress> regions = ht.getRegionsInfo();
876     assertEquals("Tried to create " + expectedRegions + " regions "
877         + "but only found " + regions.size(), expectedRegions, regions.size());
878     // Disable table.
879     admin.disableTable(tableName);
880     // Enable table, use round-robin assignment to assign regions.
881     admin.enableTable(tableName);
882 
883     // Check the assignment.
884     HTable metaTable = new HTable(TEST_UTIL.getConfiguration(),
885         HConstants.META_TABLE_NAME);
886     List<HRegionInfo> regionInfos = admin.getTableRegions(tableName);
887     Map<String, Integer> serverMap = new HashMap<String, Integer>();
888     for (int i = 0, j = regionInfos.size(); i < j; i++) {
889       HRegionInfo hri = regionInfos.get(i);
890       Get get = new Get(hri.getRegionName());
891       Result result = metaTable.get(get);
892       String server = Bytes.toString(result.getValue(HConstants.CATALOG_FAMILY,
893           HConstants.SERVER_QUALIFIER));
894       Integer regioncount = serverMap.get(server);
895       if (regioncount == null) {
896         regioncount = 0;
897       }
898       regioncount++;
899       serverMap.put(server, regioncount);
900     }
901     List<Map.Entry<String, Integer>> entryList = new ArrayList<Map.Entry<String, Integer>>(
902         serverMap.entrySet());
903     Collections.sort(entryList, new Comparator<Map.Entry<String, Integer>>() {
904       public int compare(Map.Entry<String, Integer> oa,
905           Map.Entry<String, Integer> ob) {
906         return (oa.getValue() - ob.getValue());
907       }
908     });
909     assertTrue(entryList.size() == 3);
910     assertTrue((entryList.get(2).getValue() - entryList.get(0).getValue()) < 2);
911   }
912 
913   /**
914    * Multi-family scenario. Tests forcing split from client and
915    * having scanners successfully ride over split.
916    * @throws Exception
917    * @throws IOException
918    */
919   @Test
920   public void testForceSplitMultiFamily() throws Exception {
921     int numVersions = HColumnDescriptor.DEFAULT_VERSIONS;
922 
923     // use small HFile block size so that we can have lots of blocks in HFile
924     // Otherwise, if there is only one block,
925     // HFileBlockIndex.midKey()'s value == startKey
926     int blockSize = 256;
927     byte[][] familyNames = new byte[][] { Bytes.toBytes("cf1"),
928       Bytes.toBytes("cf2") };
929 
930     // one of the column families isn't splittable
931     int[] rowCounts = new int[] { 6000, 1 };
932     splitTest(null, familyNames, rowCounts, numVersions, blockSize);
933 
934     rowCounts = new int[] { 1, 6000 };
935     splitTest(null, familyNames, rowCounts, numVersions, blockSize);
936 
937     // one column family has much smaller data than the other
938     // the split key should be based on the largest column family
939     rowCounts = new int[] { 6000, 300 };
940     splitTest(null, familyNames, rowCounts, numVersions, blockSize);
941 
942     rowCounts = new int[] { 300, 6000 };
943     splitTest(null, familyNames, rowCounts, numVersions, blockSize);
944 
945   }
946 
947   void splitTest(byte[] splitPoint, byte[][] familyNames, int[] rowCounts,
948     int numVersions, int blockSize) throws Exception {
949     byte [] tableName = Bytes.toBytes("testForceSplit");
950     assertFalse(admin.tableExists(tableName));
951     final HTable table = TEST_UTIL.createTable(tableName, familyNames,
952       numVersions, blockSize);
953     int rowCount = 0;
954     byte[] q = new byte[0];
955 
956     // insert rows into column families. The number of rows that have values
957     // in a specific column family is decided by rowCounts[familyIndex]
958     for (int index = 0; index < familyNames.length; index++) {
959       ArrayList<Put> puts = new ArrayList<Put>(rowCounts[index]);
960       for (int i = 0; i < rowCounts[index]; i++) {
961         byte[] k = Bytes.toBytes(i);
962         Put put = new Put(k);
963         put.add(familyNames[index], q, k);
964         puts.add(put);
965       }
966       table.put(puts);
967 
968       if ( rowCount < rowCounts[index] ) {
969         rowCount = rowCounts[index];
970       }
971     }
972 
973     // get the initial layout (should just be one region)
974     Map<HRegionInfo,HServerAddress> m = table.getRegionsInfo();
975     System.out.println("Initial regions (" + m.size() + "): " + m);
976     assertTrue(m.size() == 1);
977 
978     // Verify row count
979     Scan scan = new Scan();
980     ResultScanner scanner = table.getScanner(scan);
981     int rows = 0;
982     for(@SuppressWarnings("unused") Result result : scanner) {
983       rows++;
984     }
985     scanner.close();
986     assertEquals(rowCount, rows);
987 
988     // Have an outstanding scan going on to make sure we can scan over splits.
989     scan = new Scan();
990     scanner = table.getScanner(scan);
991     // Scan first row so we are into first region before split happens.
992     scanner.next();
993 
994     final AtomicInteger count = new AtomicInteger(0);
995     Thread t = new Thread("CheckForSplit") {
996       public void run() {
997         for (int i = 0; i < 20; i++) {
998           try {
999             sleep(1000);
1000           } catch (InterruptedException e) {
1001             continue;
1002           }
1003           // check again    table = new HTable(conf, tableName);
1004           Map<HRegionInfo, HServerAddress> regions = null;
1005           try {
1006             regions = table.getRegionsInfo();
1007           } catch (IOException e) {
1008             e.printStackTrace();
1009           }
1010           if (regions == null) continue;
1011           count.set(regions.size());
1012           if (count.get() >= 2) break;
1013           LOG.debug("Cycle waiting on split");
1014         }
1015       }
1016     };
1017     t.start();
1018     // Split the table
1019     this.admin.split(tableName, splitPoint);
1020     t.join();
1021 
1022     // Verify row count
1023     rows = 1; // We counted one row above.
1024     for (@SuppressWarnings("unused") Result result : scanner) {
1025       rows++;
1026       if (rows > rowCount) {
1027         scanner.close();
1028         assertTrue("Scanned more than expected (" + rowCount + ")", false);
1029       }
1030     }
1031     scanner.close();
1032     assertEquals(rowCount, rows);
1033 
1034     Map<HRegionInfo, HServerAddress> regions = null;
1035     try {
1036       regions = table.getRegionsInfo();
1037     } catch (IOException e) {
1038       e.printStackTrace();
1039     }
1040     assertEquals(2, regions.size());
1041     HRegionInfo[] r = regions.keySet().toArray(new HRegionInfo[0]);
1042     if (splitPoint != null) {
1043       // make sure the split point matches our explicit configuration
1044       assertEquals(Bytes.toString(splitPoint),
1045           Bytes.toString(r[0].getEndKey()));
1046       assertEquals(Bytes.toString(splitPoint),
1047           Bytes.toString(r[1].getStartKey()));
1048       LOG.debug("Properly split on " + Bytes.toString(splitPoint));
1049     } else {
1050       if (familyNames.length > 1) {
1051         int splitKey = Bytes.toInt(r[0].getEndKey());
1052         // check if splitKey is based on the largest column family
1053         // in terms of it store size
1054         int deltaForLargestFamily = Math.abs(rowCount/2 - splitKey);
1055         for (int index = 0; index < familyNames.length; index++) {
1056           int delta = Math.abs(rowCounts[index]/2 - splitKey);
1057           assertTrue(delta >= deltaForLargestFamily);
1058         }
1059       }
1060     }
1061     TEST_UTIL.deleteTable(tableName);
1062     table.close();
1063   }
1064 
1065   /**
1066    * HADOOP-2156
1067    * @throws IOException
1068    */
1069   @Test (expected=IllegalArgumentException.class)
1070   public void testEmptyHHTableDescriptor() throws IOException {
1071     this.admin.createTable(new HTableDescriptor());
1072   }
1073 
1074   @Test (expected=IllegalArgumentException.class)
1075   public void testInvalidHColumnDescriptor() throws IOException {
1076      new HColumnDescriptor("/cfamily/name");
1077   }
1078 
1079   @Test(timeout=300000)
1080   public void testEnableDisableAddColumnDeleteColumn() throws Exception {
1081     ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
1082     byte [] tableName = Bytes.toBytes("testMasterAdmin");
1083     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
1084     while (!ZKTableReadOnly.isEnabledTable(zkw, "testMasterAdmin")) {
1085       Thread.sleep(10);
1086     }
1087     this.admin.disableTable(tableName);
1088     try {
1089       new HTable(TEST_UTIL.getConfiguration(), tableName);
1090     } catch (DoNotRetryIOException e) {
1091       //expected
1092     }
1093 
1094     this.admin.addColumn(tableName, new HColumnDescriptor("col2"));
1095     this.admin.enableTable(tableName);
1096     try {
1097       this.admin.deleteColumn(tableName, Bytes.toBytes("col2"));
1098     } catch (TableNotDisabledException e) {
1099       LOG.info(e);
1100     }
1101     this.admin.disableTable(tableName);
1102     this.admin.deleteTable(tableName);
1103   }
1104 
1105   @Test
1106   public void testCreateBadTables() throws IOException {
1107     String msg = null;
1108     try {
1109       this.admin.createTable(HTableDescriptor.ROOT_TABLEDESC);
1110     } catch (IllegalArgumentException e) {
1111       msg = e.toString();
1112     }
1113     assertTrue("Unexcepted exception message " + msg, msg != null &&
1114       msg.startsWith(IllegalArgumentException.class.getName()) &&
1115       msg.contains(HTableDescriptor.ROOT_TABLEDESC.getNameAsString()));
1116     msg = null;
1117     try {
1118       this.admin.createTable(HTableDescriptor.META_TABLEDESC);
1119     } catch(IllegalArgumentException e) {
1120       msg = e.toString();
1121     }
1122     assertTrue("Unexcepted exception message " + msg, msg != null &&
1123       msg.startsWith(IllegalArgumentException.class.getName()) &&
1124       msg.contains(HTableDescriptor.META_TABLEDESC.getNameAsString()));
1125 
1126     // Now try and do concurrent creation with a bunch of threads.
1127     final HTableDescriptor threadDesc =
1128       new HTableDescriptor("threaded_testCreateBadTables");
1129     threadDesc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
1130     int count = 10;
1131     Thread [] threads = new Thread [count];
1132     final AtomicInteger successes = new AtomicInteger(0);
1133     final AtomicInteger failures = new AtomicInteger(0);
1134     final HBaseAdmin localAdmin = this.admin;
1135     for (int i = 0; i < count; i++) {
1136       threads[i] = new Thread(Integer.toString(i)) {
1137         @Override
1138         public void run() {
1139           try {
1140             localAdmin.createTable(threadDesc);
1141             successes.incrementAndGet();
1142           } catch (TableExistsException e) {
1143             failures.incrementAndGet();
1144           } catch (IOException e) {
1145             throw new RuntimeException("Failed threaded create" + getName(), e);
1146           }
1147         }
1148       };
1149     }
1150     for (int i = 0; i < count; i++) {
1151       threads[i].start();
1152     }
1153     for (int i = 0; i < count; i++) {
1154       while(threads[i].isAlive()) {
1155         try {
1156           Thread.sleep(100);
1157         } catch (InterruptedException e) {
1158           // continue
1159         }
1160       }
1161     }
1162     // All threads are now dead.  Count up how many tables were created and
1163     // how many failed w/ appropriate exception.
1164     assertEquals(1, successes.get());
1165     assertEquals(count - 1, failures.get());
1166   }
1167 
1168   /**
1169    * Test for hadoop-1581 'HBASE: Unopenable tablename bug'.
1170    * @throws Exception
1171    */
1172   @Test
1173   public void testTableNameClash() throws Exception {
1174     String name = "testTableNameClash";
1175     admin.createTable(new HTableDescriptor(name + "SOMEUPPERCASE"));
1176     admin.createTable(new HTableDescriptor(name));
1177     // Before fix, below would fail throwing a NoServerForRegionException.
1178     new HTable(TEST_UTIL.getConfiguration(), name).close();
1179   }
1180 
1181   /***
1182    * HMaster.createTable used to be kind of synchronous call
1183    * Thus creating of table with lots of regions can cause RPC timeout
1184    * After the fix to make createTable truly async, RPC timeout shouldn't be an
1185    * issue anymore
1186    * @throws Exception
1187    */
1188   @Test
1189   public void testCreateTableRPCTimeOut() throws Exception {
1190     String name = "testCreateTableRPCTimeOut";
1191     int oldTimeout = TEST_UTIL.getConfiguration().
1192       getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
1193     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500);
1194     try {
1195       int expectedRegions = 100;
1196       // Use 80 bit numbers to make sure we aren't limited
1197       byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
1198       byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
1199       HBaseAdmin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
1200       hbaseadmin.createTable(new HTableDescriptor(name), startKey, endKey,
1201         expectedRegions);
1202       hbaseadmin.close();
1203     } finally {
1204       TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, oldTimeout);
1205     }
1206   }
1207 
1208   /**
1209    * Test read only tables
1210    * @throws Exception
1211    */
1212   @Test
1213   public void testReadOnlyTable() throws Exception {
1214     byte [] name = Bytes.toBytes("testReadOnlyTable");
1215     HTable table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
1216     byte[] value = Bytes.toBytes("somedata");
1217     // This used to use an empty row... That must have been a bug
1218     Put put = new Put(value);
1219     put.add(HConstants.CATALOG_FAMILY, HConstants.CATALOG_FAMILY, value);
1220     table.put(put);
1221     table.close();
1222   }
1223 
1224   /**
1225    * Test that user table names can contain '-' and '.' so long as they do not
1226    * start with same. HBASE-771
1227    * @throws IOException
1228    */
1229   @Test
1230   public void testTableNames() throws IOException {
1231     byte[][] illegalNames = new byte[][] {
1232         Bytes.toBytes("-bad"),
1233         Bytes.toBytes(".bad"),
1234         HConstants.ROOT_TABLE_NAME,
1235         HConstants.META_TABLE_NAME
1236     };
1237     for (int i = 0; i < illegalNames.length; i++) {
1238       try {
1239         new HTableDescriptor(illegalNames[i]);
1240         throw new IOException("Did not detect '" +
1241           Bytes.toString(illegalNames[i]) + "' as an illegal user table name");
1242       } catch (IllegalArgumentException e) {
1243         // expected
1244       }
1245     }
1246     byte[] legalName = Bytes.toBytes("g-oo.d");
1247     try {
1248       new HTableDescriptor(legalName);
1249     } catch (IllegalArgumentException e) {
1250       throw new IOException("Legal user table name: '" +
1251         Bytes.toString(legalName) + "' caused IllegalArgumentException: " +
1252         e.getMessage());
1253     }
1254   }
1255 
1256   /**
1257    * For HADOOP-2579
1258    * @throws IOException
1259    */
1260   @Test (expected=TableExistsException.class)
1261   public void testTableExistsExceptionWithATable() throws IOException {
1262     final byte [] name = Bytes.toBytes("testTableExistsExceptionWithATable");
1263     TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY).close();
1264     TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
1265   }
1266 
1267   /**
1268    * Can't disable a table if the table isn't in enabled state
1269    * @throws IOException
1270    */
1271   @Test (expected=TableNotEnabledException.class)
1272   public void testTableNotEnabledExceptionWithATable() throws IOException {
1273     final byte [] name = Bytes.toBytes(
1274       "testTableNotEnabledExceptionWithATable");
1275     TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY).close();
1276     this.admin.disableTable(name);
1277     this.admin.disableTable(name);
1278   }
1279 
1280   /**
1281    * Can't enable a table if the table isn't in disabled state
1282    * @throws IOException
1283    */
1284   @Test (expected=TableNotDisabledException.class)
1285   public void testTableNotDisabledExceptionWithATable() throws IOException {
1286     final byte [] name = Bytes.toBytes(
1287       "testTableNotDisabledExceptionWithATable");
1288     HTable t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
1289     try {
1290     this.admin.enableTable(name);
1291     }finally {
1292        t.close();
1293     }
1294   }
1295 
1296   /**
1297    * For HADOOP-2579
1298    * @throws IOException
1299    */
1300   @Test (expected=TableNotFoundException.class)
1301   public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
1302     new HTable(TEST_UTIL.getConfiguration(),
1303         "testTableNotFoundExceptionWithoutAnyTables");
1304   }
1305   @Test
1306   public void testShouldCloseTheRegionBasedOnTheEncodedRegionName()
1307       throws Exception {
1308     byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion");
1309     createTableWithDefaultConf(TABLENAME);
1310 
1311     HRegionInfo info = null;
1312     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
1313     List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
1314     for (HRegionInfo regionInfo : onlineRegions) {
1315       if (!regionInfo.isMetaTable()) {
1316         info = regionInfo;
1317         admin.closeRegionWithEncodedRegionName(regionInfo.getEncodedName(), rs
1318             .getServerName().getServerName());
1319       }
1320     }
1321     boolean isInList = rs.getOnlineRegions().contains(info);
1322     long timeout = System.currentTimeMillis() + 10000;
1323     while ((System.currentTimeMillis() < timeout) && (isInList)) {
1324       Thread.sleep(100);
1325       isInList = rs.getOnlineRegions().contains(info);
1326     }
1327     assertFalse("The region should not be present in online regions list.",
1328         isInList);
1329   }
1330 
1331   @Test
1332   public void testCloseRegionIfInvalidRegionNameIsPassed() throws Exception {
1333     byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion1");
1334     createTableWithDefaultConf(TABLENAME);
1335 
1336     HRegionInfo info = null;
1337     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
1338     List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
1339     for (HRegionInfo regionInfo : onlineRegions) {
1340       if (!regionInfo.isMetaTable()) {
1341         if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion1")) {
1342           info = regionInfo;
1343           admin.closeRegionWithEncodedRegionName("sample", rs.getServerName()
1344               .getServerName());
1345         }
1346       }
1347     }
1348     onlineRegions = rs.getOnlineRegions();
1349     assertTrue("The region should be present in online regions list.",
1350         onlineRegions.contains(info));
1351   }
1352 
1353   @Test
1354   public void testCloseRegionThatFetchesTheHRIFromMeta() throws Exception {
1355     byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion2");
1356     createTableWithDefaultConf(TABLENAME);
1357 
1358     HRegionInfo info = null;
1359     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
1360     List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
1361     for (HRegionInfo regionInfo : onlineRegions) {
1362       if (!regionInfo.isMetaTable()) {
1363 
1364         if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion2")) {
1365           info = regionInfo;
1366           admin.closeRegion(regionInfo.getRegionNameAsString(), rs
1367               .getServerName().getServerName());
1368         }
1369       }
1370     }
1371 
1372     boolean isInList = rs.getOnlineRegions().contains(info);
1373     long timeout = System.currentTimeMillis() + 10000;
1374     while ((System.currentTimeMillis() < timeout) && (isInList)) {
1375       Thread.sleep(100);
1376       isInList = rs.getOnlineRegions().contains(info);
1377     }
1378 
1379     assertFalse("The region should not be present in online regions list.",
1380       isInList);
1381   }
1382 
1383   @Test
1384   public void testCloseRegionWhenServerNameIsNull() throws Exception {
1385     byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion3");
1386     createTableWithDefaultConf(TABLENAME);
1387 
1388     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
1389 
1390     try {
1391       List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
1392       for (HRegionInfo regionInfo : onlineRegions) {
1393         if (!regionInfo.isMetaTable()) {
1394           if (regionInfo.getRegionNameAsString()
1395               .contains("TestHBACloseRegion3")) {
1396             admin.closeRegionWithEncodedRegionName(regionInfo.getEncodedName(),
1397                 null);
1398           }
1399         }
1400       }
1401       fail("The test should throw exception if the servername passed is null.");
1402     } catch (IllegalArgumentException e) {
1403     }
1404   }
1405 
1406 
1407   @Test
1408   public void testCloseRegionWhenServerNameIsEmpty() throws Exception {
1409     byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegionWhenServerNameIsEmpty");
1410     createTableWithDefaultConf(TABLENAME);
1411 
1412     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
1413 
1414     try {
1415       List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
1416       for (HRegionInfo regionInfo : onlineRegions) {
1417         if (!regionInfo.isMetaTable()) {
1418           if (regionInfo.getRegionNameAsString()
1419               .contains("TestHBACloseRegionWhenServerNameIsEmpty")) {
1420             admin.closeRegionWithEncodedRegionName(regionInfo.getEncodedName(),
1421                 " ");
1422           }
1423         }
1424       }
1425       fail("The test should throw exception if the servername passed is empty.");
1426     } catch (IllegalArgumentException e) {
1427     }
1428   }
1429 
1430   @Test
1431   public void testCloseRegionWhenEncodedRegionNameIsNotGiven() throws Exception {
1432     byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion4");
1433     createTableWithDefaultConf(TABLENAME);
1434 
1435     HRegionInfo info = null;
1436     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
1437 
1438     List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
1439     for (HRegionInfo regionInfo : onlineRegions) {
1440       if (!regionInfo.isMetaTable()) {
1441         if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion4")) {
1442           info = regionInfo;
1443           admin.closeRegionWithEncodedRegionName(regionInfo
1444               .getRegionNameAsString(), rs.getServerName().getServerName());
1445         }
1446       }
1447     }
1448     onlineRegions = rs.getOnlineRegions();
1449     assertTrue("The region should be present in online regions list.",
1450         onlineRegions.contains(info));
1451   }
1452 
1453   private HBaseAdmin createTable(byte[] TABLENAME) throws IOException {
1454 
1455     Configuration config = TEST_UTIL.getConfiguration();
1456     HBaseAdmin admin = new HBaseAdmin(config);
1457 
1458     HTableDescriptor htd = new HTableDescriptor(TABLENAME);
1459     HColumnDescriptor hcd = new HColumnDescriptor("value");
1460 
1461     htd.addFamily(hcd);
1462     admin.createTable(htd, null);
1463     return admin;
1464   }
1465 
1466   private void createTableWithDefaultConf(byte[] TABLENAME) throws IOException {
1467     HTableDescriptor htd = new HTableDescriptor(TABLENAME);
1468     HColumnDescriptor hcd = new HColumnDescriptor("value");
1469     htd.addFamily(hcd);
1470 
1471     admin.createTable(htd, null);
1472   }
1473 
1474   /**
1475    * For HBASE-2556
1476    * @throws IOException
1477    */
1478   @Test
1479   public void testGetTableRegions() throws IOException {
1480 
1481     byte[] tableName = Bytes.toBytes("testGetTableRegions");
1482 
1483     int expectedRegions = 10;
1484 
1485     // Use 80 bit numbers to make sure we aren't limited
1486     byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
1487     byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
1488 
1489 
1490     HTableDescriptor desc = new HTableDescriptor(tableName);
1491     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
1492     admin.createTable(desc, startKey, endKey, expectedRegions);
1493 
1494     List<HRegionInfo> RegionInfos = admin.getTableRegions(tableName);
1495 
1496     assertEquals("Tried to create " + expectedRegions + " regions " +
1497         "but only found " + RegionInfos.size(),
1498         expectedRegions, RegionInfos.size());
1499 
1500  }
1501 
1502   @Test
1503   public void testHLogRollWriting() throws Exception {
1504     setUpforLogRolling();
1505     String className = this.getClass().getName();
1506     StringBuilder v = new StringBuilder(className);
1507     while (v.length() < 1000) {
1508       v.append(className);
1509     }
1510     byte[] value = Bytes.toBytes(v.toString());
1511     HRegionServer regionServer = startAndWriteData("TestLogRolling", value);
1512     LOG.info("after writing there are "
1513         + HLogUtilsForTests.getNumLogFiles(regionServer.getWAL()) + " log files");
1514 
1515     // flush all regions
1516 
1517     List<HRegion> regions = new ArrayList<HRegion>(regionServer
1518         .getOnlineRegionsLocalContext());
1519     for (HRegion r : regions) {
1520       r.flushcache();
1521     }
1522     admin.rollHLogWriter(regionServer.getServerName().getServerName());
1523     int count = HLogUtilsForTests.getNumLogFiles(regionServer.getWAL());
1524     LOG.info("after flushing all regions and rolling logs there are " +
1525         count + " log files");
1526     assertTrue(("actual count: " + count), count <= 2);
1527   }
1528 
1529   private void setUpforLogRolling() {
1530     // Force a region split after every 768KB
1531     TEST_UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE,
1532         768L * 1024L);
1533 
1534     // We roll the log after every 32 writes
1535     TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32);
1536 
1537     TEST_UTIL.getConfiguration().setInt(
1538         "hbase.regionserver.logroll.errors.tolerated", 2);
1539     TEST_UTIL.getConfiguration().setInt("ipc.ping.interval", 10 * 1000);
1540     TEST_UTIL.getConfiguration().setInt("ipc.socket.timeout", 10 * 1000);
1541     TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000);
1542 
1543     // For less frequently updated regions flush after every 2 flushes
1544     TEST_UTIL.getConfiguration().setInt(
1545         "hbase.hregion.memstore.optionalflushcount", 2);
1546 
1547     // We flush the cache after every 8192 bytes
1548     TEST_UTIL.getConfiguration().setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
1549         8192);
1550 
1551     // Increase the amount of time between client retries
1552     TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
1553 
1554     // Reduce thread wake frequency so that other threads can get
1555     // a chance to run.
1556     TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY,
1557         2 * 1000);
1558 
1559     /**** configuration for testLogRollOnDatanodeDeath ****/
1560     // make sure log.hflush() calls syncFs() to open a pipeline
1561     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
1562     // lower the namenode & datanode heartbeat so the namenode
1563     // quickly detects datanode failures
1564     TEST_UTIL.getConfiguration().setInt("heartbeat.recheck.interval", 5000);
1565     TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
1566     // the namenode might still try to choose the recently-dead datanode
1567     // for a pipeline, so try to a new pipeline multiple times
1568     TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30);
1569     TEST_UTIL.getConfiguration().setInt(
1570         "hbase.regionserver.hlog.tolerable.lowreplication", 2);
1571     TEST_UTIL.getConfiguration().setInt(
1572         "hbase.regionserver.hlog.lowreplication.rolllimit", 3);
1573   }
1574 
1575   private HRegionServer startAndWriteData(String tableName, byte[] value)
1576       throws IOException {
1577     // When the META table can be opened, the region servers are running
1578     new HTable(
1579       TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME).close();
1580     HRegionServer regionServer = TEST_UTIL.getHBaseCluster()
1581         .getRegionServerThreads().get(0).getRegionServer();
1582 
1583     // Create the test table and open it
1584     HTableDescriptor desc = new HTableDescriptor(tableName);
1585     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
1586     admin.createTable(desc);
1587     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
1588 
1589     regionServer = TEST_UTIL.getRSForFirstRegionInTable(Bytes
1590         .toBytes(tableName));
1591     for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
1592       Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
1593       put.add(HConstants.CATALOG_FAMILY, null, value);
1594       table.put(put);
1595       if (i % 32 == 0) {
1596         // After every 32 writes sleep to let the log roller run
1597         try {
1598           Thread.sleep(2000);
1599         } catch (InterruptedException e) {
1600           // continue
1601         }
1602       }
1603     }
1604 
1605     table.close();
1606     return regionServer;
1607   }
1608 
1609   /**
1610    * HBASE-4417 checkHBaseAvailable() doesn't close zk connections
1611    */
1612   @Test
1613   public void testCheckHBaseAvailableClosesConnection() throws Exception {
1614     Configuration conf = TEST_UTIL.getConfiguration();
1615 
1616     int initialCount = HConnectionTestingUtility.getConnectionCount();
1617     HBaseAdmin.checkHBaseAvailable(conf);
1618     int finalCount = HConnectionTestingUtility.getConnectionCount();
1619 
1620     Assert.assertEquals(initialCount, finalCount) ;
1621   }
1622 
1623   @Test
1624   public void testDisableCatalogTable() throws Exception {
1625     try {
1626       this.admin.disableTable(".META.");
1627       fail("Expected to throw IllegalArgumentException");
1628     } catch (IllegalArgumentException e) {
1629     }
1630     // Before the fix for HBASE-6146, the below table creation was failing as the META table
1631     // actually getting disabled by the disableTable() call.
1632     HTableDescriptor htd = new HTableDescriptor("testDisableCatalogTable".getBytes());
1633     HColumnDescriptor hcd = new HColumnDescriptor("cf1".getBytes());
1634     htd.addFamily(hcd);
1635     TEST_UTIL.getHBaseAdmin().createTable(htd);
1636   }
1637 
1638   @Test
1639   public void testGetRegion() throws Exception {
1640     final String name = "testGetRegion";
1641     LOG.info("Started " + name);
1642     final byte [] nameBytes = Bytes.toBytes(name);
1643     HTable t = TEST_UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
1644     TEST_UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
1645     CatalogTracker ct = new CatalogTracker(TEST_UTIL.getConfiguration());
1646     ct.start();
1647     try {
1648       HRegionLocation regionLocation = t.getRegionLocation("mmm");
1649       HRegionInfo region = regionLocation.getRegionInfo();
1650       byte[] regionName = region.getRegionName();
1651       Pair<HRegionInfo, ServerName> pair = admin.getRegion(regionName, ct);
1652       assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
1653       pair = admin.getRegion(region.getEncodedNameAsBytes(), ct);
1654       assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
1655     } finally {
1656       ct.stop();
1657     }
1658   }
1659 
1660   @Test
1661   public void testRootTableSplit() throws Exception {
1662     Scan s = new Scan();
1663     HTable rootTable = new HTable(TEST_UTIL.getConfiguration(), HConstants.ROOT_TABLE_NAME);
1664     ResultScanner scanner = rootTable.getScanner(s);
1665     Result metaEntry = scanner.next();
1666     this.admin.split(HConstants.ROOT_TABLE_NAME, metaEntry.getRow());
1667     Thread.sleep(1000);
1668     List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(HConstants.ROOT_TABLE_NAME);
1669     assertEquals("ROOT region should not be splitted.",1, regions.size());
1670   }
1671 
1672   @org.junit.Rule
1673   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
1674     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
1675 }