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