diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java index 94bdf694635..41799556ee2 100644 --- a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java +++ b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java @@ -21,12 +21,11 @@ package org.apache.hbase.archetypes.exemplars.shaded_client; import java.io.IOException; import java.util.Map.Entry; import java.util.NavigableMap; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -34,6 +33,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.util.Bytes; /** @@ -110,8 +110,10 @@ public final class HelloHBase { + "], with one Column Family [" + Bytes.toString(MY_COLUMN_FAMILY_NAME) + "]."); - admin.createTable(new HTableDescriptor(MY_TABLE_NAME) - .addFamily(new HColumnDescriptor(MY_COLUMN_FAMILY_NAME))); + admin.createTable(new TableDescriptorBuilder.ModifyableTableDescriptor(MY_TABLE_NAME) + .setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + MY_COLUMN_FAMILY_NAME))); } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index 64978bcd773..94a8bde142d 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -33,9 +33,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; @@ -49,11 +47,14 @@ import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient; import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.cleaner.LogCleaner; import org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner; import org.apache.hadoop.hbase.security.HadoopSecurityEnabledUserProviderForTesting; @@ -83,7 +84,7 @@ public class TestBackupBase { protected static Configuration conf2; protected static TableName table1 = TableName.valueOf("table1"); - protected static HTableDescriptor table1Desc; + protected static TableDescriptorBuilder.ModifyableTableDescriptor table1Desc; protected static TableName table2 = TableName.valueOf("table2"); protected static TableName table3 = TableName.valueOf("table3"); protected static TableName table4 = TableName.valueOf("table4"); @@ -427,9 +428,11 @@ public class TestBackupBase { ha.createNamespace(desc3); ha.createNamespace(desc4); - HTableDescriptor desc = new HTableDescriptor(table1); - HColumnDescriptor fam = new HColumnDescriptor(famName); - desc.addFamily(fam); + TableDescriptorBuilder.ModifyableTableDescriptor desc = + new TableDescriptorBuilder.ModifyableTableDescriptor(table1); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName); + desc.setColumnFamily(familyDescriptor); ha.createTable(desc); table1Desc = desc; Connection conn = ConnectionFactory.createConnection(conf1); @@ -437,8 +440,8 @@ public class TestBackupBase { loadTable(table); table.close(); table2 = TableName.valueOf("ns2:test-" + tid + 1); - desc = new HTableDescriptor(table2); - desc.addFamily(fam); + desc = new TableDescriptorBuilder.ModifyableTableDescriptor(table2); + desc.setColumnFamily(familyDescriptor); ha.createTable(desc); table = conn.getTable(table2); loadTable(table); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java index d7c2cd020c0..b53c3e6dcf7 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -24,12 +24,12 @@ import java.util.Collection; import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -82,11 +82,13 @@ public class TestIncrementalBackup extends TestBackupBase { final byte[] fam3Name = Bytes.toBytes("f3"); final byte[] mobName = Bytes.toBytes("mob"); - table1Desc.addFamily(new HColumnDescriptor(fam3Name)); - HColumnDescriptor mobHcd = new HColumnDescriptor(mobName); + table1Desc.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name)); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor mobHcd = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(mobName); mobHcd.setMobEnabled(true); mobHcd.setMobThreshold(5L); - table1Desc.addFamily(mobHcd); + table1Desc.setColumnFamily(mobHcd); HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc); try (Connection conn = ConnectionFactory.createConnection(conf1)) { @@ -105,7 +107,6 @@ public class TestIncrementalBackup extends TestBackupBase { Assert.assertEquals(HBaseTestingUtility.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_FAM3); LOG.debug("written " + ADD_ROWS + " rows to " + table1); - // additionally, insert rows to MOB cf int NB_ROWS_MOB = 111; insertIntoTable(conn, table1, mobName, 3, NB_ROWS_MOB); @@ -113,7 +114,6 @@ public class TestIncrementalBackup extends TestBackupBase { t1.close(); Assert.assertEquals(HBaseTestingUtility.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_MOB); - Table t2 = conn.getTable(table2); Put p2; for (int i = 0; i < 5; i++) { @@ -124,13 +124,11 @@ public class TestIncrementalBackup extends TestBackupBase { Assert.assertEquals(NB_ROWS_IN_BATCH + 5, HBaseTestingUtility.countRows(t2)); t2.close(); LOG.debug("written " + 5 + " rows to " + table2); - // split table1 MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); List regions = cluster.getRegions(table1); byte[] name = regions.get(0).getRegionInfo().getRegionName(); long startSplitTime = EnvironmentEdgeManager.currentTime(); - try { admin.splitRegionAsync(name).get(); } catch (Exception e) { @@ -141,7 +139,6 @@ public class TestIncrementalBackup extends TestBackupBase { while (!admin.isTableAvailable(table1)) { Thread.sleep(100); } - long endSplitTime = EnvironmentEdgeManager.currentTime(); // split finished LOG.debug("split finished in =" + (endSplitTime - startSplitTime)); @@ -154,10 +151,11 @@ public class TestIncrementalBackup extends TestBackupBase { // add column family f2 to table1 final byte[] fam2Name = Bytes.toBytes("f2"); - table1Desc.addFamily(new HColumnDescriptor(fam2Name)); + table1Desc.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2Name)); // drop column family f3 - table1Desc.removeFamily(fam3Name); + table1Desc.removeColumnFamily(fam3Name); HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc); int NB_ROWS_FAM2 = 7; @@ -222,4 +220,4 @@ public class TestIncrementalBackup extends TestBackupBase { admin.close(); } } -} \ No newline at end of file +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java index 546cf414ebb..aeb1edb55c7 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -25,7 +25,6 @@ import java.util.Collection; import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; @@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.backup.impl.TableBackupClient; import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -83,7 +83,8 @@ public class TestIncrementalBackupWithFailures extends TestBackupBase { List tables = Lists.newArrayList(table1, table2); final byte[] fam3Name = Bytes.toBytes("f3"); - table1Desc.addFamily(new HColumnDescriptor(fam3Name)); + table1Desc.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name)); HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc); Connection conn = ConnectionFactory.createConnection(conf1); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java index 2d99e0dd86e..832a78204e8 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -23,10 +23,10 @@ import java.io.IOException; import java.util.concurrent.CountDownLatch; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -93,12 +93,14 @@ public class TestRemoteBackup extends TestBackupBase { }); t.start(); - table1Desc.addFamily(new HColumnDescriptor(fam3Name)); + table1Desc.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name)); // family 2 is MOB enabled - HColumnDescriptor hcd = new HColumnDescriptor(fam2Name); - hcd.setMobEnabled(true); - hcd.setMobThreshold(0L); - table1Desc.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2Name); + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(0L); + table1Desc.setColumnFamily(familyDescriptor); HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc); SnapshotTestingUtils.loadData(TEST_UTIL, table1, 50, fam2Name); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java index 7af2b8f6aa6..58b5d0148a3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java @@ -113,7 +113,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable getConfiguration() { - return delegatee.getValues().entrySet().stream() - .collect(Collectors.toMap( - e -> Bytes.toString(e.getKey().get(), e.getKey().getOffset(), e.getKey().getLength()), - e -> Bytes.toString(e.getValue().get(), e.getValue().getOffset(), e.getValue().getLength()) - )); + return delegatee.getConfiguration(); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index 3b68007aa8f..630224bc71a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -35,6 +35,8 @@ import java.util.TreeSet; import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; + import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -671,6 +673,19 @@ public class TableDescriptorBuilder { return Collections.unmodifiableMap(values); } + /** + * Getter for fetching an unmodifiable map. + */ + public Map getConfiguration() { + return getValues().entrySet().stream() + .collect(Collectors.toMap( + e -> Bytes.toString(e.getKey().get(), e.getKey().getOffset(), + e.getKey().getLength()), + e -> Bytes.toString(e.getValue().get(), e.getValue().getOffset(), + e.getValue().getLength()) + )); + } + /** * Setter for storing metadata as a (key, value) pair in {@link #values} map * diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java index 0e7226832d1..8149ba04c00 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -55,11 +56,10 @@ public class TestHColumnDescriptor { @Test public void testPb() throws DeserializationException { HColumnDescriptor hcd = new HColumnDescriptor( - new HColumnDescriptor(HConstants.CATALOG_FAMILY) - .setInMemory(true) - .setScope(HConstants.REPLICATION_SCOPE_LOCAL) - .setBloomFilterType(BloomType.NONE) - .setCacheDataInL1(true)); + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(HConstants.CATALOG_FAMILY) + .setInMemory(true) + .setScope(HConstants.REPLICATION_SCOPE_LOCAL) + .setBloomFilterType(BloomType.NONE)); final int v = 123; hcd.setBlocksize(v); hcd.setTimeToLive(v); @@ -107,7 +107,7 @@ public class TestHColumnDescriptor { public void testHColumnDescriptorShouldThrowIAEWhenFamilyNameEmpty() throws Exception { expectedEx.expect(IllegalArgumentException.class); expectedEx.expectMessage("Column Family name can not be empty"); - new HColumnDescriptor(Bytes.toBytes("")); + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("")); } @Test diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java index 94d05f77283..4e05a82a37e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -25,7 +25,10 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.Arrays; import java.util.regex.Pattern; + +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -309,41 +312,52 @@ public class TestHTableDescriptor { @Test public void testModifyFamily() { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); byte[] familyName = Bytes.toBytes("cf"); - HColumnDescriptor hcd = new HColumnDescriptor(familyName); - hcd.setBlocksize(1000); - hcd.setDFSReplication((short) 3); - htd.addFamily(hcd); - assertEquals(1000, htd.getFamily(familyName).getBlocksize()); - assertEquals(3, htd.getFamily(familyName).getDFSReplication()); - hcd = new HColumnDescriptor(familyName); - hcd.setBlocksize(2000); - hcd.setDFSReplication((short) 1); - htd.modifyFamily(hcd); - assertEquals(2000, htd.getFamily(familyName).getBlocksize()); - assertEquals(1, htd.getFamily(familyName).getDFSReplication()); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName) + .setBlocksize(1000) + .setDFSReplication((short) 3); + tableDescriptor.setColumnFamily(familyDescriptor); + assertEquals(1000, tableDescriptor.getColumnFamily(familyName).getBlocksize()); + assertEquals(3, tableDescriptor.getColumnFamily(familyName).getDFSReplication()); + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName) + .setBlocksize(2000) + .setDFSReplication((short) 1); + tableDescriptor.modifyColumnFamily(familyDescriptor); + assertEquals(2000, tableDescriptor.getColumnFamily(familyName).getBlocksize()); + assertEquals(1, tableDescriptor.getColumnFamily(familyName).getDFSReplication()); } @Test(expected=IllegalArgumentException.class) public void testModifyInexistentFamily() { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); byte[] familyName = Bytes.toBytes("cf"); - HColumnDescriptor hcd = new HColumnDescriptor(familyName); - htd.modifyFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName); + tableDescriptor.modifyColumnFamily(familyDescriptor); } @Test(expected=IllegalArgumentException.class) public void testAddDuplicateFamilies() { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); byte[] familyName = Bytes.toBytes("cf"); - HColumnDescriptor hcd = new HColumnDescriptor(familyName); - hcd.setBlocksize(1000); - htd.addFamily(hcd); - assertEquals(1000, htd.getFamily(familyName).getBlocksize()); - hcd = new HColumnDescriptor(familyName); - hcd.setBlocksize(2000); - htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName) + .setBlocksize(1000); + tableDescriptor.setColumnFamily(familyDescriptor); + assertEquals(1000, tableDescriptor.getColumnFamily(familyName).getBlocksize()); + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName) + .setBlocksize(2000); + tableDescriptor.setColumnFamily(familyDescriptor); } @Test diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java index 19f872193b8..3410e134987 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -25,7 +25,6 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.regex.Pattern; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -279,11 +278,12 @@ public class TestTableDescriptorBuilder { @Test(expected=IllegalArgumentException.class) public void testModifyInexistentFamily() { byte[] familyName = Bytes.toBytes("cf"); - HColumnDescriptor hcd = new HColumnDescriptor(familyName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName); TableDescriptor htd = TableDescriptorBuilder - .newBuilder(TableName.valueOf(name.getMethodName())) - .modifyColumnFamily(hcd) - .build(); + .newBuilder(TableName.valueOf(name.getMethodName())) + .modifyColumnFamily(familyDescriptor) + .build(); } @Test(expected=IllegalArgumentException.class) diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java index c108db28a2a..2407dcabaf0 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java @@ -28,12 +28,12 @@ import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse; @@ -92,9 +92,12 @@ public class TestBatchCoprocessorEndpoint { ProtobufCoprocessorService.class.getName()); util.startMiniCluster(2); Admin admin = util.getAdmin(); - HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); - desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); - admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); + + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); + admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); util.waitUntilAllRegionsAssigned(TEST_TABLE); admin.close(); diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java index 3edc26b1507..7cc6c6af50a 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java @@ -33,16 +33,16 @@ import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; @@ -98,9 +98,12 @@ public class TestCoprocessorEndpoint { ProtobufCoprocessorService.class.getName()); util.startMiniCluster(2); Admin admin = util.getAdmin(); - HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); - desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); - admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); + + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); + admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); util.waitUntilAllRegionsAssigned(TEST_TABLE); Table table = util.getConnection().getTable(TEST_TABLE); diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java index b25c17d0949..7464055bdc6 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java @@ -25,12 +25,13 @@ import java.io.IOException; import java.util.Map; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos; import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; @@ -78,11 +79,15 @@ public class TestCoprocessorTableEndpoint { public void testCoprocessorTableEndpoint() throws Throwable { final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); - desc.addCoprocessor(ColumnAggregationEndpoint.class.getName()); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); - createTable(desc); + tableDescriptor.setColumnFamily(familyDescriptor); + tableDescriptor.setCoprocessor(ColumnAggregationEndpoint.class.getName()); + + createTable(tableDescriptor); verifyTable(tableName); } @@ -90,13 +95,16 @@ public class TestCoprocessorTableEndpoint { public void testDynamicCoprocessorTableEndpoint() throws Throwable { final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); - createTable(desc); + tableDescriptor.setColumnFamily(familyDescriptor); + createTable(tableDescriptor); - desc.addCoprocessor(ColumnAggregationEndpoint.class.getName()); - updateTable(desc); + tableDescriptor.setCoprocessor(ColumnAggregationEndpoint.class.getName()); + updateTable(tableDescriptor); verifyTable(tableName); } @@ -132,11 +140,12 @@ public class TestCoprocessorTableEndpoint { }); } - private static final void createTable(HTableDescriptor desc) throws Exception { + private static final void createTable( + final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) throws Exception { Admin admin = TEST_UTIL.getAdmin(); - admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); - TEST_UTIL.waitUntilAllRegionsAssigned(desc.getTableName()); - Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); + admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); + TEST_UTIL.waitUntilAllRegionsAssigned(tableDescriptor.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(tableDescriptor.getTableName()); try { for (int i = 0; i < ROWSIZE; i++) { Put put = new Put(ROWS[i]); @@ -148,11 +157,12 @@ public class TestCoprocessorTableEndpoint { } } - private static void updateTable(HTableDescriptor desc) throws Exception { + private static void updateTable( + final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) throws Exception { Admin admin = TEST_UTIL.getAdmin(); - admin.disableTable(desc.getTableName()); - admin.modifyTable(desc); - admin.enableTable(desc.getTableName()); + admin.disableTable(tableDescriptor.getTableName()); + admin.modifyTable(tableDescriptor); + admin.enableTable(tableDescriptor.getTableName()); } private static final void verifyTable(TableName tableName) throws Throwable { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java index fc79abb4410..feca9d0b153 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java @@ -21,6 +21,8 @@ package org.apache.hadoop.hbase; import java.io.IOException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StripeStoreEngine; @@ -38,11 +40,16 @@ public class IntegrationTestIngestStripeCompactions extends IntegrationTestInges @Override protected void initTable() throws IOException { // Do the same as the LoadTestTool does, but with different table configuration. - HTableDescriptor htd = new HTableDescriptor(getTablename()); - htd.setConfiguration(StoreEngine.STORE_ENGINE_CLASS_KEY, StripeStoreEngine.class.getName()); - htd.setConfiguration(HStore.BLOCKING_STOREFILES_KEY, "100"); - HColumnDescriptor hcd = new HColumnDescriptor(HFileTestUtil.DEFAULT_COLUMN_FAMILY); - HBaseTestingUtility.createPreSplitLoadTestTable(util.getConfiguration(), htd, hcd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(getTablename()); + tableDescriptor.setValue(StoreEngine.STORE_ENGINE_CLASS_KEY, + StripeStoreEngine.class.getName()); + tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "100"); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HFileTestUtil.DEFAULT_COLUMN_FAMILY); + HBaseTestingUtility.createPreSplitLoadTestTable(util.getConfiguration(), + tableDescriptor, familyDescriptor); } public static void main(String[] args) throws Exception { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java index 128767b4b29..8deccd49905 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java @@ -24,12 +24,14 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; @@ -187,16 +189,18 @@ public class IntegrationTestLazyCfLoading { Configuration conf = util.getConfiguration(); String encodingKey = String.format(ENCODING_KEY, this.getClass().getSimpleName()); DataBlockEncoding blockEncoding = DataBlockEncoding.valueOf(conf.get(encodingKey, "FAST_DIFF")); - HTableDescriptor htd = new HTableDescriptor(TABLE_NAME); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); for (byte[] cf : dataGen.getColumnFamilies()) { - HColumnDescriptor hcd = new HColumnDescriptor(cf); - hcd.setDataBlockEncoding(blockEncoding); - htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cf); + familyDescriptor.setDataBlockEncoding(blockEncoding); + tableDescriptor.setColumnFamily(familyDescriptor); } int serverCount = util.getHBaseClusterInterface().getClusterMetrics() .getLiveServerMetrics().size(); byte[][] splits = new RegionSplitter.HexStringSplit().split(serverCount * REGIONS_PER_SERVER); - util.getAdmin().createTable(htd, splits); + util.getAdmin().createTable(tableDescriptor, splits); LOG.info("Created table"); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java index 5fb9bb0a4c4..5e8cd800053 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java @@ -30,10 +30,12 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; import org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor; import org.apache.hadoop.hbase.mob.MobConstants; @@ -96,8 +98,8 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase { .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); private static Configuration conf; - private static HTableDescriptor hdt; - private static HColumnDescriptor hcd; + private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; + private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor; private static Admin admin; private static Table table = null; private static MobFileCleanerChore chore; @@ -124,13 +126,13 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase { private void createTestTable() throws IOException { // Create test table - hdt = util.createTableDescriptor(TableName.valueOf("testMobCompactTable")); - hcd = new HColumnDescriptor(fam); - hcd.setMobEnabled(true); - hcd.setMobThreshold(mobLen); - hcd.setMaxVersions(1); - hdt.addFamily(hcd); - table = util.createTable(hdt, null); + tableDescriptor = util.createModifyableTableDescriptor("testMobCompactTable"); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(mobLen); + familyDescriptor.setMaxVersions(1); + tableDescriptor.setColumnFamily(familyDescriptor); + table = util.createTable(tableDescriptor, null); } @After @@ -247,7 +249,7 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase { public void run() { while (run) { try { - admin.majorCompact(hdt.getTableName(), fam); + admin.majorCompact(tableDescriptor.getTableName(), fam); Thread.sleep(120000); } catch (Exception e) { LOG.error("MOB Stress Test FAILED", e); @@ -355,8 +357,8 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase { } finally { - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); + admin.disableTable(tableDescriptor.getTableName()); + admin.deleteTable(tableDescriptor.getTableName()); } LOG.info("MOB Stress Test finished OK"); printStats(rowsToLoad); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java index 86ccfc602e1..3e94a19cd55 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java @@ -23,6 +23,9 @@ import java.util.Locale; import java.util.Set; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreEngine; @@ -264,42 +267,48 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool { System.out.println(s); } - private HTableDescriptor createHtd(boolean isStripe) throws Exception { - HTableDescriptor htd = new HTableDescriptor(TABLE_NAME); - htd.addFamily(new HColumnDescriptor(COLUMN_FAMILY)); + private TableDescriptorBuilder.ModifyableTableDescriptor createHtd(boolean isStripe) + throws Exception { + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_FAMILY); + tableDescriptor.setColumnFamily(familyDescriptor); String noSplitsPolicy = DisabledRegionSplitPolicy.class.getName(); - htd.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, noSplitsPolicy); + tableDescriptor.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, noSplitsPolicy); if (isStripe) { - htd.setConfiguration(StoreEngine.STORE_ENGINE_CLASS_KEY, StripeStoreEngine.class.getName()); + tableDescriptor.setValue(StoreEngine.STORE_ENGINE_CLASS_KEY, + StripeStoreEngine.class.getName()); if (initialStripeCount != null) { - htd.setConfiguration( + tableDescriptor.setValue( StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialStripeCount.toString()); - htd.setConfiguration( + tableDescriptor.setValue( HStore.BLOCKING_STOREFILES_KEY, Long.toString(10 * initialStripeCount)); } else { - htd.setConfiguration(HStore.BLOCKING_STOREFILES_KEY, "500"); + tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "500"); } if (splitSize != null) { - htd.setConfiguration(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize.toString()); + tableDescriptor.setValue(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize.toString()); } if (splitParts != null) { - htd.setConfiguration(StripeStoreConfig.SPLIT_PARTS_KEY, splitParts.toString()); + tableDescriptor.setValue(StripeStoreConfig.SPLIT_PARTS_KEY, splitParts.toString()); } } else { - htd.setConfiguration(HStore.BLOCKING_STOREFILES_KEY, "10"); // default + tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "10"); // default } - return htd; + return tableDescriptor; } - protected void createTable(HTableDescriptor htd) throws Exception { + protected void createTable(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) + throws Exception { deleteTable(); if (util.getHBaseClusterInterface() instanceof MiniHBaseCluster) { LOG.warn("Test does not make a lot of sense for minicluster. Will set flush size low."); - htd.setConfiguration(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, "1048576"); + tableDescriptor.setValue(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, "1048576"); } byte[][] splits = new RegionSplitter.HexStringSplit().split( util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size()); - util.getAdmin().createTable(htd, splits); + util.getAdmin().createTable(tableDescriptor, splits); } public static class SeqShardedDataGenerator extends LoadTestDataGenerator { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index 95cc587edf5..784e5b423e0 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -43,10 +43,8 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestBase; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.MasterNotRunningException; @@ -54,6 +52,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutatorParams; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionConfiguration; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -65,6 +64,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; @@ -756,18 +756,24 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) { if (!admin.tableExists(tableName)) { - HTableDescriptor htd = new HTableDescriptor(getTableName(getConf())); - htd.addFamily(new HColumnDescriptor(FAMILY_NAME)); - // Always add these families. Just skip writing to them when we do not test per CF flush. - htd.addFamily(new HColumnDescriptor(BIG_FAMILY_NAME)); - htd.addFamily(new HColumnDescriptor(TINY_FAMILY_NAME)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(getTableName(getConf())); + + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME); // if -DuseMob=true force all data through mob path. - if (conf.getBoolean("useMob", false)) { - for (HColumnDescriptor hcd : htd.getColumnFamilies() ) { - hcd.setMobEnabled(true); - hcd.setMobThreshold(4); - } - } + setMobProperties(conf, familyDescriptor); + tableDescriptor.setColumnFamily(familyDescriptor); + // Always add these families. Just skip writing to them when we do not test per CF flush. + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BIG_FAMILY_NAME); + setMobProperties(conf, familyDescriptor); + tableDescriptor.setColumnFamily(familyDescriptor); + + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TINY_FAMILY_NAME); + setMobProperties(conf, familyDescriptor); + tableDescriptor.setColumnFamily(familyDescriptor); // If we want to pre-split compute how many splits. if (conf.getBoolean(HBaseTestingUtility.PRESPLIT_TEST_TABLE_KEY, @@ -786,12 +792,12 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { byte[][] splits = new RegionSplitter.UniformSplit().split(totalNumberOfRegions); - admin.createTable(htd, splits); + admin.createTable(tableDescriptor, splits); } else { // Looks like we're just letting things play out. // Create a table with on region by default. // This will make the splitting work hard. - admin.createTable(htd); + admin.createTable(tableDescriptor); } } } catch (MasterNotRunningException e) { @@ -908,6 +914,14 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { } } + private static void setMobProperties(final Configuration conf, + final ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor) { + if (conf.getBoolean("useMob", false)) { + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(4); + } + } + /** * Tool to search missing rows in WALs and hfiles. * Pass in file or dir of keys to search for. Key file must have been written by Verify step diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java index 6b397769cca..8550f4e4141 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java @@ -28,21 +28,21 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutatorParams; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.mapreduce.Import; @@ -147,13 +147,15 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB private void createTable(Admin admin, TableName tableName, boolean setVersion, boolean acl) throws IOException { if (!admin.tableExists(tableName)) { - HTableDescriptor htd = new HTableDescriptor(tableName); - HColumnDescriptor family = new HColumnDescriptor(FAMILY_NAME); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME); if (setVersion) { - family.setMaxVersions(DEFAULT_TABLES_COUNT); + familyDescriptor.setMaxVersions(DEFAULT_TABLES_COUNT); } - htd.addFamily(family); - admin.createTable(htd); + tableDescriptor.setColumnFamily(familyDescriptor); + admin.createTable(tableDescriptor); if (acl) { LOG.info("Granting permissions for user " + USER.getShortName()); Permission.Action[] actions = { Permission.Action.READ }; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java index aedc676edfe..6fc4ba72008 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java @@ -41,21 +41,22 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestBase; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutatorParams; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.NMapInputFormat; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; @@ -329,15 +330,15 @@ public void cleanUpCluster() throws Exception { } } - protected Job doLoad(Configuration conf, HTableDescriptor htd) throws Exception { + protected Job doLoad(Configuration conf, TableDescriptor tableDescriptor) throws Exception { Path outputDir = getTestDir(TEST_NAME, "load-output"); LOG.info("Load output dir: " + outputDir); NMapInputFormat.setNumMapTasks(conf, conf.getInt(NUM_MAP_TASKS_KEY, NUM_MAP_TASKS_DEFAULT)); - conf.set(TABLE_NAME_KEY, htd.getTableName().getNameAsString()); + conf.set(TABLE_NAME_KEY, tableDescriptor.getTableName().getNameAsString()); Job job = Job.getInstance(conf); - job.setJobName(TEST_NAME + " Load for " + htd.getTableName()); + job.setJobName(TEST_NAME + " Load for " + tableDescriptor.getTableName()); job.setJarByClass(this.getClass()); setMapperClass(job); job.setInputFormatClass(NMapInputFormat.class); @@ -357,19 +358,19 @@ public void cleanUpCluster() throws Exception { job.setMapperClass(LoadMapper.class); } - protected void doVerify(Configuration conf, HTableDescriptor htd) throws Exception { + protected void doVerify(Configuration conf, TableDescriptor tableDescriptor) throws Exception { Path outputDir = getTestDir(TEST_NAME, "verify-output"); LOG.info("Verify output dir: " + outputDir); Job job = Job.getInstance(conf); job.setJarByClass(this.getClass()); - job.setJobName(TEST_NAME + " Verification for " + htd.getTableName()); + job.setJobName(TEST_NAME + " Verification for " + tableDescriptor.getTableName()); setJobScannerConf(job); Scan scan = new Scan(); TableMapReduceUtil.initTableMapperJob( - htd.getTableName().getNameAsString(), scan, VerifyMapper.class, + tableDescriptor.getTableName().getNameAsString(), scan, VerifyMapper.class, BytesWritable.class, BytesWritable.class, job); TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), AbstractHBaseTool.class); int scannerCaching = conf.getInt("verify.scannercaching", SCANNER_CACHING); @@ -532,18 +533,21 @@ public void cleanUpCluster() throws Exception { @Test public void testLoadAndVerify() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TEST_NAME)); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_NAME)); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); Admin admin = getTestingUtil(getConf()).getAdmin(); - admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), 40); + admin.createTable(tableDescriptor, Bytes.toBytes(0L), Bytes.toBytes(-1L), 40); - doLoad(getConf(), htd); - doVerify(getConf(), htd); + doLoad(getConf(), tableDescriptor); + doVerify(getConf(), tableDescriptor); // Only disable and drop if we succeeded to verify - otherwise it's useful // to leave it around for post-mortem - getTestingUtil(getConf()).deleteTable(htd.getTableName()); + getTestingUtil(getConf()).deleteTable(tableDescriptor.getTableName()); } @Override @@ -613,20 +617,22 @@ public void cleanUpCluster() throws Exception { // create HTableDescriptor for specified table TableName table = getTablename(); - HTableDescriptor htd = new HTableDescriptor(table); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(table); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); if (doLoad) { try (Connection conn = ConnectionFactory.createConnection(getConf()); Admin admin = conn.getAdmin()) { - admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits); - doLoad(getConf(), htd); + admin.createTable(tableDescriptor, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits); + doLoad(getConf(), tableDescriptor); } } if (doVerify) { - doVerify(getConf(), htd); + doVerify(getConf(), tableDescriptor); if (doDelete) { - getTestingUtil(getConf()).deleteTable(htd.getTableName()); + getTestingUtil(getConf()).deleteTable(tableDescriptor.getTableName()); } } if (doSearch) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java index b15cee6aa65..6c71057ddf6 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java @@ -26,16 +26,17 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapper; @@ -237,7 +238,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT } @Override - protected Job doLoad(Configuration conf, HTableDescriptor htd) throws Exception { + protected Job doLoad(Configuration conf, TableDescriptor htd) throws Exception { Job job = super.doLoad(conf, htd); this.numRowsLoadedWithExp1 = job.getCounters().findCounter(Counters.ROWS_VIS_EXP_1).getValue(); this.numRowsLoadedWithExp2 = job.getCounters().findCounter(Counters.ROWS_VIS_EXP_2).getValue(); @@ -260,13 +261,14 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT } @Override - protected void doVerify(final Configuration conf, final HTableDescriptor htd) throws Exception { + protected void doVerify(final Configuration conf, final TableDescriptor tableDescriptor) + throws Exception { System.out.println(String.format("Verifying for auths %s, %s, %s, %s", CONFIDENTIAL, TOPSECRET, SECRET, PRIVATE)); PrivilegedExceptionAction scanAction = new PrivilegedExceptionAction() { @Override public Job run() throws Exception { - return doVerify(conf, htd, CONFIDENTIAL, TOPSECRET, SECRET, PRIVATE); + return doVerify(conf, tableDescriptor, CONFIDENTIAL, TOPSECRET, SECRET, PRIVATE); } }; Job job = USER1.runAs(scanAction); @@ -284,7 +286,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT scanAction = new PrivilegedExceptionAction() { @Override public Job run() throws Exception { - return doVerify(conf, htd, PRIVATE, PUBLIC); + return doVerify(conf, tableDescriptor, PRIVATE, PUBLIC); } }; job = USER1.runAs(scanAction); @@ -302,7 +304,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT scanAction = new PrivilegedExceptionAction() { @Override public Job run() throws Exception { - return doVerify(conf, htd, PRIVATE, PUBLIC); + return doVerify(conf, tableDescriptor, PRIVATE, PUBLIC); } }; job = USER2.runAs(scanAction); @@ -316,16 +318,16 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT assertEquals(0, this.numRowsReadWithExp4); } - private Job doVerify(Configuration conf, HTableDescriptor htd, String... auths) + private Job doVerify(Configuration conf, TableDescriptor tableDescriptor, String... auths) throws IOException, InterruptedException, ClassNotFoundException { Path outputDir = getTestDir(TEST_NAME, "verify-output"); Job job = new Job(conf); job.setJarByClass(this.getClass()); - job.setJobName(TEST_NAME + " Verification for " + htd.getTableName()); + job.setJobName(TEST_NAME + " Verification for " + tableDescriptor.getTableName()); setJobScannerConf(job); Scan scan = new Scan(); scan.setAuthorizations(new Authorizations(auths)); - TableMapReduceUtil.initTableMapperJob(htd.getTableName().getNameAsString(), scan, + TableMapReduceUtil.initTableMapperJob(tableDescriptor.getTableName().getNameAsString(), scan, VerifyMapper.class, NullWritable.class, NullWritable.class, job); TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), AbstractHBaseTool.class); int scannerCaching = conf.getInt("verify.scannercaching", SCANNER_CACHING); @@ -363,15 +365,17 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT IntegrationTestingUtility.setUseDistributedCluster(getConf()); int numPresplits = getConf().getInt("loadmapper.numPresplits", 5); // create HTableDescriptor for specified table - HTableDescriptor htd = new HTableDescriptor(getTablename()); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(getTablename()); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); try (Connection conn = ConnectionFactory.createConnection(getConf()); Admin admin = conn.getAdmin()) { - admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits); + admin.createTable(tableDescriptor, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits); } - doLoad(getConf(), htd); - doVerify(getConf(), htd); + doLoad(getConf(), tableDescriptor); + doVerify(getConf(), tableDescriptor); getTestingUtil(getConf()).deleteTable(getTablename()); return 0; } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java index 0fd9483cce6..5cb8e850948 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java @@ -32,12 +32,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -596,15 +596,17 @@ public class ImportTsv extends Configured implements Tool { private static void createTable(Admin admin, TableName tableName, String[] columns) throws IOException { - HTableDescriptor htd = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); Set cfSet = getColumnFamilies(columns); for (String cf : cfSet) { - HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(cf)); - htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes(cf)); + tableDescriptor.setColumnFamily(familyDescriptor); } LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.", tableName, cfSet)); - admin.createTable(htd); + admin.createTable(tableDescriptor); } private static void deleteTable(Configuration conf, String[] args) { diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java index 9da0cadb40f..cb6dc1b354c 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutatorParams; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Consistency; @@ -66,6 +67,8 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.Filter; @@ -339,8 +342,8 @@ public class PerformanceEvaluation extends Configured implements Tool { throw new IllegalStateException( "Must specify an existing table for read commands. Run a write command first."); } - HTableDescriptor desc = - exists ? new HTableDescriptor(admin.getDescriptor(TableName.valueOf(opts.tableName))) : null; + TableDescriptor desc = + exists ? admin.getDescriptor(TableName.valueOf(opts.tableName)) : null; byte[][] splits = getSplits(opts); // recreate the table when user has requested presplit or when existing @@ -396,11 +399,14 @@ public class PerformanceEvaluation extends Configured implements Tool { /** * Create an HTableDescriptor from provided TestOptions. */ - protected static HTableDescriptor getTableDescriptor(TestOptions opts) { - HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(opts.tableName)); + protected static TableDescriptor getTableDescriptor(TestOptions opts) { + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(opts.tableName)); + for (int family = 0; family < opts.families; family++) { byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family); - HColumnDescriptor familyDesc = new HColumnDescriptor(familyName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDesc = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName); familyDesc.setDataBlockEncoding(opts.blockEncoding); familyDesc.setCompressionType(opts.compression); familyDesc.setBloomFilterType(opts.bloomType); @@ -409,15 +415,15 @@ public class PerformanceEvaluation extends Configured implements Tool { familyDesc.setInMemory(true); } familyDesc.setInMemoryCompaction(opts.inMemoryCompaction); - tableDesc.addFamily(familyDesc); + tableDescriptor.setColumnFamily(familyDesc); } if (opts.replicas != DEFAULT_OPTS.replicas) { - tableDesc.setRegionReplication(opts.replicas); + tableDescriptor.setRegionReplication(opts.replicas); } if (opts.splitPolicy != null && !opts.splitPolicy.equals(DEFAULT_OPTS.splitPolicy)) { - tableDesc.setRegionSplitPolicyClassName(opts.splitPolicy); + tableDescriptor.setRegionSplitPolicyClassName(opts.splitPolicy); } - return tableDesc; + return tableDescriptor; } /** diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java index 8e3373a40f8..140bd4bf2bc 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java @@ -42,6 +42,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.PerformanceEvaluation.RandomReadTest; import org.apache.hadoop.hbase.PerformanceEvaluation.TestOptions; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.regionserver.CompactingMemStore; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -66,10 +68,10 @@ public class TestPerformanceEvaluation { new PerformanceEvaluation.TestOptions(); assertEquals(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_DEFAULT, defaultOpts.getInMemoryCompaction().toString()); - HTableDescriptor htd = PerformanceEvaluation.getTableDescriptor(defaultOpts); - for (HColumnDescriptor hcd: htd.getFamilies()) { + TableDescriptor tableDescriptor = PerformanceEvaluation.getTableDescriptor(defaultOpts); + for (ColumnFamilyDescriptor familyDescriptor : tableDescriptor.getColumnFamilies()) { assertEquals(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_DEFAULT, - hcd.getInMemoryCompaction().toString()); + familyDescriptor.getInMemoryCompaction().toString()); } } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java index 6c4d0b6f06b..2a32f0c3682 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java @@ -31,10 +31,9 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; @@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; @@ -150,12 +150,14 @@ public class TestTimeRangeMapRed { @Test public void testTimeRangeMapRed() - throws IOException, InterruptedException, ClassNotFoundException { - final HTableDescriptor desc = new HTableDescriptor(TABLE_NAME); - final HColumnDescriptor col = new HColumnDescriptor(FAMILY_NAME); - col.setMaxVersions(Integer.MAX_VALUE); - desc.addFamily(col); - admin.createTable(desc); + throws IOException, InterruptedException, ClassNotFoundException { + final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + final ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME); + familyDescriptor.setMaxVersions(Integer.MAX_VALUE); + tableDescriptor.setColumnFamily(familyDescriptor); + admin.createTable(tableDescriptor); List puts = new ArrayList<>(); for (Map.Entry entry : TIMESTAMP.entrySet()) { Put put = new Put(KEY); @@ -163,7 +165,7 @@ public class TestTimeRangeMapRed { put.addColumn(FAMILY_NAME, COLUMN_NAME, entry.getKey(), Bytes.toBytes(false)); puts.add(put); } - Table table = UTIL.getConnection().getTable(desc.getTableName()); + Table table = UTIL.getConnection().getTable(tableDescriptor.getTableName()); table.put(puts); runTestOnTable(); verify(table); diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java index 152fe4c2c51..296d9755113 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java @@ -30,10 +30,10 @@ import javax.xml.stream.XMLInputFactory; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamReader; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.rest.Constants; import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel; import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel; @@ -277,7 +277,7 @@ public class RemoteAdmin { * @param desc table descriptor for table * @throws IOException if a remote or network exception occurs */ - public void createTable(HTableDescriptor desc) + public void createTable(TableDescriptor desc) throws IOException { TableSchemaModel model = new TableSchemaModel(desc); StringBuilder path = new StringBuilder(); diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java index 98b0ec8a72d..6e99d826e4e 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java @@ -36,6 +36,7 @@ import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlRootElement; import javax.xml.namespace.QName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -87,15 +88,15 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler { /** * Constructor - * @param htd the table descriptor + * @param tableDescriptor the table descriptor */ - public TableSchemaModel(HTableDescriptor htd) { - setName(htd.getTableName().getNameAsString()); - for (Map.Entry e : htd.getValues().entrySet()) { + public TableSchemaModel(TableDescriptor tableDescriptor) { + setName(tableDescriptor.getTableName().getNameAsString()); + for (Map.Entry e : tableDescriptor.getValues().entrySet()) { addAttribute(Bytes.toString(e.getKey().get()), Bytes.toString(e.getValue().get())); } - for (HColumnDescriptor hcd : htd.getFamilies()) { + for (ColumnFamilyDescriptor hcd : tableDescriptor.getColumnFamilies()) { ColumnSchemaModel columnModel = new ColumnSchemaModel(); columnModel.setName(hcd.getNameAsString()); for (Map.Entry e: diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java index a6f36fa0162..6db38e64617 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java @@ -43,13 +43,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; @@ -59,6 +58,8 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.PageFilter; @@ -127,7 +128,7 @@ public class PerformanceEvaluation extends Configured implements Tool { public static final byte[] QUALIFIER_NAME = Bytes.toBytes("data"); private TableName tableName = TABLE_NAME; - protected HTableDescriptor TABLE_DESCRIPTOR; + protected TableDescriptorBuilder.ModifyableTableDescriptor TABLE_DESCRIPTOR; protected Map commands = new TreeMap<>(); protected static Cluster cluster = new Cluster(); @@ -512,7 +513,7 @@ public class PerformanceEvaluation extends Configured implements Tool { * @throws IOException if an operation on the table fails */ private boolean checkTable(RemoteAdmin admin) throws IOException { - HTableDescriptor tableDescriptor = getDescriptor(); + TableDescriptor tableDescriptor = getDescriptor(); if (this.presplitRegions > 0) { // presplit requested if (admin.isTableAvailable(tableDescriptor.getTableName().getName())) { @@ -536,16 +537,17 @@ public class PerformanceEvaluation extends Configured implements Tool { return admin.isTableAvailable(tableDescriptor.getTableName().getName()); } - protected HTableDescriptor getDescriptor() { + protected TableDescriptor getDescriptor() { if (TABLE_DESCRIPTOR == null) { - TABLE_DESCRIPTOR = new HTableDescriptor(tableName); - HColumnDescriptor family = new HColumnDescriptor(FAMILY_NAME); - family.setDataBlockEncoding(blockEncoding); - family.setCompressionType(compression); + TABLE_DESCRIPTOR = new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME); + familyDescriptor.setDataBlockEncoding(blockEncoding); + familyDescriptor.setCompressionType(compression); if (inMemoryCF) { - family.setInMemory(true); + familyDescriptor.setInMemory(true); } - TABLE_DESCRIPTOR.addFamily(family); + TABLE_DESCRIPTOR.setColumnFamily(familyDescriptor); } return TABLE_DESCRIPTOR; } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java index 7d548886b98..acb34772012 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java @@ -34,17 +34,17 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; @@ -140,10 +140,13 @@ public class TestScannersWithFilters { REST_TEST_UTIL.getServletPort())); Admin admin = TEST_UTIL.getAdmin(); if (!admin.tableExists(TABLE)) { - HTableDescriptor htd = new HTableDescriptor(TABLE); - htd.addFamily(new HColumnDescriptor(FAMILIES[0])); - htd.addFamily(new HColumnDescriptor(FAMILIES[1])); - admin.createTable(htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0])); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1])); + admin.createTable(tableDescriptor); Table table = TEST_UTIL.getConnection().getTable(TABLE); // Insert first half for (byte[] ROW : ROWS_ONE) { diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java index a47697f140c..4080a1259d6 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java @@ -32,10 +32,9 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -44,6 +43,7 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.rest.HBaseRESTTestingUtility; import org.apache.hadoop.hbase.rest.RESTServlet; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -121,11 +121,19 @@ public class TestRemoteTable { admin.deleteTable(TABLE); } - HTableDescriptor htd = new HTableDescriptor(TABLE); - htd.addFamily(new HColumnDescriptor(COLUMN_1).setMaxVersions(3)); - htd.addFamily(new HColumnDescriptor(COLUMN_2).setMaxVersions(3)); - htd.addFamily(new HColumnDescriptor(COLUMN_3).setMaxVersions(3)); - admin.createTable(htd); + + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_1) + .setMaxVersions(3)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_2) + .setMaxVersions(3)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_3) + .setMaxVersions(3)); + admin.createTable(tableDescriptor); try (Table table = TEST_UTIL.getConnection().getTable(TABLE)) { Put put = new Put(ROW_1); put.addColumn(COLUMN_1, QUALIFIER_1, TS_2, VALUE_1); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java index 759d2520b0b..138113620e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.regex.Pattern; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,6 +95,18 @@ public final class Constraints { desc.addCoprocessor(clazz); } + public static void enable(TableDescriptorBuilder.ModifyableTableDescriptor desc) + throws IOException { + // if the CP has already been loaded, do nothing + String clazz = ConstraintProcessor.class.getName(); + if (desc.hasCoprocessor(clazz)) { + return; + } + + // add the constrain processor CP to the table + desc.setCoprocessor(clazz); + } + /** * Turn off processing constraints for a given table, even if constraints have * been turned on or added. @@ -106,6 +119,10 @@ public final class Constraints { desc.removeCoprocessor(ConstraintProcessor.class.getName()); } + public static void disable(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) { + tableDescriptor.removeCoprocessor(ConstraintProcessor.class.getName()); + } + /** * Remove all {@link Constraint Constraints} that have been added to the table * and turn off the constraint processing. @@ -170,6 +187,16 @@ public final class Constraints { return value == null ? null : new Pair<>(key, value); } + private static Pair getKeyValueForClass( + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, + Class clazz) { + // get the serialized version of the constraint + String key = serializeConstraintClass(clazz); + String value = tableDescriptor.getValue(key); + + return value == null ? null : new Pair<>(key, value); + } + /** * Add configuration-less constraints to the table. *

@@ -203,6 +230,38 @@ public final class Constraints { updateLatestPriority(desc, priority); } + /** + * Add configuration-less constraints to the table. + *

+ * This will overwrite any configuration associated with the previous + * constraint of the same class. + *

+ * Each constraint, when added to the table, will have a specific priority, + * dictating the order in which the {@link Constraint} will be run. A + * {@link Constraint} earlier in the list will be run before those later in + * the list. The same logic applies between two Constraints over time (earlier + * added is run first on the regionserver). + * + * @param tableDescriptor TableDescriptorBuilder.ModifyableTableDescriptor + * to add {@link Constraint} + * @param constraints {@link Constraint} to add. All constraints are + * considered automatically enabled on add + * @throws IOException If constraint could not be serialized/added to table + */ + public static void add(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, + Class... constraints) throws IOException { + // make sure constraints are enabled + enable(tableDescriptor); + long priority = getNextPriority(tableDescriptor); + + // store each constraint + for (Class clazz : constraints) { + writeConstraint(tableDescriptor, serializeConstraintClass(clazz), + configure(null, true, priority)); + } + updateLatestPriority(tableDescriptor, priority); + } + /** * Add constraints and their associated configurations to the table. *

@@ -337,6 +396,13 @@ public final class Constraints { desc.setValue(key, serializeConfiguration(conf)); } + private static void writeConstraint( + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, String key, + Configuration conf) throws IOException { + // store the key and conf in the descriptor + tableDescriptor.setValue(key, serializeConfiguration(conf)); + } + /** * Write the configuration to a String * @@ -385,7 +451,7 @@ public final class Constraints { return readConfiguration(Bytes.toBytes(bytes)); } - private static long getNextPriority(HTableDescriptor desc) { + private static long getNextPriority(TableDescriptor desc) { String value = desc.getValue(COUNTER_KEY); long priority; @@ -404,6 +470,12 @@ public final class Constraints { desc.setValue(COUNTER_KEY, Long.toString(priority)); } + private static void updateLatestPriority( + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, long priority) { + // update the max priority + tableDescriptor.setValue(COUNTER_KEY, Long.toString(priority)); + } + /** * Update the configuration for the {@link Constraint}; does not change the * order in which the constraint is run. @@ -493,6 +565,12 @@ public final class Constraints { changeConstraintEnabled(desc, clazz, false); } + public static void disableConstraint( + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, + Class clazz) throws IOException { + changeConstraintEnabled(tableDescriptor, clazz, false); + } + /** * Change the whether the constraint (if it is already present) is enabled or * disabled. @@ -516,6 +594,26 @@ public final class Constraints { writeConstraint(desc, entry.getFirst(), conf); } + private static void changeConstraintEnabled( + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, + Class clazz, boolean enabled) throws IOException { + // get the original constraint + Pair entry = getKeyValueForClass(tableDescriptor, clazz); + if (entry == null) { + throw new IllegalArgumentException("Constraint: " + clazz.getName() + + " is not associated with this table. You can't enable it!"); + } + + // create a new configuration from that conf + Configuration conf = readConfiguration(entry.getSecond()); + + // set that it is enabled + conf.setBoolean(ENABLED_KEY, enabled); + + // write it back out + writeConstraint(tableDescriptor, entry.getFirst(), conf); + } + /** * Check to see if the given constraint is enabled. * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java index 9053405b13e..e6acc4befcc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -27,13 +27,12 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; @@ -41,6 +40,7 @@ import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -75,17 +75,17 @@ public class QuotaUtil extends QuotaTableUtil { public static final long DEFAULT_WRITE_CAPACITY_UNIT = 1024; /** Table descriptor for Quota internal table */ - public static final HTableDescriptor QUOTA_TABLE_DESC = - new HTableDescriptor(QUOTA_TABLE_NAME); + public static final TableDescriptorBuilder.ModifyableTableDescriptor QUOTA_TABLE_DESC = + new TableDescriptorBuilder.ModifyableTableDescriptor(QUOTA_TABLE_NAME); static { - QUOTA_TABLE_DESC.addFamily( - new HColumnDescriptor(QUOTA_FAMILY_INFO) + QUOTA_TABLE_DESC.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(QUOTA_FAMILY_INFO) .setScope(HConstants.REPLICATION_SCOPE_LOCAL) .setBloomFilterType(BloomType.ROW) .setMaxVersions(1) ); - QUOTA_TABLE_DESC.addFamily( - new HColumnDescriptor(QUOTA_FAMILY_USAGE) + QUOTA_TABLE_DESC.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(QUOTA_FAMILY_USAGE) .setScope(HConstants.REPLICATION_SCOPE_LOCAL) .setBloomFilterType(BloomType.ROW) .setMaxVersions(1) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java index 2a18551ffcd..e5cd118bb08 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java @@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.PrivateCellUtil; @@ -55,6 +54,7 @@ import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Increment; @@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.coprocessor.CoprocessorException; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -210,18 +211,21 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso public void postStartMaster(ObserverContext ctx) throws IOException { // Need to create the new system table for labels here if (!MetaTableAccessor.tableExists(ctx.getEnvironment().getConnection(), LABELS_TABLE_NAME)) { - HTableDescriptor labelsTable = new HTableDescriptor(LABELS_TABLE_NAME); - HColumnDescriptor labelsColumn = new HColumnDescriptor(LABELS_TABLE_FAMILY); - labelsColumn.setBloomFilterType(BloomType.NONE); - labelsColumn.setBlockCacheEnabled(false); // We will cache all the labels. No need of normal - // table block cache. - labelsTable.addFamily(labelsColumn); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(LABELS_TABLE_NAME); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(LABELS_TABLE_FAMILY); + familyDescriptor.setBloomFilterType(BloomType.NONE); + // We will cache all the labels. No need of normal + // table block cache. + familyDescriptor.setBlockCacheEnabled(false); + tableDescriptor.setColumnFamily(familyDescriptor); // Let the "labels" table having only one region always. We are not expecting too many labels in // the system. - labelsTable.setValue(HTableDescriptor.SPLIT_POLICY, + tableDescriptor.setValue(HTableDescriptor.SPLIT_POLICY, DisabledRegionSplitPolicy.class.getName()); try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) { - admin.createTable(labelsTable); + admin.createTable(tableDescriptor); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java index d3aff18269e..eab64e4bed8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java @@ -25,11 +25,14 @@ import java.util.NavigableMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.Region; @@ -151,23 +154,24 @@ public abstract class HBaseTestCase extends TestCase { /** * You must call close on the returned region and then close on the log file it created. Do * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to close both the region and the WAL. - * @param desc - * @param startKey - * @param endKey + * @param tableDescriptor TableDescriptor + * @param startKey Start Key + * @param endKey End Key * @return An {@link HRegion} - * @throws IOException + * @throws IOException If thrown by + * {@link #createNewHRegion(TableDescriptor, byte[], byte[], Configuration)} */ - public HRegion createNewHRegion(HTableDescriptor desc, byte [] startKey, + public HRegion createNewHRegion(TableDescriptor tableDescriptor, byte [] startKey, byte [] endKey) throws IOException { - return createNewHRegion(desc, startKey, endKey, this.conf); + return createNewHRegion(tableDescriptor, startKey, endKey, this.conf); } - public HRegion createNewHRegion(HTableDescriptor desc, byte [] startKey, + public HRegion createNewHRegion(TableDescriptor tableDescriptor, byte [] startKey, byte [] endKey, Configuration conf) throws IOException { - HRegionInfo hri = new HRegionInfo(desc.getTableName(), startKey, endKey); - return HBaseTestingUtility.createRegionAndWAL(hri, testDir, conf, desc); + HRegionInfo hri = new HRegionInfo(tableDescriptor.getTableName(), startKey, endKey); + return HBaseTestingUtility.createRegionAndWAL(hri, testDir, conf, tableDescriptor); } protected HRegion openClosedRegion(final HRegion closedRegion) @@ -181,7 +185,7 @@ public abstract class HBaseTestCase extends TestCase { * @param name Name to give table. * @return Column descriptor. */ - protected HTableDescriptor createTableDescriptor(final String name) { + protected TableDescriptor createTableDescriptor(final String name) { return createTableDescriptor(name, MAXVERSIONS); } @@ -192,7 +196,7 @@ public abstract class HBaseTestCase extends TestCase { * @param versions How many versions to allow per column. * @return Column descriptor. */ - protected HTableDescriptor createTableDescriptor(final String name, + protected TableDescriptor createTableDescriptor(final String name, final int versions) { return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS, versions, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); @@ -205,11 +209,13 @@ public abstract class HBaseTestCase extends TestCase { * @param versions How many versions to allow per column. * @return Column descriptor. */ - protected HTableDescriptor createTableDescriptor(final String name, + protected TableDescriptor createTableDescriptor(final String name, final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name)); for (byte[] cfName : new byte[][]{ fam1, fam2, fam3 }) { - htd.addFamily(new HColumnDescriptor(cfName) + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName) .setMinVersions(minVersions) .setMaxVersions(versions) .setKeepDeletedCells(keepDeleted) @@ -217,7 +223,7 @@ public abstract class HBaseTestCase extends TestCase { .setTimeToLive(ttl) ); } - return htd; + return tableDescriptor; } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index c1b4db2a3b2..834fbac77c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -1683,18 +1683,21 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { */ public Table createTable(TableName tableName, byte[][] families, int numVersions, byte[][] splitKeys) throws IOException { - HTableDescriptor desc = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); for (byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family) + .setMaxVersions(numVersions); if (isNewVersionBehaviorEnabled()) { - hcd.setNewVersionBehavior(true); + familyDescriptor.setNewVersionBehavior(true); } - desc.addFamily(hcd); + tableDescriptor.setColumnFamily(familyDescriptor); } if (splitKeys != null) { - getAdmin().createTable(desc, splitKeys); + getAdmin().createTable(tableDescriptor, splitKeys); } else { - getAdmin().createTable(desc); + getAdmin().createTable(tableDescriptor); } // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are // assigned @@ -1726,17 +1729,19 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { */ public Table createTable(TableName tableName, byte[][] families, int numVersions, int blockSize) throws IOException { - HTableDescriptor desc = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); for (byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family) + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family) .setMaxVersions(numVersions) .setBlocksize(blockSize); if (isNewVersionBehaviorEnabled()) { - hcd.setNewVersionBehavior(true); + familyDescriptor.setNewVersionBehavior(true); } - desc.addFamily(hcd); + tableDescriptor.setColumnFamily(familyDescriptor); } - getAdmin().createTable(desc); + getAdmin().createTable(tableDescriptor); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are // assigned waitUntilAllRegionsAssigned(tableName); @@ -1745,25 +1750,27 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { public Table createTable(TableName tableName, byte[][] families, int numVersions, int blockSize, String cpName) throws IOException { - HTableDescriptor desc = new HTableDescriptor(tableName); - for (byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family) - .setMaxVersions(numVersions) - .setBlocksize(blockSize); - if (isNewVersionBehaviorEnabled()) { - hcd.setNewVersionBehavior(true); - } - desc.addFamily(hcd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + for (byte[] family : families) { + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family) + .setMaxVersions(numVersions) + .setBlocksize(blockSize); + if (isNewVersionBehaviorEnabled()) { + familyDescriptor.setNewVersionBehavior(true); } - if(cpName != null) { - desc.addCoprocessor(cpName); - } - getAdmin().createTable(desc); - // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are - // assigned - waitUntilAllRegionsAssigned(tableName); - return getConnection().getTable(tableName); + tableDescriptor.setColumnFamily(familyDescriptor); } + if (cpName != null) { + tableDescriptor.setCoprocessor(cpName); + } + getAdmin().createTable(tableDescriptor); + // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are + // assigned + waitUntilAllRegionsAssigned(tableName); + return getConnection().getTable(tableName); + } /** * Create a table. @@ -1774,20 +1781,21 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { * @throws IOException */ public Table createTable(TableName tableName, byte[][] families, - int[] numVersions) - throws IOException { - HTableDescriptor desc = new HTableDescriptor(tableName); + int[] numVersions) throws IOException { + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); int i = 0; for (byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family) + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family) .setMaxVersions(numVersions[i]); if (isNewVersionBehaviorEnabled()) { - hcd.setNewVersionBehavior(true); + familyDescriptor.setNewVersionBehavior(true); } - desc.addFamily(hcd); + tableDescriptor.setColumnFamily(familyDescriptor); i++; } - getAdmin().createTable(desc); + getAdmin().createTable(tableDescriptor); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are // assigned waitUntilAllRegionsAssigned(tableName); @@ -1804,13 +1812,15 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { */ public Table createTable(TableName tableName, byte[] family, byte[][] splitRows) throws IOException { - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor hcd = new HColumnDescriptor(family); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); if (isNewVersionBehaviorEnabled()) { - hcd.setNewVersionBehavior(true); + familyDescriptor.setNewVersionBehavior(true); } - desc.addFamily(hcd); - getAdmin().createTable(desc, splitRows); + tableDescriptor.setColumnFamily(familyDescriptor); + getAdmin().createTable(tableDescriptor, splitRows); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are // assigned waitUntilAllRegionsAssigned(tableName); @@ -1893,22 +1903,52 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { public static final byte [] START_KEY_BYTES = {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR}; public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET); + public TableDescriptorBuilder.ModifyableTableDescriptor createModifyableTableDescriptor( + final String name) { + return createModifyableTableDescriptor(TableName.valueOf(name), + HColumnDescriptor.DEFAULT_MIN_VERSIONS, + MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); + } + public HTableDescriptor createTableDescriptor(final TableName name, final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) { - HTableDescriptor htd = new HTableDescriptor(name); - for (byte[] cfName : new byte[][]{ fam1, fam2, fam3 }) { - HColumnDescriptor hcd = new HColumnDescriptor(cfName) + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(name); + for (byte[] cfName : new byte[][]{fam1, fam2, fam3}) { + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName) .setMinVersions(minVersions) .setMaxVersions(versions) .setKeepDeletedCells(keepDeleted) .setBlockCacheEnabled(false) .setTimeToLive(ttl); if (isNewVersionBehaviorEnabled()) { - hcd.setNewVersionBehavior(true); + familyDescriptor.setNewVersionBehavior(true); } - htd.addFamily(hcd); + tableDescriptor.setColumnFamily(familyDescriptor); } - return htd; + return new HTableDescriptor(tableDescriptor); + } + + public TableDescriptorBuilder.ModifyableTableDescriptor createModifyableTableDescriptor( + final TableName name, final int minVersions, final int versions, final int ttl, + KeepDeletedCells keepDeleted) { + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(name); + for (byte[] cfName : new byte[][]{fam1, fam2, fam3}) { + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName) + .setMinVersions(minVersions) + .setMaxVersions(versions) + .setKeepDeletedCells(keepDeleted) + .setBlockCacheEnabled(false) + .setTimeToLive(ttl); + if (isNewVersionBehaviorEnabled()) { + familyDescriptor.setNewVersionBehavior(true); + } + tableDescriptor.setColumnFamily(familyDescriptor); + } + return tableDescriptor; } /** @@ -1928,16 +1968,19 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { public HTableDescriptor createTableDescriptor(final TableName tableName, byte[][] families, int maxVersions) { - HTableDescriptor desc = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + for (byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family) + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family) .setMaxVersions(maxVersions); if (isNewVersionBehaviorEnabled()) { - hcd.setNewVersionBehavior(true); + familyDescriptor.setNewVersionBehavior(true); } - desc.addFamily(hcd); + tableDescriptor.setColumnFamily(familyDescriptor); } - return desc; + return new HTableDescriptor(tableDescriptor); } /** @@ -1997,25 +2040,27 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { boolean isReadOnly, Durability durability, WAL wal, boolean[] compactedMemStore, byte[]... families) throws IOException { - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.setReadOnly(isReadOnly); - int i=0; + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setReadOnly(isReadOnly); + int i = 0; for (byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family); - if(compactedMemStore != null && i < compactedMemStore.length) { - hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + if (compactedMemStore != null && i < compactedMemStore.length) { + familyDescriptor.setInMemoryCompaction(MemoryCompactionPolicy.BASIC); } else { - hcd.setInMemoryCompaction(MemoryCompactionPolicy.NONE); + familyDescriptor.setInMemoryCompaction(MemoryCompactionPolicy.NONE); } i++; // Set default to be three versions. - hcd.setMaxVersions(Integer.MAX_VALUE); - htd.addFamily(hcd); + familyDescriptor.setMaxVersions(Integer.MAX_VALUE); + tableDescriptor.setColumnFamily(familyDescriptor); } - htd.setDurability(durability); - HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false); - return createLocalHRegion(info, htd, wal); + tableDescriptor.setDurability(durability); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), startKey, stopKey, false); + return createLocalHRegion(info, tableDescriptor, wal); } // @@ -3878,13 +3923,16 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication, Durability durability) throws IOException { - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.setDurability(durability); - desc.setRegionReplication(regionReplication); - HColumnDescriptor hcd = new HColumnDescriptor(columnFamily); - hcd.setDataBlockEncoding(dataBlockEncoding); - hcd.setCompressionType(compression); - return createPreSplitLoadTestTable(conf, desc, hcd, numRegionsPerServer); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setDurability(durability); + tableDescriptor.setRegionReplication(regionReplication); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily); + familyDescriptor.setDataBlockEncoding(dataBlockEncoding); + familyDescriptor.setCompressionType(compression); + return createPreSplitLoadTestTable(conf, tableDescriptor, familyDescriptor, + numRegionsPerServer); } /** @@ -3897,17 +3945,19 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication, Durability durability) throws IOException { - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.setDurability(durability); - desc.setRegionReplication(regionReplication); - HColumnDescriptor[] hcds = new HColumnDescriptor[columnFamilies.length]; + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setDurability(durability); + tableDescriptor.setRegionReplication(regionReplication); + ColumnFamilyDescriptor[] hcds = new ColumnFamilyDescriptor[columnFamilies.length]; for (int i = 0; i < columnFamilies.length; i++) { - HColumnDescriptor hcd = new HColumnDescriptor(columnFamilies[i]); - hcd.setDataBlockEncoding(dataBlockEncoding); - hcd.setCompressionType(compression); - hcds[i] = hcd; + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamilies[i]); + familyDescriptor.setDataBlockEncoding(dataBlockEncoding); + familyDescriptor.setCompressionType(compression); + hcds[i] = familyDescriptor; } - return createPreSplitLoadTestTable(conf, desc, hcds, numRegionsPerServer); + return createPreSplitLoadTestTable(conf, tableDescriptor, hcds, numRegionsPerServer); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java index e7a9b450bd0..76cd6e4178b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java @@ -23,7 +23,9 @@ import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -80,10 +82,12 @@ public class TestHColumnDescriptorDefaultVersions { public void testCreateTableWithDefault() throws IOException { Admin admin = TEST_UTIL.getAdmin(); // Create a table with one family - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); - baseHtd.addFamily(hcd); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + tableDescriptor.setColumnFamily(familyDescriptor); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the column descriptor @@ -101,11 +105,13 @@ public class TestHColumnDescriptorDefaultVersions { Admin admin = TEST_UTIL.getAdmin(); // Create a table with one family - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); - hcd.setMaxVersions(TEST_UTIL.getConfiguration().getInt("hbase.column.max.version", 1)); - baseHtd.addFamily(hcd); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY) + .setMaxVersions(TEST_UTIL.getConfiguration().getInt("hbase.column.max.version", 1)); + tableDescriptor.setColumnFamily(familyDescriptor); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the column descriptor @@ -123,11 +129,13 @@ public class TestHColumnDescriptorDefaultVersions { Admin admin = TEST_UTIL.getAdmin(); // Create a table with one family - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); - hcd.setMaxVersions(5); - baseHtd.addFamily(hcd); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY) + .setMaxVersions(5); + tableDescriptor.setColumnFamily(familyDescriptor); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the column descriptor @@ -140,15 +148,16 @@ public class TestHColumnDescriptorDefaultVersions { @Test public void testHColumnDescriptorCachedMaxVersions() throws Exception { - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); - hcd.setMaxVersions(5); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + familyDescriptor.setMaxVersions(5); // Verify the max version - assertEquals(5, hcd.getMaxVersions()); + assertEquals(5, familyDescriptor.getMaxVersions()); // modify the max version - hcd.setValue(Bytes.toBytes(HConstants.VERSIONS), Bytes.toBytes("8")); + familyDescriptor.setValue(Bytes.toBytes(HConstants.VERSIONS), Bytes.toBytes("8")); // Verify the max version - assertEquals(8, hcd.getMaxVersions()); + assertEquals(8, familyDescriptor.getMaxVersions()); } private void verifyHColumnDescriptor(int expected, final TableName tableName, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java index 9d952d18e1e..5a36b81d6ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,12 +28,14 @@ import java.util.List; import java.util.NavigableMap; import org.apache.hadoop.hbase.TimestampTestBase.FlushCache; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -96,12 +98,17 @@ public class TestMultiVersions { */ @Test public void testTimestamps() throws Exception { - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - HColumnDescriptor hcd = new HColumnDescriptor(TimestampTestBase.FAMILY_NAME); - hcd.setMaxVersions(3); - desc.addFamily(hcd); - this.admin.createTable(desc); - Table table = UTIL.getConnection().getTable(desc.getTableName()); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + TimestampTestBase.FAMILY_NAME); + + familyDescriptor.setMaxVersions(3); + tableDescriptor.setColumnFamily(familyDescriptor); + this.admin.createTable(tableDescriptor); + Table table = UTIL.getConnection().getTable(tableDescriptor.getTableName()); // TODO: Remove these deprecated classes or pull them in here if this is // only test using them. TimestampTestBase.doTestDelete(table, new FlushCache() { @@ -137,14 +144,18 @@ public class TestMultiVersions { final byte [] value2 = Bytes.toBytes("value2"); final long timestamp1 = 100L; final long timestamp2 = 200L; - final HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - HColumnDescriptor hcd = new HColumnDescriptor(contents); - hcd.setMaxVersions(3); - desc.addFamily(hcd); - this.admin.createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(contents); + + familyDescriptor.setMaxVersions(3); + tableDescriptor.setColumnFamily(familyDescriptor); + this.admin.createTable(tableDescriptor); Put put = new Put(row, timestamp1); put.addColumn(contents, contents, value1); - Table table = UTIL.getConnection().getTable(desc.getTableName()); + Table table = UTIL.getConnection().getTable(tableDescriptor.getTableName()); table.put(put); // Shut down and restart the HBase cluster table.close(); @@ -154,7 +165,7 @@ public class TestMultiVersions { .numRegionServers(NUM_SLAVES).build(); UTIL.startMiniHBaseCluster(option); // Make a new connection. - table = UTIL.getConnection().getTable(desc.getTableName()); + table = UTIL.getConnection().getTable(tableDescriptor.getTableName()); // Overwrite previous value put = new Put(row, timestamp2); put.addColumn(contents, contents, value2); @@ -199,15 +210,19 @@ public class TestMultiVersions { @Test public void testScanMultipleVersions() throws Exception { final TableName tableName = TableName.valueOf(name.getMethodName()); - final HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); final byte [][] rows = new byte[][] { Bytes.toBytes("row_0200"), Bytes.toBytes("row_0800") }; final byte [][] splitRows = new byte[][] {Bytes.toBytes("row_0500")}; final long [] timestamp = new long[] {100L, 1000L}; - this.admin.createTable(desc, splitRows); + this.admin.createTable(tableDescriptor, splitRows); Table table = UTIL.getConnection().getTable(tableName); // Assert we got the region layout wanted. Pair keys = UTIL.getConnection() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java index ed37713d631..170182ed6a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java @@ -27,10 +27,12 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -75,7 +77,7 @@ public class TestRegionRebalancing { private static final Logger LOG = LoggerFactory.getLogger(TestRegionRebalancing.class); private final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private RegionLocator regionLocator; - private HTableDescriptor desc; + private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; private String balancerName; public TestRegionRebalancing(String balancerName) { @@ -93,8 +95,11 @@ public class TestRegionRebalancing { UTIL.getConfiguration().set("hbase.master.loadbalancer.class", this.balancerName); // set minCostNeedBalance to 0, make sure balancer run UTIL.startMiniCluster(1); - this.desc = new HTableDescriptor(TableName.valueOf("test")); - this.desc.addFamily(new HColumnDescriptor(FAMILY_NAME)); + + this.tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf("test")); + this.tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME)); } /** @@ -108,9 +113,9 @@ public class TestRegionRebalancing { throws IOException, InterruptedException { try(Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); Admin admin = connection.getAdmin()) { - admin.createTable(this.desc, Arrays.copyOfRange(HBaseTestingUtility.KEYS, + admin.createTable(this.tableDescriptor, Arrays.copyOfRange(HBaseTestingUtility.KEYS, 1, HBaseTestingUtility.KEYS.length)); - this.regionLocator = connection.getRegionLocator(this.desc.getTableName()); + this.regionLocator = connection.getRegionLocator(this.tableDescriptor.getTableName()); MetaTableAccessor.fullScanMetaAndPrint(admin.getConnection()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java index 195a2b355c6..fed0623fb42 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java @@ -589,19 +589,23 @@ public class TestSerialization { */ protected HTableDescriptor createTableDescriptor(final String name, final int versions) { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); - htd.addFamily(new HColumnDescriptor(fam1) + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1) .setMaxVersions(versions) .setBlockCacheEnabled(false) ); - htd.addFamily(new HColumnDescriptor(fam2) + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2) .setMaxVersions(versions) .setBlockCacheEnabled(false) ); - htd.addFamily(new HColumnDescriptor(fam3) + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3) .setMaxVersions(versions) .setBlockCacheEnabled(false) ); - return htd; + return new HTableDescriptor(tableDescriptor); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index dae50da34fc..558af189148 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -34,7 +34,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -94,8 +93,13 @@ public class TestAdmin2 extends TestAdminBase { msg.contains(TableName.META_TABLE_NAME.getNameAsString())); // Now try and do concurrent creation with a bunch of threads. - final HTableDescriptor threadDesc = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - threadDesc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY); + tableDescriptor.setColumnFamily(familyDescriptor); int count = 10; Thread [] threads = new Thread [count]; final AtomicInteger successes = new AtomicInteger(0); @@ -106,7 +110,7 @@ public class TestAdmin2 extends TestAdminBase { @Override public void run() { try { - localAdmin.createTable(threadDesc); + localAdmin.createTable(tableDescriptor); successes.incrementAndGet(); } catch (TableExistsException e) { failures.incrementAndGet(); @@ -141,14 +145,21 @@ public class TestAdmin2 extends TestAdminBase { @Test public void testTableNameClash() throws Exception { final String name = this.name.getMethodName(); - HTableDescriptor htd1 = new HTableDescriptor(TableName.valueOf(name + "SOMEUPPERCASE")); - HTableDescriptor htd2 = new HTableDescriptor(TableName.valueOf(name)); - htd1.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - htd2.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - ADMIN.createTable(htd1); - ADMIN.createTable(htd2); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor1 = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name + "SOMEUPPERCASE")); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor2 = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name)); + tableDescriptor1.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + tableDescriptor2.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + ADMIN.createTable(tableDescriptor1); + ADMIN.createTable(tableDescriptor2); // Before fix, below would fail throwing a NoServerForRegionException. - TEST_UTIL.getConnection().getTable(htd2.getTableName()).close(); + TEST_UTIL.getConnection().getTable(tableDescriptor2.getTableName()).close(); } /*** @@ -169,9 +180,12 @@ public class TestAdmin2 extends TestAdminBase { byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }; byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }; Admin hbaseadmin = TEST_UTIL.getAdmin(); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); - htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - hbaseadmin.createTable(htd, startKey, endKey, expectedRegions); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + hbaseadmin.createTable(tableDescriptor, startKey, endKey, expectedRegions); } finally { TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, oldTimeout); } @@ -389,9 +403,12 @@ public class TestAdmin2 extends TestAdminBase { byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }; - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - ADMIN.createTable(desc, startKey, endKey, expectedRegions); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + ADMIN.createTable(tableDescriptor, startKey, endKey, expectedRegions); List RegionInfos = ADMIN.getRegions(tableName); @@ -487,9 +504,12 @@ public class TestAdmin2 extends TestAdminBase { TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close(); // Create the test table and open it - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - ADMIN.createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + ADMIN.createTable(tableDescriptor); Table table = TEST_UTIL.getConnection().getTable(tableName); HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName); @@ -520,11 +540,15 @@ public class TestAdmin2 extends TestAdminBase { } // Before the fix for HBASE-6146, the below table creation was failing as the hbase:meta table // actually getting disabled by the disableTable() call. - HTableDescriptor htd = - new HTableDescriptor(TableName.valueOf(Bytes.toBytes(name.getMethodName()))); - HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("cf1")); - htd.addFamily(hcd); - TEST_UTIL.getAdmin().createTable(htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(Bytes.toBytes(name.getMethodName()))); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + Bytes.toBytes("cf1")); + + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java index 4e2d6bab722..92bc15e8202 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java @@ -25,9 +25,7 @@ import java.util.Optional; import java.util.concurrent.CountDownLatch; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -86,10 +84,13 @@ public class TestEnableTable { throws IOException, InterruptedException { final TableName tableName = TableName.valueOf(name.getMethodName()); final Admin admin = TEST_UTIL.getAdmin(); - final HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(FAMILYNAME)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILYNAME); + tableDescriptor.setColumnFamily(familyDescriptor); try { - createTable(TEST_UTIL, desc, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); + createTable(TEST_UTIL, tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); } catch (Exception e) { e.printStackTrace(); fail("Got an exception while creating " + tableName); @@ -163,8 +164,8 @@ public class TestEnableTable { } public static void createTable(HBaseTestingUtility testUtil, - HTableDescriptor htd, byte [][] splitKeys) - throws Exception { + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, byte [][] splitKeys) + throws Exception { // NOTE: We need a latch because admin is not sync, // so the postOp coprocessor method may be called after the admin operation returned. MasterSyncObserver observer = testUtil.getHBaseCluster().getMaster() @@ -172,13 +173,13 @@ public class TestEnableTable { observer.tableCreationLatch = new CountDownLatch(1); Admin admin = testUtil.getAdmin(); if (splitKeys != null) { - admin.createTable(htd, splitKeys); + admin.createTable(tableDescriptor, splitKeys); } else { - admin.createTable(htd); + admin.createTable(tableDescriptor); } observer.tableCreationLatch.await(); observer.tableCreationLatch = null; - testUtil.waitUntilAllRegionsAssigned(htd.getTableName()); + testUtil.waitUntilAllRegionsAssigned(tableDescriptor.getTableName()); } public static void deleteTable(HBaseTestingUtility testUtil, TableName tableName) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 6941d0490a1..757790f6717 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -282,12 +282,15 @@ public class TestFromClientSide { final byte[] T1 = Bytes.toBytes("T1"); final byte[] T2 = Bytes.toBytes("T2"); final byte[] T3 = Bytes.toBytes("T3"); - HColumnDescriptor hcd = - new HColumnDescriptor(FAMILY).setKeepDeletedCells(KeepDeletedCells.TRUE).setMaxVersions(3); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY) + .setKeepDeletedCells(KeepDeletedCells.TRUE) + .setMaxVersions(3); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(hcd); - TEST_UTIL.getAdmin().createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); try (Table h = TEST_UTIL.getConnection().getTable(tableName)) { long ts = System.currentTimeMillis(); Put p = new Put(T1, ts); @@ -6492,12 +6495,15 @@ public class TestFromClientSide { @Test public void testCellSizeLimit() throws IOException { final TableName tableName = TableName.valueOf("testCellSizeLimit"); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.setConfiguration(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024)); // 10K - HColumnDescriptor fam = new HColumnDescriptor(FAMILY); - htd.addFamily(fam); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName) + .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024)); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + + tableDescriptor.setColumnFamily(familyDescriptor); try (Admin admin = TEST_UTIL.getAdmin()) { - admin.createTable(htd); + admin.createTable(tableDescriptor); } // Will succeed try (Table t = TEST_UTIL.getConnection().getTable(tableName)) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java index 3b3f6369d3b..1ce2dd62565 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java @@ -678,10 +678,14 @@ public class TestFromClientSide3 { } private void testPreBatchMutate(TableName tableName, Runnable rn)throws Exception { - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addCoprocessor(WaitingForScanObserver.class.getName()); - desc.addFamily(new HColumnDescriptor(FAMILY)); - TEST_UTIL.getAdmin().createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + + tableDescriptor.setCoprocessor(WaitingForScanObserver.class.getName()); + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); // Don't use waitTableAvailable(), because the scanner will mess up the co-processor ExecutorService service = Executors.newFixedThreadPool(2); @@ -712,11 +716,15 @@ public class TestFromClientSide3 { @Test public void testLockLeakWithDelta() throws Exception, Throwable { - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addCoprocessor(WaitingForMultiMutationsObserver.class.getName()); - desc.setConfiguration("hbase.rowlock.wait.duration", String.valueOf(5000)); - desc.addFamily(new HColumnDescriptor(FAMILY)); - TEST_UTIL.getAdmin().createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + + tableDescriptor.setCoprocessor(WaitingForMultiMutationsObserver.class.getName()); + tableDescriptor.setValue("hbase.rowlock.wait.duration", String.valueOf(5000)); + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); TEST_UTIL.waitTableAvailable(tableName, WAITTABLE_MILLIS); // new a connection for lower retry number. @@ -767,12 +775,16 @@ public class TestFromClientSide3 { @Test public void testMultiRowMutations() throws Exception, Throwable { - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addCoprocessor(MultiRowMutationEndpoint.class.getName()); - desc.addCoprocessor(WaitingForMultiMutationsObserver.class.getName()); - desc.setConfiguration("hbase.rowlock.wait.duration", String.valueOf(5000)); - desc.addFamily(new HColumnDescriptor(FAMILY)); - TEST_UTIL.getAdmin().createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + + tableDescriptor.setCoprocessor(MultiRowMutationEndpoint.class.getName()); + tableDescriptor.setCoprocessor(WaitingForMultiMutationsObserver.class.getName()); + tableDescriptor.setValue("hbase.rowlock.wait.duration", String.valueOf(5000)); + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); TEST_UTIL.waitTableAvailable(tableName, WAITTABLE_MILLIS); // new a connection for lower retry number. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java index dc56cf79645..021a1d22eec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java @@ -30,9 +30,7 @@ import java.lang.reflect.Field; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -86,125 +84,128 @@ public class TestIllegalTableDescriptor { @Test public void testIllegalTableDescriptor() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); // create table with 0 families - checkTableIsIllegal(htd); - htd.addFamily(hcd); - checkTableIsLegal(htd); + checkTableIsIllegal(tableDescriptor); + tableDescriptor.setColumnFamily(familyDescriptor); + checkTableIsLegal(tableDescriptor); - htd.setMaxFileSize(1024); // 1K - checkTableIsIllegal(htd); - htd.setMaxFileSize(0); - checkTableIsIllegal(htd); - htd.setMaxFileSize(1024 * 1024 * 1024); // 1G - checkTableIsLegal(htd); + tableDescriptor.setMaxFileSize(1024); // 1K + checkTableIsIllegal(tableDescriptor); + tableDescriptor.setMaxFileSize(0); + checkTableIsIllegal(tableDescriptor); + tableDescriptor.setMaxFileSize(1024 * 1024 * 1024); // 1G + checkTableIsLegal(tableDescriptor); - htd.setMemStoreFlushSize(1024); - checkTableIsIllegal(htd); - htd.setMemStoreFlushSize(0); - checkTableIsIllegal(htd); - htd.setMemStoreFlushSize(128 * 1024 * 1024); // 128M - checkTableIsLegal(htd); + tableDescriptor.setMemStoreFlushSize(1024); + checkTableIsIllegal(tableDescriptor); + tableDescriptor.setMemStoreFlushSize(0); + checkTableIsIllegal(tableDescriptor); + tableDescriptor.setMemStoreFlushSize(128 * 1024 * 1024); // 128M + checkTableIsLegal(tableDescriptor); - htd.setRegionSplitPolicyClassName("nonexisting.foo.class"); - checkTableIsIllegal(htd); - htd.setRegionSplitPolicyClassName(null); - checkTableIsLegal(htd); + tableDescriptor.setRegionSplitPolicyClassName("nonexisting.foo.class"); + checkTableIsIllegal(tableDescriptor); + tableDescriptor.setRegionSplitPolicyClassName(null); + checkTableIsLegal(tableDescriptor); - htd.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, "nonexisting.foo.class"); - checkTableIsIllegal(htd); - htd.remove(HConstants.HBASE_REGION_SPLIT_POLICY_KEY); - checkTableIsLegal(htd); + tableDescriptor.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, "nonexisting.foo.class"); + checkTableIsIllegal(tableDescriptor); + tableDescriptor.removeValue(Bytes.toBytes(HConstants.HBASE_REGION_SPLIT_POLICY_KEY)); + checkTableIsLegal(tableDescriptor); - hcd.setBlocksize(0); - checkTableIsIllegal(htd); - hcd.setBlocksize(1024 * 1024 * 128); // 128M - checkTableIsIllegal(htd); - hcd.setBlocksize(1024); - checkTableIsLegal(htd); + familyDescriptor.setBlocksize(0); + checkTableIsIllegal(tableDescriptor); + familyDescriptor.setBlocksize(1024 * 1024 * 128); // 128M + checkTableIsIllegal(tableDescriptor); + familyDescriptor.setBlocksize(1024); + checkTableIsLegal(tableDescriptor); - hcd.setTimeToLive(0); - checkTableIsIllegal(htd); - hcd.setTimeToLive(-1); - checkTableIsIllegal(htd); - hcd.setTimeToLive(1); - checkTableIsLegal(htd); + familyDescriptor.setTimeToLive(0); + checkTableIsIllegal(tableDescriptor); + familyDescriptor.setTimeToLive(-1); + checkTableIsIllegal(tableDescriptor); + familyDescriptor.setTimeToLive(1); + checkTableIsLegal(tableDescriptor); - hcd.setMinVersions(-1); - checkTableIsIllegal(htd); - hcd.setMinVersions(3); + familyDescriptor.setMinVersions(-1); + checkTableIsIllegal(tableDescriptor); + familyDescriptor.setMinVersions(3); try { - hcd.setMaxVersions(2); + familyDescriptor.setMaxVersions(2); fail(); } catch (IllegalArgumentException ex) { // expected - hcd.setMaxVersions(10); + familyDescriptor.setMaxVersions(10); } - checkTableIsLegal(htd); + checkTableIsLegal(tableDescriptor); // HBASE-13776 Setting illegal versions for HColumnDescriptor // does not throw IllegalArgumentException // finally, minVersions must be less than or equal to maxVersions - hcd.setMaxVersions(4); - hcd.setMinVersions(5); - checkTableIsIllegal(htd); - hcd.setMinVersions(3); + familyDescriptor.setMaxVersions(4); + familyDescriptor.setMinVersions(5); + checkTableIsIllegal(tableDescriptor); + familyDescriptor.setMinVersions(3); - hcd.setScope(-1); - checkTableIsIllegal(htd); - hcd.setScope(0); - checkTableIsLegal(htd); + familyDescriptor.setScope(-1); + checkTableIsIllegal(tableDescriptor); + familyDescriptor.setScope(0); + checkTableIsLegal(tableDescriptor); - hcd.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "INVALID"); - checkTableIsIllegal(htd); - hcd.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "NONE"); - checkTableIsLegal(htd); + familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "INVALID"); + checkTableIsIllegal(tableDescriptor); + familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "NONE"); + checkTableIsLegal(tableDescriptor); try { - hcd.setDFSReplication((short) -1); + familyDescriptor.setDFSReplication((short) -1); fail("Illegal value for setDFSReplication did not throw"); } catch (IllegalArgumentException e) { // pass } // set an illegal DFS replication value by hand - hcd.setValue(HColumnDescriptor.DFS_REPLICATION, "-1"); - checkTableIsIllegal(htd); + familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.DFS_REPLICATION, "-1"); + checkTableIsIllegal(tableDescriptor); try { - hcd.setDFSReplication((short) -1); + familyDescriptor.setDFSReplication((short) -1); fail("Should throw exception if an illegal value is explicitly being set"); } catch (IllegalArgumentException e) { // pass } // check the conf settings to disable sanity checks - htd.setMemStoreFlushSize(0); + tableDescriptor.setMemStoreFlushSize(0); // Check that logs warn on invalid table but allow it. - htd.setConfiguration(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString()); - checkTableIsLegal(htd); + tableDescriptor.setValue(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString()); + checkTableIsLegal(tableDescriptor); verify(LOGGER).warn(contains("MEMSTORE_FLUSHSIZE for table " + "descriptor or \"hbase.hregion.memstore.flush.size\" (0) is too small, which might " + "cause very frequent flushing.")); } - private void checkTableIsLegal(HTableDescriptor htd) throws IOException { + private void checkTableIsLegal(TableDescriptor tableDescriptor) throws IOException { Admin admin = TEST_UTIL.getAdmin(); - admin.createTable(htd); - assertTrue(admin.tableExists(htd.getTableName())); - TEST_UTIL.deleteTable(htd.getTableName()); + admin.createTable(tableDescriptor); + assertTrue(admin.tableExists(tableDescriptor.getTableName())); + TEST_UTIL.deleteTable(tableDescriptor.getTableName()); } - private void checkTableIsIllegal(HTableDescriptor htd) throws IOException { + private void checkTableIsIllegal(TableDescriptor tableDescriptor) throws IOException { Admin admin = TEST_UTIL.getAdmin(); try { - admin.createTable(htd); + admin.createTable(tableDescriptor); fail(); } catch(Exception ex) { // should throw ex } - assertFalse(admin.tableExists(htd.getTableName())); + assertFalse(admin.tableExists(tableDescriptor.getTableName())); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java index 32da9bdfc87..b4533c569db 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java @@ -22,9 +22,7 @@ import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTestConst; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; @@ -60,14 +58,18 @@ public class TestIntraRowPagination { byte [][] FAMILIES = HTestConst.makeNAscii(HTestConst.DEFAULT_CF_BYTES, 3); byte [][] QUALIFIERS = HTestConst.makeNAscii(HTestConst.DEFAULT_QUALIFIER_BYTES, 10); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(HTestConst.DEFAULT_TABLE_BYTES)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(HTestConst.DEFAULT_TABLE_BYTES)); + HRegionInfo info = new HRegionInfo(HTestConst.DEFAULT_TABLE, null, null, false); for (byte[] family : FAMILIES) { - HColumnDescriptor hcd = new HColumnDescriptor(family); - htd.addFamily(hcd); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + tableDescriptor.setColumnFamily(familyDescriptor); } HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), - TEST_UTIL.getConfiguration(), htd); + TEST_UTIL.getConfiguration(), tableDescriptor); try { Put put; Scan scan; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index 47326e9f542..10d3ea69816 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -372,17 +372,18 @@ public class TestReplicaWithCluster { @SuppressWarnings("deprecation") @Test public void testReplicaAndReplication() throws Exception { - HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testReplicaAndReplication"), - HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER, - HColumnDescriptor.DEFAULT_KEEP_DELETED); - hdt.setRegionReplication(NB_SERVERS); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + HTU.createModifyableTableDescriptor("testReplicaAndReplication"); + tableDescriptor.setRegionReplication(NB_SERVERS); - HColumnDescriptor fam = new HColumnDescriptor(row); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - hdt.addFamily(fam); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(row); - hdt.addCoprocessor(SlowMeCopro.class.getName()); - HTU.getAdmin().createTable(hdt, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tableDescriptor.setColumnFamily(familyDescriptor); + + tableDescriptor.setCoprocessor(SlowMeCopro.class.getName()); + HTU.getAdmin().createTable(tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); Configuration conf2 = HBaseConfiguration.create(HTU.getConfiguration()); conf2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); @@ -393,7 +394,7 @@ public class TestReplicaWithCluster { HTU2.setZkCluster(miniZK); HTU2.startMiniCluster(NB_SERVERS); LOG.info("Setup second Zk"); - HTU2.getAdmin().createTable(hdt, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); + HTU2.getAdmin().createTable(tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); Admin admin = ConnectionFactory.createConnection(HTU.getConfiguration()).getAdmin(); @@ -404,7 +405,7 @@ public class TestReplicaWithCluster { Put p = new Put(row); p.addColumn(row, row, row); - final Table table = HTU.getConnection().getTable(hdt.getTableName()); + final Table table = HTU.getConnection().getTable(tableDescriptor.getTableName()); table.put(p); HTU.getAdmin().flush(table.getName()); @@ -428,7 +429,7 @@ public class TestReplicaWithCluster { table.close(); LOG.info("stale get on the first cluster done. Now for the second."); - final Table table2 = HTU.getConnection().getTable(hdt.getTableName()); + final Table table2 = HTU.getConnection().getTable(tableDescriptor.getTableName()); Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { try { @@ -446,11 +447,11 @@ public class TestReplicaWithCluster { }); table2.close(); - HTU.getAdmin().disableTable(hdt.getTableName()); - HTU.deleteTable(hdt.getTableName()); + HTU.getAdmin().disableTable(tableDescriptor.getTableName()); + HTU.deleteTable(tableDescriptor.getTableName()); - HTU2.getAdmin().disableTable(hdt.getTableName()); - HTU2.deleteTable(hdt.getTableName()); + HTU2.getAdmin().disableTable(tableDescriptor.getTableName()); + HTU2.deleteTable(tableDescriptor.getTableName()); // We shutdown HTU2 minicluster later, in afterClass(), as shutting down // the minicluster has negative impact of deleting all HConnections in JVM. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java index 7388a909962..5f53d7e7712 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTestConst; @@ -772,7 +771,8 @@ public class TestScannersFromClientSide { table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, ts, value)); assertArrayEquals(value, table.get(new Get(ROW)).getValue(FAMILY, QUALIFIER)); TEST_UTIL.getAdmin().modifyColumnFamily(tableName, - new HColumnDescriptor(FAMILY).setTimeToLive(5)); + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY) + .setTimeToLive(5)); try (ResultScanner scanner = table.getScanner(FAMILY)) { assertNull(scanner.next()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java index 9d073743cb6..ecb6973653f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -26,8 +26,6 @@ import java.util.Map.Entry; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -72,16 +70,19 @@ public class TestSizeFailures { qualifiers.add(Bytes.toBytes(Integer.toString(i))); } - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); - HTableDescriptor desc = new HTableDescriptor(TABLENAME); - desc.addFamily(hcd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLENAME); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + + tableDescriptor.setColumnFamily(familyDescriptor); byte[][] splits = new byte[9][2]; for (int i = 1; i < 10; i++) { int split = 48 + i; splits[i - 1][0] = (byte) (split >>> 8); splits[i - 1][0] = (byte) (split); } - TEST_UTIL.getAdmin().createTable(desc, splits); + TEST_UTIL.getAdmin().createTable(tableDescriptor, splits); Connection conn = TEST_UTIL.getConnection(); try (Table table = conn.getTable(TABLENAME)) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java index 01540fa2ef1..9b35e7071ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java @@ -24,7 +24,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -322,10 +321,11 @@ public class TestSnapshotCloneIndependence { private void runTestSnapshotMetadataChangesIndependent() throws Exception { // Add a new column family to the original table byte[] TEST_FAM_2 = Bytes.toBytes("fam2"); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM_2); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAM_2); admin.disableTable(originalTableName); - admin.addColumnFamily(originalTableName, hcd); + admin.addColumnFamily(originalTableName, familyDescriptor); // Verify that it is not in the snapshot admin.enableTable(originalTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java index 1af70cb184e..77462f8e0de 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -156,26 +155,32 @@ public class TestSnapshotMetadata { originalTableName = TableName.valueOf(sourceTableNameAsString); // enable replication on a column family - HColumnDescriptor maxVersionsColumn = new HColumnDescriptor(MAX_VERSIONS_FAM); - HColumnDescriptor bloomFilterColumn = new HColumnDescriptor(BLOOMFILTER_FAM); - HColumnDescriptor dataBlockColumn = new HColumnDescriptor(COMPRESSED_FAM); - HColumnDescriptor blockSizeColumn = new HColumnDescriptor(BLOCKSIZE_FAM); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor maxVersionsColumn = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(MAX_VERSIONS_FAM); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor bloomFilterColumn = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BLOOMFILTER_FAM); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor dataBlockColumn = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COMPRESSED_FAM); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor blockSizeColumn = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BLOCKSIZE_FAM); maxVersionsColumn.setMaxVersions(MAX_VERSIONS); bloomFilterColumn.setBloomFilterType(BLOOM_TYPE); dataBlockColumn.setDataBlockEncoding(DATA_BLOCK_ENCODING_TYPE); blockSizeColumn.setBlocksize(BLOCK_SIZE); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(sourceTableNameAsString)); - htd.addFamily(maxVersionsColumn); - htd.addFamily(bloomFilterColumn); - htd.addFamily(dataBlockColumn); - htd.addFamily(blockSizeColumn); - htd.setValue(TEST_CUSTOM_VALUE, TEST_CUSTOM_VALUE); - htd.setConfiguration(TEST_CONF_CUSTOM_VALUE, TEST_CONF_CUSTOM_VALUE); - assertTrue(htd.getConfiguration().size() > 0); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(sourceTableNameAsString)); + tableDescriptor.setColumnFamily(maxVersionsColumn); + tableDescriptor.setColumnFamily(bloomFilterColumn); + tableDescriptor.setColumnFamily(dataBlockColumn); + tableDescriptor.setColumnFamily(blockSizeColumn); + tableDescriptor.setValue(TEST_CUSTOM_VALUE, TEST_CUSTOM_VALUE); + tableDescriptor.setValue(TEST_CONF_CUSTOM_VALUE, TEST_CONF_CUSTOM_VALUE); + assertTrue(tableDescriptor.getConfiguration().size() > 0); - admin.createTable(htd); + admin.createTable(tableDescriptor); Table original = UTIL.getConnection().getTable(originalTableName); originalTableName = TableName.valueOf(sourceTableNameAsString); originalTableDescriptor = new HTableDescriptor(admin.getDescriptor(originalTableName)); @@ -299,8 +304,9 @@ public class TestSnapshotMetadata { final byte[] newFamilyName = Bytes.toBytes(newFamilyNameAsString); admin.disableTable(originalTableName); - HColumnDescriptor hcd = new HColumnDescriptor(newFamilyName); - admin.addColumnFamily(originalTableName, hcd); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(newFamilyName); + admin.addColumnFamily(originalTableName, familyDescriptor); assertTrue("New column family was not added.", admin.getDescriptor(originalTableName).toString().contains(newFamilyNameAsString)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java index 7e8ba6548b6..40952a35a36 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -23,11 +23,11 @@ import static org.junit.Assert.fail; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -76,14 +76,17 @@ public class TestConstraint { public void testConstraintPasses() throws Exception { // create the table // it would be nice if this was also a method on the util - HTableDescriptor desc = new HTableDescriptor(tableName); - for (byte[] family : new byte[][] { dummy, test }) { - desc.addFamily(new HColumnDescriptor(family)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + for (byte[] family : new byte[][]{dummy, test}) { + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } // add a constraint - Constraints.add(desc, CheckWasRunConstraint.class); + Constraints.add(tableDescriptor, CheckWasRunConstraint.class); - util.getAdmin().createTable(desc); + util.getAdmin().createTable(tableDescriptor); Table table = util.getConnection().getTable(tableName); try { // test that we don't fail on a valid put @@ -108,15 +111,17 @@ public class TestConstraint { // create the table // it would be nice if this was also a method on the util - HTableDescriptor desc = new HTableDescriptor(tableName); - for (byte[] family : new byte[][] { dummy, test }) { - desc.addFamily(new HColumnDescriptor(family)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + for (byte[] family : new byte[][]{dummy, test}) { + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } // add a constraint that is sure to fail - Constraints.add(desc, AllFailConstraint.class); + Constraints.add(tableDescriptor, AllFailConstraint.class); - util.getAdmin().createTable(desc); + util.getAdmin().createTable(tableDescriptor); Table table = util.getConnection().getTable(tableName); // test that we do fail on violation @@ -141,21 +146,23 @@ public class TestConstraint { @Test public void testDisableConstraint() throws Throwable { // create the table - HTableDescriptor desc = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); // add a family to the table - for (byte[] family : new byte[][] { dummy, test }) { - desc.addFamily(new HColumnDescriptor(family)); + for (byte[] family : new byte[][]{dummy, test}) { + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } // add a constraint to make sure it others get run - Constraints.add(desc, CheckWasRunConstraint.class); + Constraints.add(tableDescriptor, CheckWasRunConstraint.class); // Add Constraint to check - Constraints.add(desc, AllFailConstraint.class); + Constraints.add(tableDescriptor, AllFailConstraint.class); // and then disable the failing constraint - Constraints.disableConstraint(desc, AllFailConstraint.class); + Constraints.disableConstraint(tableDescriptor, AllFailConstraint.class); - util.getAdmin().createTable(desc); + util.getAdmin().createTable(tableDescriptor); Table table = util.getConnection().getTable(tableName); try { // test that we don't fail because its disabled @@ -177,18 +184,21 @@ public class TestConstraint { @Test public void testDisableConstraints() throws Throwable { // create the table - HTableDescriptor desc = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + // add a family to the table - for (byte[] family : new byte[][] { dummy, test }) { - desc.addFamily(new HColumnDescriptor(family)); + for (byte[] family : new byte[][]{dummy, test}) { + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } // add a constraint to check to see if is run - Constraints.add(desc, CheckWasRunConstraint.class); + Constraints.add(tableDescriptor, CheckWasRunConstraint.class); // then disable all the constraints - Constraints.disable(desc); + Constraints.disable(tableDescriptor); - util.getAdmin().createTable(desc); + util.getAdmin().createTable(tableDescriptor); Table table = util.getConnection().getTable(tableName); try { // test that we do fail on violation @@ -210,18 +220,21 @@ public class TestConstraint { @Test public void testIsUnloaded() throws Exception { // create the table - HTableDescriptor desc = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + // add a family to the table - for (byte[] family : new byte[][] { dummy, test }) { - desc.addFamily(new HColumnDescriptor(family)); + for (byte[] family : new byte[][]{dummy, test}) { + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } // make sure that constraints are unloaded - Constraints.add(desc, RuntimeFailConstraint.class); + Constraints.add(tableDescriptor, RuntimeFailConstraint.class); // add a constraint to check to see if is run - Constraints.add(desc, CheckWasRunConstraint.class); + Constraints.add(tableDescriptor, CheckWasRunConstraint.class); CheckWasRunConstraint.wasRun = false; - util.getAdmin().createTable(desc); + util.getAdmin().createTable(tableDescriptor); Table table = util.getConnection().getTable(tableName); // test that we do fail on violation diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java index 094c4570363..59c31570910 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -40,14 +40,14 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -412,9 +412,11 @@ public class TestCoprocessorInterface { HRegion initHRegion (TableName tableName, String callingMethod, Configuration conf, Class [] implClasses, byte [][] families) throws IOException { - HTableDescriptor htd = new HTableDescriptor(tableName); - for(byte [] family : families) { - htd.addFamily(new HColumnDescriptor(family)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + for (byte[] family : families) { + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); RegionInfo info = RegionInfoBuilder.newBuilder(tableName) @@ -423,7 +425,7 @@ public class TestCoprocessorInterface { .setSplit(false) .build(); Path path = new Path(DIR + callingMethod); - HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, tableDescriptor); // this following piece is a hack. RegionCoprocessorHost host = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java index 494c52d5c16..fa8cf7da78d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -34,9 +34,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -401,10 +399,10 @@ public class TestCoprocessorMetrics { try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); Admin admin = connection.getAdmin()) { admin.createTable( - new HTableDescriptor(tableName) - .addFamily(new HColumnDescriptor(foo)) - // add the coprocessor for the region - .addCoprocessor(CustomRegionObserver.class.getName())); + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName) + .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo)) + // add the coprocessor for the region + .setCoprocessor(CustomRegionObserver.class.getName())); try (Table table = connection.getTable(tableName)) { table.get(new Get(foo)); table.get(new Get(foo)); // 2 gets @@ -420,11 +418,12 @@ public class TestCoprocessorMetrics { try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); Admin admin = connection.getAdmin()) { admin.createTable( - new HTableDescriptor(tableName) - .addFamily(new HColumnDescriptor(foo)) - // add the coprocessor for the region - .addCoprocessor(CustomRegionObserver.class.getName()) - , new byte[][]{foo}); // create with 2 regions + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName) + .setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo)) + // add the coprocessor for the region + .setCoprocessor(CustomRegionObserver.class.getName()), new byte[][]{foo}); + // create with 2 regions try (Table table = connection.getTable(tableName); RegionLocator locator = connection.getRegionLocator(tableName)) { table.get(new Get(bar)); @@ -445,15 +444,17 @@ public class TestCoprocessorMetrics { try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); Admin admin = connection.getAdmin()) { admin.createTable( - new HTableDescriptor(tableName1) - .addFamily(new HColumnDescriptor(foo)) - // add the coprocessor for the region - .addCoprocessor(CustomRegionObserver.class.getName())); + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName1) + .setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo)) + // add the coprocessor for the region + .setCoprocessor(CustomRegionObserver.class.getName())); admin.createTable( - new HTableDescriptor(tableName2) - .addFamily(new HColumnDescriptor(foo)) - // add the coprocessor for the region - .addCoprocessor(CustomRegionObserver.class.getName())); + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName2) + .setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo)) + // add the coprocessor for the region + .setCoprocessor(CustomRegionObserver.class.getName())); try (Table table1 = connection.getTable(tableName1); Table table2 = connection.getTable(tableName2)) { table1.get(new Get(bar)); @@ -469,11 +470,12 @@ public class TestCoprocessorMetrics { try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); Admin admin = connection.getAdmin()) { admin.createTable( - new HTableDescriptor(tableName) - .addFamily(new HColumnDescriptor(foo)) - // add the coprocessor for the region. We add two different coprocessors - .addCoprocessor(CustomRegionObserver.class.getName()) - .addCoprocessor(CustomRegionObserver2.class.getName())); + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName) + .setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo)) + // add the coprocessor for the region. We add two different coprocessors + .setCoprocessor(CustomRegionObserver.class.getName()) + .setCoprocessor(CustomRegionObserver2.class.getName())); try (Table table = connection.getTable(tableName)) { table.get(new Get(foo)); table.get(new Get(foo)); // 2 gets @@ -491,11 +493,12 @@ public class TestCoprocessorMetrics { try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); Admin admin = connection.getAdmin()) { admin.createTable( - new HTableDescriptor(tableName) - .addFamily(new HColumnDescriptor(foo)) - // add the coprocessor for the region - .addCoprocessor(CustomRegionObserver.class.getName()) - , new byte[][]{foo}); // create with 2 regions + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName) + .setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo)) + // add the coprocessor for the region + .setCoprocessor(CustomRegionObserver.class.getName()), new byte[][]{foo}); + // create with 2 regions try (Table table = connection.getTable(tableName)) { table.get(new Get(foo)); table.get(new Get(foo)); // 2 gets @@ -535,10 +538,11 @@ public class TestCoprocessorMetrics { try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); Admin admin = connection.getAdmin()) { admin.createTable( - new HTableDescriptor(tableName) - .addFamily(new HColumnDescriptor(foo)) - // add the coprocessor for the region - .addCoprocessor(CustomRegionEndpoint.class.getName())); + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName) + .setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo)) + // add the coprocessor for the region + .setCoprocessor(CustomRegionEndpoint.class.getName())); try (Table table = connection.getTable(tableName)) { List mutations = Lists.newArrayList(new Put(foo), new Put(bar)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java index d0031ff65dd..a0b040378e7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -29,14 +29,15 @@ import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; @@ -88,11 +89,14 @@ public class TestMasterCoprocessorExceptionWithAbort { public void run() { // create a table : master coprocessor will throw an exception and not // catch it. - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TEST_TABLE)); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE)); + ColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); + tableDescriptor.setColumnFamily(familyDescriptor); try { Admin admin = UTIL.getAdmin(); - admin.createTable(htd); + admin.createTable(tableDescriptor); fail("BuggyMasterObserver failed to throw an exception."); } catch (IOException e) { assertEquals("HBaseAdmin threw an interrupted IOException as expected.", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java index f3e0d4c9f17..c4636fbc608 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,13 +28,13 @@ import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; @@ -196,13 +196,16 @@ public class TestMasterCoprocessorExceptionWithRemove { BuggyMasterObserver.class.getName(); assertTrue(HMaster.getLoadedCoprocessors().contains(coprocessorName)); - HTableDescriptor htd1 = new HTableDescriptor(TableName.valueOf(TEST_TABLE1)); - htd1.addFamily(new HColumnDescriptor(TEST_FAMILY1)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor1 = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE1)); + + tableDescriptor1.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY1)); boolean threwDNRE = false; try { Admin admin = UTIL.getAdmin(); - admin.createTable(htd1); + admin.createTable(tableDescriptor1); } catch (IOException e) { if (e.getClass().getName().equals("org.apache.hadoop.hbase.DoNotRetryIOException")) { threwDNRE = true; @@ -226,11 +229,13 @@ public class TestMasterCoprocessorExceptionWithRemove { // Verify that BuggyMasterObserver has been removed due to its misbehavior // by creating another table: should not have a problem this time. - HTableDescriptor htd2 = new HTableDescriptor(TableName.valueOf(TEST_TABLE2)); - htd2.addFamily(new HColumnDescriptor(TEST_FAMILY2)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor2 = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE2)); + tableDescriptor2.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY2)); Admin admin = UTIL.getAdmin(); try { - admin.createTable(htd2); + admin.createTable(tableDescriptor2); } catch (IOException e) { fail("Failed to create table after buggy coprocessor removal: " + e); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java index d49f9e20192..eef3229be80 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -32,7 +32,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.MasterSwitchType; @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.RegionPlan; @@ -1323,12 +1324,15 @@ public class TestMasterObserver { assertFalse("No table created yet", cp.wasCreateTableCalled()); // create a table - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); try(Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); Admin admin = connection.getAdmin()) { tableCreationLatch = new CountDownLatch(1); - admin.createTable(htd, Arrays.copyOfRange(HBaseTestingUtility.KEYS, + admin.createTable(tableDescriptor, Arrays.copyOfRange(HBaseTestingUtility.KEYS, 1, HBaseTestingUtility.KEYS.length)); assertTrue("Test table should be created", cp.wasCreateTableCalled()); @@ -1338,7 +1342,7 @@ public class TestMasterObserver { assertTrue("Table create handler should be called.", cp.wasCreateTableActionCalled()); - RegionLocator regionLocator = connection.getRegionLocator(htd.getTableName()); + RegionLocator regionLocator = connection.getRegionLocator(tableDescriptor.getTableName()); List regions = regionLocator.getAllRegionLocations(); admin.mergeRegionsAsync(regions.get(0).getRegion().getEncodedNameAsBytes(), @@ -1367,8 +1371,8 @@ public class TestMasterObserver { assertTrue(admin.isTableDisabled(tableName)); // modify table - htd.setMaxFileSize(512 * 1024 * 1024); - modifyTableSync(admin, tableName, htd); + tableDescriptor.setMaxFileSize(512 * 1024 * 1024); + modifyTableSync(admin, tableName, tableDescriptor); assertTrue("Test table should have been modified", cp.wasModifyTableCalled()); @@ -1389,7 +1393,7 @@ public class TestMasterObserver { // When bypass was supported, we'd turn off bypass and rerun tests. Leaving rerun in place. cp.resetStates(); - admin.createTable(htd); + admin.createTable(tableDescriptor); assertTrue("Test table should be created", cp.wasCreateTableCalled()); tableCreationLatch.await(); assertTrue("Table pre create handler called.", cp @@ -1408,8 +1412,8 @@ public class TestMasterObserver { cp.wasDisableTableActionCalled()); // modify table - htd.setMaxFileSize(512 * 1024 * 1024); - modifyTableSync(admin, tableName, htd); + tableDescriptor.setMaxFileSize(512 * 1024 * 1024); + modifyTableSync(admin, tableName, tableDescriptor); assertTrue("Test table should have been modified", cp.wasModifyTableCalled()); @@ -1451,12 +1455,15 @@ public class TestMasterObserver { cp.resetStates(); // create a table - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); Admin admin = UTIL.getAdmin(); tableCreationLatch = new CountDownLatch(1); - admin.createTable(htd); + admin.createTable(tableDescriptor); tableCreationLatch.await(); tableCreationLatch = new CountDownLatch(1); @@ -1527,13 +1534,14 @@ public class TestMasterObserver { // been removed so the testing code was removed. } - private void modifyTableSync(Admin admin, TableName tableName, HTableDescriptor htd) + private void modifyTableSync(Admin admin, TableName tableName, TableDescriptor tableDescriptor) throws IOException { - admin.modifyTable(htd); + admin.modifyTable(tableDescriptor); //wait until modify table finishes for (int t = 0; t < 100; t++) { //10 sec timeout - HTableDescriptor td = new HTableDescriptor(admin.getDescriptor(htd.getTableName())); - if (td.equals(htd)) { + HTableDescriptor td = new HTableDescriptor( + admin.getDescriptor(tableDescriptor.getTableName())); + if (td.equals(tableDescriptor)) { break; } Threads.sleep(100); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java index 860a86b1e5a..715c15d3051 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -29,16 +29,16 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Threads; @@ -164,18 +164,24 @@ public class TestOpenTableInCoprocessor { throws Throwable { // Check if given class implements RegionObserver. assert(RegionObserver.class.isAssignableFrom(clazz)); - HTableDescriptor primary = new HTableDescriptor(primaryTable); - primary.addFamily(new HColumnDescriptor(family)); - // add our coprocessor - primary.addCoprocessor(clazz.getName()); + TableDescriptorBuilder.ModifyableTableDescriptor primaryDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(primaryTable); - HTableDescriptor other = new HTableDescriptor(otherTable); - other.addFamily(new HColumnDescriptor(family)); + primaryDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); + // add our coprocessor + primaryDescriptor.setCoprocessor(clazz.getName()); + + TableDescriptorBuilder.ModifyableTableDescriptor otherDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(otherTable); + + otherDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); Admin admin = UTIL.getAdmin(); - admin.createTable(primary); - admin.createTable(other); + admin.createTable(primaryDescriptor); + admin.createTable(otherDescriptor); Table table = UTIL.getConnection().getTable(TableName.valueOf("primary")); Put p = new Put(new byte[] { 'a' }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java index 21435d94308..9e2bd7d6aa5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java @@ -29,9 +29,8 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; @@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -98,11 +98,12 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors { } private void createTable(String coprocessor) throws IOException { - HTableDescriptor htd = new HTableDescriptor(tableName) - .addFamily(new HColumnDescriptor(dummy)) - .addFamily(new HColumnDescriptor(test)) - .addCoprocessor(coprocessor); - util.getAdmin().createTable(htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName) + .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(dummy)) + .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(test)) + .setCoprocessor(coprocessor); + util.getAdmin().createTable(tableDescriptor); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java index e9a354a2688..e72069cc6bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -37,8 +37,6 @@ import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; @@ -530,10 +528,13 @@ public class TestRegionObserverInterface { admin.deleteTable(compactTable); } - HTableDescriptor htd = new HTableDescriptor(compactTable); - htd.addFamily(new HColumnDescriptor(A)); - htd.addCoprocessor(EvenOnlyCompactor.class.getName()); - admin.createTable(htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(compactTable); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(A)); + tableDescriptor.setCoprocessor(EvenOnlyCompactor.class.getName()); + admin.createTable(tableDescriptor); Table table = util.getConnection().getTable(compactTable); for (long i = 1; i <= 10; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index bdea5fd2a88..8aea4166f3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.concurrent.CountDownLatch; @@ -31,12 +32,12 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker; @@ -183,15 +185,17 @@ public class TestRegionObserverScannerOpenHook { HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf, byte[]... families) throws IOException { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); for (byte[] family : families) { - htd.addFamily(new HColumnDescriptor(family)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); Path path = new Path(DIR + callingMethod); WAL wal = HBaseTestingUtility.createWal(conf, path, info); - HRegion r = HRegion.createHRegion(info, path, conf, htd, wal); + HRegion r = HRegion.createHRegion(info, path, conf, tableDescriptor, wal); // this following piece is a hack. currently a coprocessorHost // is secretly loaded at OpenRegionHandler. we don't really // start a region server here, so just manually create cphost @@ -303,24 +307,37 @@ public class TestRegionObserverScannerOpenHook { UTIL.startMiniCluster(); byte[] ROW = Bytes.toBytes("testRow"); byte[] A = Bytes.toBytes("A"); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - desc.addFamily(new HColumnDescriptor(A)); - desc.addCoprocessor(EmptyRegionObsever.class.getName(), null, Coprocessor.PRIORITY_USER, null); - desc.addCoprocessor(NoDataFromCompaction.class.getName(), null, Coprocessor.PRIORITY_HIGHEST, - null); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(A)); + tableDescriptor.setCoprocessor( + CoprocessorDescriptorBuilder.newBuilder(EmptyRegionObsever.class.getName()) + .setJarPath(null) + .setPriority(Coprocessor.PRIORITY_USER) + .setProperties(Collections.emptyMap()) + .build()); + tableDescriptor.setCoprocessor( + CoprocessorDescriptorBuilder.newBuilder(NoDataFromCompaction.class.getName()) + .setJarPath(null) + .setPriority(Coprocessor.PRIORITY_HIGHEST) + .setProperties(Collections.emptyMap()) + .build()); Admin admin = UTIL.getAdmin(); - admin.createTable(desc); + admin.createTable(tableDescriptor); - Table table = UTIL.getConnection().getTable(desc.getTableName()); + Table table = UTIL.getConnection().getTable(tableDescriptor.getTableName()); // put a row and flush it to disk Put put = new Put(ROW); put.addColumn(A, A, A); table.put(put); - HRegionServer rs = UTIL.getRSForFirstRegionInTable(desc.getTableName()); - List regions = rs.getRegions(desc.getTableName()); + HRegionServer rs = UTIL.getRSForFirstRegionInTable(tableDescriptor.getTableName()); + List regions = rs.getRegions(tableDescriptor.getTableName()); assertEquals("More than 1 region serving test table with 1 row", 1, regions.size()); Region region = regions.get(0); admin.flushRegion(region.getRegionInfo().getRegionName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java index ab5da8f585a..eeb0c3fa037 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -25,12 +25,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl; @@ -120,14 +120,18 @@ public class TestRegionObserverStacking extends TestCase { HRegion initHRegion (byte [] tableName, String callingMethod, Configuration conf, byte [] ... families) throws IOException { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); - for(byte [] family : families) { - htd.addFamily(new HColumnDescriptor(family)); + + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); + + for (byte[] family : families) { + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); Path path = new Path(DIR + callingMethod); - HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, tableDescriptor); // this following piece is a hack. currently a coprocessorHost // is secretly loaded at OpenRegionHandler. we don't really // start a region server here, so just manually create cphost diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java index ac7928f931a..50dd3e32603 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java @@ -30,14 +30,14 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -57,12 +57,15 @@ public class FilterTestingCluster { protected static void createTable(TableName tableName, String columnFamilyName) { assertNotNull("HBaseAdmin is not initialized successfully.", admin); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor colDef = new HColumnDescriptor(Bytes.toBytes(columnFamilyName)); - desc.addFamily(colDef); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + Bytes.toBytes(columnFamilyName))); try { - admin.createTable(desc); + admin.createTable(tableDescriptor); createdTables.add(tableName); assertTrue("Fail to create the table", admin.tableExists(tableName)); } catch (IOException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java index dc8a42364a0..a7fa823c475 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -30,13 +30,13 @@ import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -83,16 +83,21 @@ public class TestDependentColumnFilter { public void setUp() throws Exception { testVals = makeTestVals(); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(this.getClass().getSimpleName())); - HColumnDescriptor hcd0 = new HColumnDescriptor(FAMILIES[0]); - hcd0.setMaxVersions(3); - htd.addFamily(hcd0); - HColumnDescriptor hcd1 = new HColumnDescriptor(FAMILIES[1]); - hcd1.setMaxVersions(3); - htd.addFamily(hcd1); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(this.getClass().getSimpleName())); + + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd0 = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0]) + .setMaxVersions(3); + tableDescriptor.setColumnFamily(hcd0); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd1 = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1]) + .setMaxVersions(3); + tableDescriptor.setColumnFamily(hcd1); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), - TEST_UTIL.getConfiguration(), htd); + TEST_UTIL.getConfiguration(), tableDescriptor); addData(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index 9a94bd31613..3200b1ac8e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -32,16 +32,16 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -150,17 +150,26 @@ public class TestFilter { @Before public void setUp() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("TestFilter")); - HColumnDescriptor family0 = new HColumnDescriptor(FAMILIES[0]).setVersions(100, 100); - htd.addFamily(family0); - htd.addFamily(new HColumnDescriptor(FAMILIES[1])); - htd.addFamily(new HColumnDescriptor(FAMILIES_1[0])); - htd.addFamily(new HColumnDescriptor(FAMILIES_1[1])); - htd.addFamily(new HColumnDescriptor(NEW_FAMILIES[0])); - htd.addFamily(new HColumnDescriptor(NEW_FAMILIES[1])); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf("TestFilter")); + + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor family0 = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0]) + .setVersions(100, 100); + tableDescriptor.setColumnFamily(family0); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1])); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES_1[0])); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES_1[1])); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(NEW_FAMILIES[0])); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(NEW_FAMILIES[1])); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), - TEST_UTIL.getConfiguration(), htd); + TEST_UTIL.getConfiguration(), tableDescriptor); // Insert first half for(byte [] ROW : ROWS_ONE) { @@ -1505,11 +1514,15 @@ public class TestFilter { public void testFilterListWithPrefixFilter() throws IOException { byte[] family = Bytes.toBytes("f1"); byte[] qualifier = Bytes.toBytes("q1"); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - htd.addFamily(new HColumnDescriptor(family)); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), - TEST_UTIL.getConfiguration(), htd); + TEST_UTIL.getConfiguration(), tableDescriptor); for(int i=0; i<5; i++) { Put p = new Put(Bytes.toBytes((char)('a'+i) + "row")); @@ -2275,11 +2288,15 @@ public class TestFilter { @Ignore("TODO: intentionally disabled?") public void testNestedFilterListWithSCVF() throws IOException { byte[] columnStatus = Bytes.toBytes("S"); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - htd.addFamily(new HColumnDescriptor(FAMILIES[0])); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0])); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), - TEST_UTIL.getConfiguration(), htd); + TEST_UTIL.getConfiguration(), tableDescriptor); for(int i=0; i<10; i++) { Put p = new Put(Bytes.toBytes("row" + i)); p.setDurability(Durability.SKIP_WAL); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java index 8b8b4313c70..82434c0e5b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -27,14 +27,14 @@ import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTestConst; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -82,14 +82,17 @@ public class TestFilterFromRegionSide { @BeforeClass public static void setUpBeforeClass() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TABLE_NAME); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + for (byte[] family : FAMILIES) { - HColumnDescriptor hcd = new HColumnDescriptor(family); - htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + tableDescriptor.setColumnFamily(familyDescriptor); } - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - REGION = HBaseTestingUtility - .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); + REGION = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), + TEST_UTIL.getConfiguration(), tableDescriptor); for(Put put:createPuts(ROWS, FAMILIES, QUALIFIERS, VALUE)){ REGION.put(put); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java index 05df8b0c18b..a00b1b1df55 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -29,13 +29,12 @@ import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -150,12 +150,15 @@ public class TestFilterWrapper { assertNotNull("HBaseAdmin is not initialized successfully.", admin); if (admin != null) { - HTableDescriptor desc = new HTableDescriptor(name); - HColumnDescriptor coldef = new HColumnDescriptor(Bytes.toBytes("f1")); - desc.addFamily(coldef); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(name); + + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("f1")); + tableDescriptor.setColumnFamily(familyDescriptor); try { - admin.createTable(desc); + admin.createTable(tableDescriptor); assertTrue("Fail to create the table", admin.tableExists(name)); } catch (IOException e) { assertNull("Exception found while creating table", e); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java index 5184b14f753..e433ccf507e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -22,14 +22,14 @@ import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.testclassification.FilterTests; @@ -67,9 +67,11 @@ public class TestInvocationRecordFilter { @Before public void setUp() throws Exception { - HTableDescriptor htd = new HTableDescriptor( - TableName.valueOf(TABLE_NAME_BYTES)); - htd.addFamily(new HColumnDescriptor(FAMILY_NAME_BYTES)); + TableDescriptorBuilder.ModifyableTableDescriptor htd = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TABLE_NAME_BYTES)); + + htd.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME_BYTES)); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java index decbb42eee4..dfe0d4c91a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -26,13 +26,13 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; @@ -62,11 +62,16 @@ public class TestSeekBeforeWithReverseScan { @Before public void setUp() throws Exception { TableName tableName = TableName.valueOf(getClass().getSimpleName()); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(cfName).setDataBlockEncoding(DataBlockEncoding.FAST_DIFF)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName) + .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF)); HRegionInfo info = new HRegionInfo(tableName, null, null, false); Path path = testUtil.getDataTestDir(getClass().getSimpleName()); - region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(), htd); + region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(), + tableDescriptor); } @After diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java index 72eb5c06cab..9aabc636801 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -30,12 +30,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -101,13 +101,17 @@ public class TestScannerSelectionUsingKeyRange { public void testScannerSelection() throws IOException { Configuration conf = TEST_UTIL.getConfiguration(); conf.setInt("hbase.hstore.compactionThreshold", 10000); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY_BYTES).setBlockCacheEnabled(true) + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_BYTES) + .setBlockCacheEnabled(true) .setBloomFilterType(bloomType); - HTableDescriptor htd = new HTableDescriptor(TABLE); - htd.addFamily(hcd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE); + + tableDescriptor.setColumnFamily(familyDescriptor); HRegionInfo info = new HRegionInfo(TABLE); HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf, - htd); + tableDescriptor); for (int iFile = 0; iFile < NUM_FILES; ++iFile) { for (int iRow = 0; iRow < NUM_ROWS; ++iRow) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index fdc22c7f254..bdcc602dfe5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java @@ -32,9 +32,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.PleaseHoldException; @@ -241,9 +239,12 @@ public class TestMaster { int msgInterval = conf.getInt("hbase.regionserver.msginterval", 100); // insert some data into META TableName tableName = TableName.valueOf("testFlushSeqId"); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(Bytes.toBytes("cf"))); - Table table = TEST_UTIL.createTable(desc, null); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("cf"))); + Table table = TEST_UTIL.createTable(tableDescriptor, null); // flush META region TEST_UTIL.flush(TableName.META_TABLE_NAME); // wait for regionserver report diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java index 1d6db5cd8cf..1241e261332 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -24,10 +24,10 @@ import java.util.AbstractMap.SimpleImmutableEntry; import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; @@ -125,10 +125,14 @@ public class TestMasterMetricsWrapper { TableName table = TableName.valueOf("testRegionNumber"); try { RegionInfo hri; - HTableDescriptor desc = new HTableDescriptor(table); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(table); + byte[] FAMILY = Bytes.toBytes("FAMILY"); - desc.addFamily(new HColumnDescriptor(FAMILY)); - TEST_UTIL.getAdmin().createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), 5); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)); + TEST_UTIL.getAdmin().createTable(tableDescriptor, Bytes.toBytes("A"), + Bytes.toBytes("Z"), 5); // wait till the table is assigned long timeoutTime = System.currentTimeMillis() + 1000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index 1d78e13583f..aa9408b9a63 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -36,21 +36,21 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper; import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer; import org.apache.hadoop.hbase.favored.FavoredNodesPlan; @@ -549,9 +549,13 @@ public class TestRegionPlacement { splitKeys[i - 1] = new byte[] { splitKey, splitKey, splitKey }; } - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, splitKeys); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, splitKeys); try (RegionLocator r = CONNECTION.getRegionLocator(tableName)) { List regions = r.getAllRegionLocations(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java index e7e744698fa..6a4fc8c9111 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java @@ -26,13 +26,13 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -83,10 +83,14 @@ public class TestFavoredNodeTableImport { admin.balancerSwitch(false, true); String tableName = "testFNImport"; - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), REGION_NUM); - UTIL.waitTableAvailable(desc.getTableName()); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("a"), Bytes.toBytes("z"), REGION_NUM); + UTIL.waitTableAvailable(tableDescriptor.getTableName()); admin.balancerSwitch(true, true); LOG.info("Shutting down cluster"); @@ -101,7 +105,7 @@ public class TestFavoredNodeTableImport { while (!master.isInitialized()) { Threads.sleep(1); } - UTIL.waitTableAvailable(desc.getTableName()); + UTIL.waitTableAvailable(tableDescriptor.getTableName()); UTIL.waitUntilNoRegionsInTransition(10000); assertTrue(master.isBalancerOn()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index 494fca32e67..8a1ee8a8dee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -35,15 +35,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper; import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.favored.FavoredNodesPlan; @@ -117,9 +117,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void testBasicBalance() throws Exception { TableName tableName = TableName.valueOf("testBasicBalance"); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); TEST_UTIL.waitTableAvailable(tableName); TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); admin.flush(tableName); @@ -150,9 +154,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void testRoundRobinAssignment() throws Exception { TableName tableName = TableName.valueOf("testRoundRobinAssignment"); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); TEST_UTIL.waitTableAvailable(tableName); TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); admin.flush(tableName); @@ -174,10 +182,14 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void testBasicRegionPlacementAndReplicaLoad() throws Exception { String tableName = "testBasicRegionPlacement"; - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); - TEST_UTIL.waitTableAvailable(desc.getTableName()); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName()); FavoredNodesManager fnm = master.getFavoredNodesManager(); List regionsOfTable = admin.getRegions(TableName.valueOf(tableName)); @@ -214,10 +226,14 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void testRandomAssignmentWithNoFavNodes() throws Exception { final String tableName = "testRandomAssignmentWithNoFavNodes"; - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc); - TEST_UTIL.waitTableAvailable(desc.getTableName()); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor); + TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName()); RegionInfo hri = admin.getRegions(TableName.valueOf(tableName)).get(0); @@ -245,9 +261,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void testBalancerWithoutFavoredNodes() throws Exception { TableName tableName = TableName.valueOf("testBalancerWithoutFavoredNodes"); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); TEST_UTIL.waitTableAvailable(tableName); final RegionInfo region = admin.getRegions(tableName).get(0); @@ -283,9 +303,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void testMisplacedRegions() throws Exception { TableName tableName = TableName.valueOf("testMisplacedRegions"); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); TEST_UTIL.waitTableAvailable(tableName); final RegionInfo misplacedRegion = admin.getRegions(tableName).get(0); @@ -330,9 +354,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void test2FavoredNodesDead() throws Exception { TableName tableName = TableName.valueOf("testAllFavoredNodesDead"); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); TEST_UTIL.waitTableAvailable(tableName); final RegionInfo region = admin.getRegions(tableName).get(0); @@ -368,9 +396,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void testAllFavoredNodesDead() throws Exception { TableName tableName = TableName.valueOf("testAllFavoredNodesDead"); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); TEST_UTIL.waitTableAvailable(tableName); final RegionInfo region = admin.getRegions(tableName).get(0); @@ -430,9 +462,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { public void testAllFavoredNodesDeadMasterRestarted() throws Exception { TableName tableName = TableName.valueOf("testAllFavoredNodesDeadMasterRestarted"); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); TEST_UTIL.waitTableAvailable(tableName); final RegionInfo region = admin.getRegions(tableName).get(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java index 9d1cffb7f85..64ecb4200cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -24,13 +24,15 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.SnapshotDescription; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; @@ -71,7 +73,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase { protected final int rowCountCF1addition = 10; private SnapshotProtos.SnapshotDescription snapshot = null; - private HTableDescriptor snapshotHTD = null; + private TableDescriptor snapshotHTD = null; @Rule public TestName name = new TestName(); @@ -106,7 +108,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase { SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF2, CF2); SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 + rowCountCF2); - snapshotHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName)); + snapshotHTD = admin.getDescriptor(snapshotTableName); admin.disableTable(snapshotTableName); // take a snapshot @@ -116,8 +118,10 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase { snapshot = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotList.get(0)); // modify the table - HColumnDescriptor columnFamilyDescriptor3 = new HColumnDescriptor(CF3); - HColumnDescriptor columnFamilyDescriptor4 = new HColumnDescriptor(CF4); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor columnFamilyDescriptor3 = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(CF3); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor columnFamilyDescriptor4 = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(CF4); admin.addColumnFamily(snapshotTableName, columnFamilyDescriptor3); admin.addColumnFamily(snapshotTableName, columnFamilyDescriptor4); admin.deleteColumnFamily(snapshotTableName, CF2); @@ -131,19 +135,22 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase { assertFalse(currentHTD.hasFamily(CF2)); assertTrue(currentHTD.hasFamily(CF3)); assertTrue(currentHTD.hasFamily(CF4)); - assertNotEquals(currentHTD.getFamiliesKeys().size(), snapshotHTD.getFamiliesKeys().size()); + assertNotEquals(currentHTD.getFamiliesKeys().size(), snapshotHTD.getColumnFamilies().length); SnapshotTestingUtils.verifyRowCount( UTIL, snapshotTableName, rowCountCF1 + rowCountCF3 + rowCountCF4 + rowCountCF1addition); admin.disableTable(snapshotTableName); } - private static HTableDescriptor createHTableDescriptor( + private static TableDescriptor createHTableDescriptor( final TableName tableName, final byte[] ... family) { - HTableDescriptor htd = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + for (int i = 0; i < family.length; ++i) { - htd.addFamily(new HColumnDescriptor(family[i])); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family[i])); } - return htd; + return tableDescriptor; } @Test @@ -162,10 +169,11 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase { public void testRestoreSnapshotToDifferentTable() throws Exception { final ProcedureExecutor procExec = getMasterProcedureExecutor(); final TableName restoredTableName = TableName.valueOf(name.getMethodName()); - final HTableDescriptor newHTD = createHTableDescriptor(restoredTableName, CF1, CF2); + final TableDescriptor tableDescriptor = createHTableDescriptor(restoredTableName, CF1, CF2); long procId = ProcedureTestingUtility.submitAndWait( - procExec, new RestoreSnapshotProcedure(procExec.getEnvironment(), newHTD, snapshot)); + procExec, new RestoreSnapshotProcedure(procExec.getEnvironment(), tableDescriptor, + snapshot)); Procedure result = procExec.getResult(procId); assertTrue(result.isFailed()); LOG.debug("Restore snapshot failed with exception: " + result.getException()); @@ -220,7 +228,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase { assertTrue(currentHTD.hasFamily(CF2)); assertFalse(currentHTD.hasFamily(CF3)); assertFalse(currentHTD.hasFamily(CF4)); - assertEquals(currentHTD.getFamiliesKeys().size(), snapshotHTD.getFamiliesKeys().size()); + assertEquals(currentHTD.getFamiliesKeys().size(), snapshotHTD.getColumnFamilies().length); SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 + rowCountCF2); } finally { UTIL.getAdmin().disableTable(snapshotTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java index 7e68b39154e..526127afac1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java @@ -29,7 +29,9 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -88,19 +90,26 @@ public class TestTableDescriptorModificationFromClient { public void testModifyTable() throws IOException { Admin admin = TEST_UTIL.getAdmin(); // Create a table with one family - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - baseHtd.addFamily(new HColumnDescriptor(FAMILY_0)); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0)); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the table descriptor verifyTableDescriptor(TABLE_NAME, FAMILY_0); // Modify the table adding another family and verify the descriptor - HTableDescriptor modifiedHtd = new HTableDescriptor(TABLE_NAME); - modifiedHtd.addFamily(new HColumnDescriptor(FAMILY_0)); - modifiedHtd.addFamily(new HColumnDescriptor(FAMILY_1)); - admin.modifyTable(modifiedHtd); + TableDescriptorBuilder.ModifyableTableDescriptor modifiedtableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + + modifiedtableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0)); + modifiedtableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1)); + admin.modifyTable(modifiedtableDescriptor); verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1); } finally { admin.deleteTable(TABLE_NAME); @@ -111,16 +120,20 @@ public class TestTableDescriptorModificationFromClient { public void testAddColumn() throws IOException { Admin admin = TEST_UTIL.getAdmin(); // Create a table with two families - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - baseHtd.addFamily(new HColumnDescriptor(FAMILY_0)); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0)); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the table descriptor verifyTableDescriptor(TABLE_NAME, FAMILY_0); // Modify the table removing one family and verify the descriptor - admin.addColumnFamily(TABLE_NAME, new HColumnDescriptor(FAMILY_1)); + admin.addColumnFamily(TABLE_NAME, + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1)); verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1); } finally { admin.deleteTable(TABLE_NAME); @@ -131,21 +144,26 @@ public class TestTableDescriptorModificationFromClient { public void testAddSameColumnFamilyTwice() throws IOException { Admin admin = TEST_UTIL.getAdmin(); // Create a table with one families - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - baseHtd.addFamily(new HColumnDescriptor(FAMILY_0)); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0)); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the table descriptor verifyTableDescriptor(TABLE_NAME, FAMILY_0); // Modify the table removing one family and verify the descriptor - admin.addColumnFamily(TABLE_NAME, new HColumnDescriptor(FAMILY_1)); + admin.addColumnFamily(TABLE_NAME, + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1)); verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1); try { // Add same column family again - expect failure - admin.addColumnFamily(TABLE_NAME, new HColumnDescriptor(FAMILY_1)); + admin.addColumnFamily(TABLE_NAME, + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1)); Assert.fail("Delete a non-exist column family should fail"); } catch (InvalidFamilyOperationException e) { // Expected. @@ -160,12 +178,15 @@ public class TestTableDescriptorModificationFromClient { public void testModifyColumnFamily() throws IOException { Admin admin = TEST_UTIL.getAdmin(); - HColumnDescriptor cfDescriptor = new HColumnDescriptor(FAMILY_0); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor cfDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0); int blockSize = cfDescriptor.getBlocksize(); // Create a table with one families - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - baseHtd.addFamily(cfDescriptor); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + + tableDescriptor.setColumnFamily(cfDescriptor); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the table descriptor @@ -189,12 +210,16 @@ public class TestTableDescriptorModificationFromClient { public void testModifyNonExistingColumnFamily() throws IOException { Admin admin = TEST_UTIL.getAdmin(); - HColumnDescriptor cfDescriptor = new HColumnDescriptor(FAMILY_1); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor cfDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1); int blockSize = cfDescriptor.getBlocksize(); // Create a table with one families - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - baseHtd.addFamily(new HColumnDescriptor(FAMILY_0)); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0)); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the table descriptor @@ -220,10 +245,14 @@ public class TestTableDescriptorModificationFromClient { public void testDeleteColumn() throws IOException { Admin admin = TEST_UTIL.getAdmin(); // Create a table with two families - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - baseHtd.addFamily(new HColumnDescriptor(FAMILY_0)); - baseHtd.addFamily(new HColumnDescriptor(FAMILY_1)); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1)); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the table descriptor @@ -241,10 +270,14 @@ public class TestTableDescriptorModificationFromClient { public void testDeleteSameColumnFamilyTwice() throws IOException { Admin admin = TEST_UTIL.getAdmin(); // Create a table with two families - HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME); - baseHtd.addFamily(new HColumnDescriptor(FAMILY_0)); - baseHtd.addFamily(new HColumnDescriptor(FAMILY_1)); - admin.createTable(baseHtd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1)); + admin.createTable(tableDescriptor); admin.disableTable(TABLE_NAME); try { // Verify the table descriptor diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java index 011fcadbf86..cf6a4225115 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -28,16 +28,16 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; import org.apache.hadoop.hbase.util.Bytes; import org.slf4j.Logger; @@ -71,8 +71,8 @@ public class MobStressToolRunner { .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); private Configuration conf; - private HTableDescriptor hdt; - private HColumnDescriptor hcd; + private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; + private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor; private Admin admin; private long count = 500000; private double failureProb = 0.1; @@ -90,20 +90,21 @@ public class MobStressToolRunner { this.count = numRows; initConf(); printConf(); - hdt = new HTableDescriptor(TableName.valueOf("testMobCompactTable")); + tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf("testMobCompactTable")); Connection conn = ConnectionFactory.createConnection(this.conf); this.admin = conn.getAdmin(); - this.hcd = new HColumnDescriptor(fam); - this.hcd.setMobEnabled(true); - this.hcd.setMobThreshold(mobLen); - this.hcd.setMaxVersions(1); - this.hdt.addFamily(hcd); - if (admin.tableExists(hdt.getTableName())) { - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); + this.familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + this.familyDescriptor.setMobEnabled(true); + this.familyDescriptor.setMobThreshold(mobLen); + this.familyDescriptor.setMaxVersions(1); + this.tableDescriptor.setColumnFamily(familyDescriptor); + if (admin.tableExists(tableDescriptor.getTableName())) { + admin.disableTable(tableDescriptor.getTableName()); + admin.deleteTable(tableDescriptor.getTableName()); } - admin.createTable(hdt); - table = conn.getTable(hdt.getTableName()); + admin.createTable(tableDescriptor); + table = conn.getTable(tableDescriptor.getTableName()); } private void printConf() { @@ -150,7 +151,7 @@ public class MobStressToolRunner { public void run() { while (run) { try { - admin.majorCompact(hdt.getTableName(), fam); + admin.majorCompact(tableDescriptor.getTableName(), fam); Thread.sleep(120000); } catch (Exception e) { LOG.error("MOB Stress Test FAILED", e); @@ -251,8 +252,8 @@ public class MobStressToolRunner { } finally { - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); + admin.disableTable(tableDescriptor.getTableName()); + admin.deleteTable(tableDescriptor.getTableName()); } LOG.info("MOB Stress Test finished OK"); printStats(count); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java index a2d0b9901da..b2e08be353f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -22,14 +22,14 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; @@ -70,34 +70,41 @@ public class TestDefaultMobStoreFlusher { TEST_UTIL.shutdownMiniCluster(); } - @Test - public void testFlushNonMobFile() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor hcd = new HColumnDescriptor(family); - hcd.setMaxVersions(4); - desc.addFamily(hcd); + @Test + public void testFlushNonMobFile() throws Exception { + final TableName tableName = TableName.valueOf(name.getMethodName()); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); - testFlushFile(desc); - } + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + familyDescriptor.setMaxVersions(4); + tableDescriptor.setColumnFamily(familyDescriptor); - @Test - public void testFlushMobFile() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor hcd = new HColumnDescriptor(family); - hcd.setMobEnabled(true); - hcd.setMobThreshold(3L); - hcd.setMaxVersions(4); - desc.addFamily(hcd); + testFlushFile(tableDescriptor); + } - testFlushFile(desc); - } + @Test + public void testFlushMobFile() throws Exception { + final TableName tableName = TableName.valueOf(name.getMethodName()); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); - private void testFlushFile(HTableDescriptor htd) throws Exception { + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + hcd.setMobEnabled(true); + hcd.setMobThreshold(3L); + hcd.setMaxVersions(4); + tableDescriptor.setColumnFamily(hcd); + + testFlushFile(tableDescriptor); + } + + private void testFlushFile(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) + throws Exception { Table table = null; try { - table = TEST_UTIL.createTable(htd, null); + table = TEST_UTIL.createTable(tableDescriptor, null); //put data Put put0 = new Put(row1); @@ -110,7 +117,7 @@ public class TestDefaultMobStoreFlusher { table.put(put1); //flush - TEST_UTIL.flush(htd.getTableName()); + TEST_UTIL.flush(tableDescriptor.getTableName()); //Scan Scan scan = new Scan(); @@ -120,7 +127,7 @@ public class TestDefaultMobStoreFlusher { //Compare int size = 0; - for (Result result: scanner) { + for (Result result : scanner) { size++; List cells = result.getColumnCells(family, qf1); // Verify the cell size diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java index f58e7f76e9a..b530df3bfb8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -30,15 +30,17 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.CompactionState; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RegionSplitter; import org.junit.After; @@ -77,8 +79,8 @@ public abstract class TestMobCompactionBase { .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); protected Configuration conf; - protected HTableDescriptor hdt; - private HColumnDescriptor hcd; + protected TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; + private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor; protected Admin admin; protected Table table = null; protected long minAgeToArchive = 10000; @@ -94,7 +96,7 @@ public abstract class TestMobCompactionBase { @Before public void setUp() throws Exception { HTU = new HBaseTestingUtility(); - hdt = HTU.createTableDescriptor(TableName.valueOf(getClass().getName())); + tableDescriptor = HTU.createModifyableTableDescriptor(getClass().getName()); conf = HTU.getConfiguration(); initConf(); @@ -102,14 +104,14 @@ public abstract class TestMobCompactionBase { HTU.startMiniCluster(); admin = HTU.getAdmin(); cleanerChore = new MobFileCleanerChore(); - hcd = new HColumnDescriptor(fam); - hcd.setMobEnabled(true); - hcd.setMobThreshold(mobLen); - hcd.setMaxVersions(1); - hdt.addFamily(hcd); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(mobLen); + familyDescriptor.setMaxVersions(1); + tableDescriptor.setColumnFamily(familyDescriptor); RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); byte[][] splitKeys = splitAlgo.split(numRegions); - table = HTU.createTable(hdt, splitKeys); + table = HTU.createTable(tableDescriptor, splitKeys); } @@ -148,8 +150,8 @@ public abstract class TestMobCompactionBase { @After public void tearDown() throws Exception { - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); + admin.disableTable(tableDescriptor.getTableName()); + admin.deleteTable(tableDescriptor.getTableName()); HTU.shutdownMiniCluster(); } @@ -163,9 +165,9 @@ public abstract class TestMobCompactionBase { long num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); assertEquals(numRegions * 3, num); // Major MOB compact - mobCompact(admin, hdt, hcd); + mobCompact(admin, tableDescriptor, familyDescriptor); // wait until compaction is complete - while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) { + while (admin.getCompactionState(tableDescriptor.getTableName()) != CompactionState.NONE) { Thread.sleep(100); } @@ -188,8 +190,8 @@ public abstract class TestMobCompactionBase { } - protected abstract void mobCompact(Admin admin2, HTableDescriptor hdt2, HColumnDescriptor hcd2) - throws IOException, InterruptedException; + protected abstract void mobCompact(Admin admin2, TableDescriptor tableDescriptor, + ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException; protected long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java index d4f13f6cb75..414ca3e9cde 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java @@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.mob; import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.ClassRule; import org.junit.Test; @@ -74,10 +74,10 @@ public class TestMobCompactionOptMode extends TestMobCompactionBase{ } @Override - protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd) - throws IOException, InterruptedException { + protected void mobCompact(Admin admin, TableDescriptor tableDescriptor, + ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { // Major compact MOB table - admin.majorCompact(hdt.getTableName(), hcd.getName()); + admin.majorCompact(tableDescriptor.getTableName(), familyDescriptor.getName()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java index 9871b966711..45fecc15b1e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java @@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.mob; import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.Before; import org.junit.ClassRule; @@ -76,6 +76,13 @@ public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionBase{ conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); } + @Override + protected void mobCompact(Admin admin, TableDescriptor tableDescriptor, + ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { + // Major compact with batch mode enabled + compactionChore.performMajorCompactionInBatches(admin, tableDescriptor, familyDescriptor); + } + @Test public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { LOG.info("MOB compaction chore generational batch mode started"); @@ -84,11 +91,4 @@ public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionBase{ } - @Override - protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd) - throws IOException, InterruptedException { - // Major compact with batch mode enabled - compactionChore.performMajorCompactionInBatches(admin, hdt, hcd); - } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java index 9145655a23b..01b6804e950 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java @@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.mob; import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.ClassRule; import org.junit.Test; @@ -57,6 +57,13 @@ public class TestMobCompactionRegularMode extends TestMobCompactionBase{ public TestMobCompactionRegularMode() { } + @Override + protected void mobCompact(Admin admin, TableDescriptor tableDescriptor, + ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { + // Major compact MOB table + admin.majorCompact(tableDescriptor.getTableName(), familyDescriptor.getName()); + } + @Test public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { LOG.info("MOB compaction regular mode started"); @@ -65,11 +72,4 @@ public class TestMobCompactionRegularMode extends TestMobCompactionBase{ } - @Override - protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd) - throws IOException, InterruptedException { - // Major compact MOB table - admin.majorCompact(hdt.getTableName(), hcd.getName()); - } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java index de1fecd4689..819f0955219 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java @@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.mob; import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.Before; import org.junit.ClassRule; @@ -73,6 +73,13 @@ public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionBa conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize); } + @Override + protected void mobCompact(Admin admin, TableDescriptor tableDescriptor, + ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { + // Major compact with batch mode enabled + compactionChore.performMajorCompactionInBatches(admin, tableDescriptor, familyDescriptor); + } + @Test public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { LOG.info("MOB compaction chore regular batch mode started"); @@ -81,11 +88,4 @@ public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionBa } - @Override - protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd) - throws IOException, InterruptedException { - // Major compact with batch mode enabled - compactionChore.performMajorCompactionInBatches(admin, hdt, hcd); - } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java index 8aadf008d34..95038f5a90d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -20,14 +20,14 @@ package org.apache.hadoop.hbase.mob; import java.util.Random; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -52,8 +52,9 @@ public class TestMobDataBlockEncoding { protected final byte[] qf3 = Bytes.toBytes("qualifier3"); private static Table table; private static Admin admin; - private static HColumnDescriptor hcd; - private static HTableDescriptor desc; + private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor + columnFamilyDescriptor; + private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; private static Random random = new Random(); private static long defaultThreshold = 10; @@ -69,17 +70,19 @@ public class TestMobDataBlockEncoding { public void setUp(long threshold, String TN, DataBlockEncoding encoding) throws Exception { - desc = new HTableDescriptor(TableName.valueOf(TN)); - hcd = new HColumnDescriptor(family); - hcd.setMobEnabled(true); - hcd.setMobThreshold(threshold); - hcd.setMaxVersions(4); - hcd.setDataBlockEncoding(encoding); - desc.addFamily(hcd); + tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TN)); + columnFamilyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + columnFamilyDescriptor.setMobEnabled(true); + columnFamilyDescriptor.setMobThreshold(threshold); + columnFamilyDescriptor.setMaxVersions(4); + columnFamilyDescriptor.setDataBlockEncoding(encoding); + tableDescriptor.setColumnFamily(columnFamilyDescriptor); admin = TEST_UTIL.getAdmin(); - admin.createTable(desc); + admin.createTable(tableDescriptor); table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()) - .getTable(TableName.valueOf(TN)); + .getTable(TableName.valueOf(TN)); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java index 59de79b3ffa..274d045b775 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -29,15 +29,15 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.CompactionState; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -77,8 +77,8 @@ public class TestMobFileCleanerChore { .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); private Configuration conf; - private HTableDescriptor hdt; - private HColumnDescriptor hcd; + private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; + private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor; private Admin admin; private Table table = null; private MobFileCleanerChore chore; @@ -87,11 +87,10 @@ public class TestMobFileCleanerChore { public TestMobFileCleanerChore() { } - @Before public void setUp() throws Exception { HTU = new HBaseTestingUtility(); - hdt = HTU.createTableDescriptor(TableName.valueOf("testMobCompactTable")); + tableDescriptor = HTU.createModifyableTableDescriptor("testMobCompactTable"); conf = HTU.getConfiguration(); initConf(); @@ -99,12 +98,12 @@ public class TestMobFileCleanerChore { HTU.startMiniCluster(); admin = HTU.getAdmin(); chore = new MobFileCleanerChore(); - hcd = new HColumnDescriptor(fam); - hcd.setMobEnabled(true); - hcd.setMobThreshold(mobLen); - hcd.setMaxVersions(1); - hdt.addFamily(hcd); - table = HTU.createTable(hdt, null); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(mobLen); + familyDescriptor.setMaxVersions(1); + tableDescriptor.setColumnFamily(familyDescriptor); + table = HTU.createTable(tableDescriptor, null); } private void initConf() { @@ -146,8 +145,8 @@ public class TestMobFileCleanerChore { @After public void tearDown() throws Exception { - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); + admin.disableTable(tableDescriptor.getTableName()); + admin.deleteTable(tableDescriptor.getTableName()); HTU.shutdownMiniCluster(); } @@ -160,9 +159,9 @@ public class TestMobFileCleanerChore { long num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); assertEquals(3, num); // Major compact - admin.majorCompact(hdt.getTableName(), fam); + admin.majorCompact(tableDescriptor.getTableName(), fam); // wait until compaction is complete - while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) { + while (admin.getCompactionState(tableDescriptor.getTableName()) != CompactionState.NONE) { Thread.sleep(100); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java index 91faeee224b..da2b494817c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -40,11 +40,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; @@ -100,8 +97,8 @@ public class TestMobStoreCompaction { private Configuration conf = null; private HRegion region = null; - private HTableDescriptor htd = null; - private HColumnDescriptor hcd = null; + private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = null; + private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = null; private long mobCellThreshold = 1000; private FileSystem fs; @@ -116,18 +113,17 @@ public class TestMobStoreCompaction { HBaseTestingUtility UTIL = new HBaseTestingUtility(conf); compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3); - htd = UTIL.createTableDescriptor(TableName.valueOf(name.getMethodName()), - HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER, - HColumnDescriptor.DEFAULT_KEEP_DELETED); - hcd = new HColumnDescriptor(COLUMN_FAMILY); - hcd.setMobEnabled(true); - hcd.setMobThreshold(mobThreshold); - hcd.setMaxVersions(1); - htd.modifyFamily(hcd); + tableDescriptor = UTIL.createModifyableTableDescriptor(name.getMethodName()); + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_FAMILY); + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(mobThreshold); + familyDescriptor.setMaxVersions(1); + tableDescriptor.modifyColumnFamily(familyDescriptor); - RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build(); - region = HBaseTestingUtility - .createRegionAndWAL(regionInfo, UTIL.getDataTestDir(), conf, htd, new MobFileCache(conf)); + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build(); + region = HBaseTestingUtility.createRegionAndWAL(regionInfo, + UTIL.getDataTestDir(), conf, tableDescriptor, new MobFileCache(conf)); fs = FileSystem.get(conf); } @@ -234,7 +230,7 @@ public class TestMobStoreCompaction { byte[] dummyData = makeDummyData(600); Path hbaseRootDir = FSUtils.getRootDir(conf); - Path basedir = new Path(hbaseRootDir, htd.getNameAsString()); + Path basedir = new Path(hbaseRootDir, tableDescriptor.getTableName().getNameAsString()); List> hfiles = new ArrayList<>(1); for (int i = 0; i < compactionThreshold; i++) { Path hpath = new Path(basedir, "hfile" + i); @@ -300,7 +296,8 @@ public class TestMobStoreCompaction { } private int countMobFiles() throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), hcd.getNameAsString()); + Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableDescriptor.getTableName(), + familyDescriptor.getNameAsString()); if (fs.exists(mobDirPath)) { FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath); return files.length; @@ -310,7 +307,8 @@ public class TestMobStoreCompaction { private long countMobCellsInMetadata() throws IOException { long mobCellsCount = 0; - Path mobDirPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), hcd.getNameAsString()); + Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableDescriptor.getTableName(), + familyDescriptor.getNameAsString()); Configuration copyOfConf = new Configuration(conf); copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); CacheConfig cacheConfig = new CacheConfig(copyOfConf); @@ -411,8 +409,8 @@ public class TestMobStoreCompaction { continue; } files.add(fileName); - Path familyPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), - hcd.getNameAsString()); + Path familyPath = MobUtils.getMobFamilyPath(conf, tableDescriptor.getTableName(), + familyDescriptor.getNameAsString()); assertTrue(fs.exists(new Path(familyPath, fileName))); } } while (hasMore); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java index 1beeffe76de..2040a0d4d27 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,10 +28,9 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.ConnectionConfiguration; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; @@ -41,6 +40,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.hfile.CorruptHFileException; import org.apache.hadoop.hbase.io.hfile.TestHFile; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -73,8 +73,8 @@ public class TestMobStoreScanner { protected final byte[] qf3 = Bytes.toBytes("qualifier3"); private static Table table; private static Admin admin; - private static HColumnDescriptor hcd; - private static HTableDescriptor desc; + private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor; + private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; private static Random random = new Random(); private static long defaultThreshold = 10; private FileSystem fs; @@ -99,14 +99,14 @@ public class TestMobStoreScanner { public void setUp(long threshold, TableName tn) throws Exception { conf = TEST_UTIL.getConfiguration(); fs = FileSystem.get(conf); - desc = new HTableDescriptor(tn); - hcd = new HColumnDescriptor(family); - hcd.setMobEnabled(true); - hcd.setMobThreshold(threshold); - hcd.setMaxVersions(4); - desc.addFamily(hcd); + tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor(tn); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(threshold); + familyDescriptor.setMaxVersions(4); + tableDescriptor.setColumnFamily(familyDescriptor); admin = TEST_UTIL.getAdmin(); - admin.createTable(desc); + admin.createTable(tableDescriptor); table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()) .getTable(tn); } @@ -401,7 +401,7 @@ public class TestMobStoreScanner { // Get the files in the mob path Path mobFamilyPath; mobFamilyPath = MobUtils.getMobFamilyPath( - TEST_UTIL.getConfiguration(), tn, hcd.getNameAsString()); + TEST_UTIL.getConfiguration(), tn, familyDescriptor.getNameAsString()); FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); FileStatus[] files = fs.listStatus(mobFamilyPath); @@ -458,7 +458,7 @@ public class TestMobStoreScanner { int valLen = Bytes.toInt(referenceValue, 0, Bytes.SIZEOF_INT); Assert.assertEquals(value.length, valLen); Path mobFamilyPath = MobUtils.getMobFamilyPath( - TEST_UTIL.getConfiguration(), tn, hcd.getNameAsString()); + TEST_UTIL.getConfiguration(), tn, familyDescriptor.getNameAsString()); Path targetPath = new Path(mobFamilyPath, fileName); FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); Assert.assertTrue(fs.exists(targetPath)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java index b3cf3487826..9bd62c0e16f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -39,9 +39,7 @@ import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.StartMiniClusterOption; @@ -346,9 +344,11 @@ public class TestNamespaceAuditor { ADMIN.createNamespace(nspDesc); final TableName tableTwo = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table2"); byte[] columnFamily = Bytes.toBytes("info"); - HTableDescriptor tableDescOne = new HTableDescriptor(tableTwo); - tableDescOne.addFamily(new HColumnDescriptor(columnFamily)); - ADMIN.createTable(tableDescOne, Bytes.toBytes("0"), Bytes.toBytes("9"), initialRegions); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableTwo); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily)); + ADMIN.createTable(tableDescriptor, Bytes.toBytes("0"), Bytes.toBytes("9"), initialRegions); Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); try (Table table = connection.getTable(tableTwo)) { UTIL.loadNumericRows(table, Bytes.toBytes("info"), 1000, 1999); @@ -442,12 +442,15 @@ public class TestNamespaceAuditor { ADMIN.createNamespace(nspDesc); final TableName tableOne = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table1"); byte[] columnFamily = Bytes.toBytes("info"); - HTableDescriptor tableDescOne = new HTableDescriptor(tableOne); - tableDescOne.addFamily(new HColumnDescriptor(columnFamily)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableOne); + + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily)); MasterSyncObserver.throwExceptionInPreCreateTableAction = true; try { try { - ADMIN.createTable(tableDescOne); + ADMIN.createTable(tableDescriptor); fail("Table " + tableOne.toString() + "creation should fail."); } catch (Exception exp) { LOG.error(exp.toString(), exp); @@ -460,7 +463,7 @@ public class TestNamespaceAuditor { MasterSyncObserver.throwExceptionInPreCreateTableAction = false; try { - ADMIN.createTable(tableDescOne); + ADMIN.createTable(tableDescriptor); } catch (Exception e) { fail("Table " + tableOne.toString() + "creation should succeed."); LOG.error(e.toString(), e); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index 7f15dc50b82..b1fee844203 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -41,10 +41,8 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MultithreadedTestUtil; import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext; import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread; @@ -301,15 +299,18 @@ public class TestAtomicOperation { private void initHRegion (byte [] tableName, String callingMethod, int [] maxVersions, byte[] ... families) throws IOException { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); + int i=0; for(byte [] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family); - hcd.setMaxVersions(maxVersions != null ? maxVersions[i++] : 1); - htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + familyDescriptor.setMaxVersions(maxVersions != null ? maxVersions[i++] : 1); + tableDescriptor.setColumnFamily(familyDescriptor); } - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - region = TEST_UTIL.createLocalHRegion(info, htd); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); + region = TEST_UTIL.createLocalHRegion(info, tableDescriptor); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java index c7bc168b41c..4f845bc06a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,14 +28,14 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.BlockCache; @@ -77,32 +77,34 @@ public class TestBlocksScanned { @Test public void testBlocksScanned() throws Exception { byte [] tableName = Bytes.toBytes("TestBlocksScanned"); - HTableDescriptor table = new HTableDescriptor(TableName.valueOf(tableName)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); - table.addFamily( - new HColumnDescriptor(FAMILY) + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY) .setMaxVersions(10) .setBlockCacheEnabled(true) .setBlocksize(BLOCK_SIZE) .setCompressionType(Compression.Algorithm.NONE) ); - _testBlocksScanned(table); + _testBlocksScanned(tableDescriptor); } @Test public void testBlocksScannedWithEncoding() throws Exception { byte [] tableName = Bytes.toBytes("TestBlocksScannedWithEncoding"); - HTableDescriptor table = new HTableDescriptor(TableName.valueOf(tableName)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); - table.addFamily( - new HColumnDescriptor(FAMILY) + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY) .setMaxVersions(10) .setBlockCacheEnabled(true) .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF) .setBlocksize(BLOCK_SIZE) .setCompressionType(Compression.Algorithm.NONE) ); - _testBlocksScanned(table); + _testBlocksScanned(tableDescriptor); } private void _testBlocksScanned(TableDescriptor td) throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java index ed9b33e1cf8..dce0c0611ef 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -46,11 +46,11 @@ import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -243,16 +243,19 @@ public class TestBulkLoad { byte[]... families) throws IOException { HRegionInfo hRegionInfo = new HRegionInfo(tableName); - HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + for (byte[] family : families) { - hTableDescriptor.addFamily(new HColumnDescriptor(family)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); // TODO We need a way to do this without creating files return HRegion.createHRegion(hRegionInfo, new Path(testFolder.newFolder().toURI()), conf, - hTableDescriptor, + tableDescriptor, log); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java index 43c81a31dcc..1a4f5a04d9e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -30,9 +30,7 @@ import java.util.Set; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.TableName; @@ -72,14 +70,17 @@ public class TestColumnSeeking { byte[] familyBytes = Bytes.toBytes("Family"); TableName table = TableName.valueOf(name.getMethodName()); - HColumnDescriptor hcd = - new HColumnDescriptor(familyBytes).setMaxVersions(1000); - hcd.setMaxVersions(3); - HTableDescriptor htd = new HTableDescriptor(table); - htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyBytes) + .setMaxVersions(1000); + familyDescriptor.setMaxVersions(3); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(table); + + tableDescriptor.setColumnFamily(familyDescriptor); HRegionInfo info = new HRegionInfo(table, null, null, false); // Set this so that the archiver writes to the temp dir as well. - HRegion region = TEST_UTIL.createLocalHRegion(info, htd); + HRegion region = TEST_UTIL.createLocalHRegion(info, tableDescriptor); try { List rows = generateRandomWords(10, "row"); List allColumns = generateRandomWords(10, "column"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java index f7f0ca04535..e79d6df6726 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java @@ -25,11 +25,12 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -111,10 +112,12 @@ public class TestCompactSplitThread { Configuration conf = TEST_UTIL.getConfiguration(); Connection conn = ConnectionFactory.createConnection(conf); try { - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(family)); - htd.setCompactionEnabled(false); - TEST_UTIL.getAdmin().createTable(htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); + tableDescriptor.setCompactionEnabled(false); + TEST_UTIL.getAdmin().createTable(tableDescriptor); TEST_UTIL.waitTableAvailable(tableName); HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java index 80e312b1718..9224dc7071c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -35,16 +35,16 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.MemoryCompactionPolicy; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException; import org.apache.hadoop.hbase.io.util.MemorySizeUtil; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -108,17 +108,21 @@ public class TestCompactingMemStore extends TestDefaultMemStore { conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f); conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000); HBaseTestingUtility hbaseUtility = new HBaseTestingUtility(conf); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("foobar")); - htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf("foobar")); + + tableDescriptor.setColumnFamily(familyDescriptor); HRegionInfo info = new HRegionInfo(TableName.valueOf("foobar"), null, null, false); WAL wal = hbaseUtility.createWal(conf, hbaseUtility.getDataTestDir(), info); - this.region = HRegion.createHRegion(info, hbaseUtility.getDataTestDir(), conf, htd, wal, true); + this.region = HRegion.createHRegion(info, hbaseUtility.getDataTestDir(), conf, + tableDescriptor, wal, true); this.regionServicesForStores = Mockito.spy(region.getRegionServicesForStores()); ThreadPoolExecutor pool = (ThreadPoolExecutor) Executors.newFixedThreadPool(1); Mockito.when(regionServicesForStores.getInMemoryCompactionPool()).thenReturn(pool); - this.store = new HStore(region, hcd, conf, false); + this.store = new HStore(region, familyDescriptor, conf, false); long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage() .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index dfb3eb0ce3e..48c7609b843 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -48,14 +48,13 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; @@ -97,7 +96,7 @@ public class TestCompaction { protected Configuration conf = UTIL.getConfiguration(); private HRegion r = null; - private HTableDescriptor htd = null; + private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = null; private static final byte [] COLUMN_FAMILY = fam1; private final byte [] STARTROW = Bytes.toBytes(START_KEY); private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY; @@ -127,19 +126,18 @@ public class TestCompaction { @Before public void setUp() throws Exception { - this.htd = UTIL.createTableDescriptor(TableName.valueOf(name.getMethodName()), - HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER, - HColumnDescriptor.DEFAULT_KEEP_DELETED); + this.tableDescriptor = UTIL.createModifyableTableDescriptor(name.getMethodName()); if (name.getMethodName().equals("testCompactionSeqId")) { UTIL.getConfiguration().set("hbase.hstore.compaction.kv.max", "10"); UTIL.getConfiguration().set( DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY, DummyCompactor.class.getName()); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); - hcd.setMaxVersions(65536); - this.htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + familyDescriptor.setMaxVersions(65536); + this.tableDescriptor.setColumnFamily(familyDescriptor); } - this.r = UTIL.createLocalHRegion(htd, null, null); + this.r = UTIL.createLocalHRegion(tableDescriptor, null, null); } @After diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java index 90a959634c3..6a8123ff78f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -27,11 +27,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; @@ -90,24 +90,27 @@ public class TestCompactionPolicy { Path basedir = new Path(DIR); String logName = "logs"; Path logdir = new Path(DIR, logName); - HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family")); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("family")); FileSystem fs = FileSystem.get(conf); fs.delete(logdir, true); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("table"))); - htd.addFamily(hcd); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(Bytes.toBytes("table"))); + tableDescriptor.setColumnFamily(familyDescriptor); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); hlog = new FSHLog(fs, basedir, logName, conf); hlog.init(); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); - region = HRegion.createHRegion(info, basedir, conf, htd, hlog); + region = HRegion.createHRegion(info, basedir, conf, tableDescriptor, hlog); region.close(); - Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName()); - region = new HRegion(tableDir, hlog, fs, conf, info, htd, null); + Path tableDir = FSUtils.getTableDir(basedir, tableDescriptor.getTableName()); + region = new HRegion(tableDir, hlog, fs, conf, info, tableDescriptor, null); - store = new HStore(region, hcd, conf, false); + store = new HStore(region, familyDescriptor, conf, false); TEST_FILE = region.getRegionFileSystem().createTempName(); fs.createNewFile(TEST_FILE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java index b327668ca61..31173ecaf23 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java @@ -23,14 +23,16 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -85,19 +87,22 @@ public class TestDeleteMobTable { return mobVal; } - private HTableDescriptor createTableDescriptor(TableName tableName, boolean hasMob) { - HTableDescriptor htd = new HTableDescriptor(tableName); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); + private TableDescriptorBuilder.ModifyableTableDescriptor createTableDescriptor( + TableName tableName, boolean hasMob) { + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); if (hasMob) { - hcd.setMobEnabled(true); - hcd.setMobThreshold(0); + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(0); } - htd.addFamily(hcd); - return htd; + tableDescriptor.setColumnFamily(familyDescriptor); + return tableDescriptor; } - private Table createTableWithOneFile(HTableDescriptor htd) throws IOException { - Table table = TEST_UTIL.createTable(htd, null); + private Table createTableWithOneFile(TableDescriptor tableDescriptor) throws IOException { + Table table = TEST_UTIL.createTable(tableDescriptor, null); try { // insert data byte[] value = generateMobValue(10); @@ -107,7 +112,7 @@ public class TestDeleteMobTable { table.put(put); // create an hfile - TEST_UTIL.getAdmin().flush(htd.getTableName()); + TEST_UTIL.getAdmin().flush(tableDescriptor.getTableName()); } catch (IOException e) { table.close(); throw e; @@ -118,17 +123,17 @@ public class TestDeleteMobTable { @Test public void testDeleteMobTable() throws Exception { final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor htd = createTableDescriptor(tableName, true); - HColumnDescriptor hcd = htd.getFamily(FAMILY); + TableDescriptor tableDescriptor = createTableDescriptor(tableName, true); + ColumnFamilyDescriptor familyDescriptor = tableDescriptor.getColumnFamily(FAMILY); String fileName = null; - Table table = createTableWithOneFile(htd); + Table table = createTableWithOneFile(tableDescriptor); try { // the mob file exists - Assert.assertEquals(1, countMobFiles(tableName, hcd.getNameAsString())); - Assert.assertEquals(0, countArchiveMobFiles(tableName, hcd.getNameAsString())); - fileName = assertHasOneMobRow(table, tableName, hcd.getNameAsString()); - Assert.assertFalse(mobArchiveExist(tableName, hcd.getNameAsString(), fileName)); + Assert.assertEquals(1, countMobFiles(tableName, familyDescriptor.getNameAsString())); + Assert.assertEquals(0, countArchiveMobFiles(tableName, familyDescriptor.getNameAsString())); + fileName = assertHasOneMobRow(table, tableName, familyDescriptor.getNameAsString()); + Assert.assertFalse(mobArchiveExist(tableName, familyDescriptor.getNameAsString(), fileName)); Assert.assertTrue(mobTableDirExist(tableName)); } finally { table.close(); @@ -136,17 +141,17 @@ public class TestDeleteMobTable { } Assert.assertFalse(TEST_UTIL.getAdmin().tableExists(tableName)); - Assert.assertEquals(0, countMobFiles(tableName, hcd.getNameAsString())); - Assert.assertEquals(1, countArchiveMobFiles(tableName, hcd.getNameAsString())); - Assert.assertTrue(mobArchiveExist(tableName, hcd.getNameAsString(), fileName)); + Assert.assertEquals(0, countMobFiles(tableName, familyDescriptor.getNameAsString())); + Assert.assertEquals(1, countArchiveMobFiles(tableName, familyDescriptor.getNameAsString())); + Assert.assertTrue(mobArchiveExist(tableName, familyDescriptor.getNameAsString(), fileName)); Assert.assertFalse(mobTableDirExist(tableName)); } @Test public void testDeleteNonMobTable() throws Exception { final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor htd = createTableDescriptor(tableName, false); - HColumnDescriptor hcd = htd.getFamily(FAMILY); + TableDescriptor htd = createTableDescriptor(tableName, false); + ColumnFamilyDescriptor hcd = htd.getColumnFamily(FAMILY); Table table = createTableWithOneFile(htd); try { @@ -168,25 +173,28 @@ public class TestDeleteMobTable { @Test public void testMobFamilyDelete() throws Exception { final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor htd = createTableDescriptor(tableName, true); - HColumnDescriptor hcd = htd.getFamily(FAMILY); - htd.addFamily(new HColumnDescriptor(Bytes.toBytes("family2"))); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + createTableDescriptor(tableName, true); + ColumnFamilyDescriptor familyDescriptor = tableDescriptor.getColumnFamily(FAMILY); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + Bytes.toBytes("family2"))); - Table table = createTableWithOneFile(htd); + Table table = createTableWithOneFile(tableDescriptor); try { // the mob file exists - Assert.assertEquals(1, countMobFiles(tableName, hcd.getNameAsString())); - Assert.assertEquals(0, countArchiveMobFiles(tableName, hcd.getNameAsString())); - String fileName = assertHasOneMobRow(table, tableName, hcd.getNameAsString()); - Assert.assertFalse(mobArchiveExist(tableName, hcd.getNameAsString(), fileName)); + Assert.assertEquals(1, countMobFiles(tableName, familyDescriptor.getNameAsString())); + Assert.assertEquals(0, countArchiveMobFiles(tableName, familyDescriptor.getNameAsString())); + String fileName = assertHasOneMobRow(table, tableName, familyDescriptor.getNameAsString()); + Assert.assertFalse(mobArchiveExist(tableName, familyDescriptor.getNameAsString(), fileName)); Assert.assertTrue(mobTableDirExist(tableName)); TEST_UTIL.getAdmin().deleteColumnFamily(tableName, FAMILY); - Assert.assertEquals(0, countMobFiles(tableName, hcd.getNameAsString())); - Assert.assertEquals(1, countArchiveMobFiles(tableName, hcd.getNameAsString())); - Assert.assertTrue(mobArchiveExist(tableName, hcd.getNameAsString(), fileName)); - Assert.assertFalse(mobColumnFamilyDirExist(tableName, hcd.getNameAsString())); + Assert.assertEquals(0, countMobFiles(tableName, familyDescriptor.getNameAsString())); + Assert.assertEquals(1, countArchiveMobFiles(tableName, familyDescriptor.getNameAsString())); + Assert.assertTrue(mobArchiveExist(tableName, familyDescriptor.getNameAsString(), fileName)); + Assert.assertFalse(mobColumnFamilyDirExist(tableName, familyDescriptor.getNameAsString())); } finally { table.close(); TEST_UTIL.deleteTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java index a5668cfba42..4cafac520ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java @@ -34,14 +34,14 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PositionedReadable; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFileContext; @@ -201,12 +201,14 @@ public class TestFSErrorsExposed { byte[] fam = Bytes.toBytes("fam"); Admin admin = util.getAdmin(); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(fam) + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam) .setMaxVersions(1) .setBlockCacheEnabled(false) ); - admin.createTable(desc); + admin.createTable(tableDescriptor); // Make a new Configuration so it makes a new connection that has the // above configuration on it; else we use the old one w/ 10 as default. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 6b4dfcfc5a1..05962e56b62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -79,12 +79,10 @@ import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MultithreadedTestUtil; @@ -2806,15 +2804,18 @@ public class TestHRegion { byte[] value2 = Bytes.toBytes("value2"); final int maxVersions = 3; - HColumnDescriptor hcd = new HColumnDescriptor(fam1); - hcd.setMaxVersions(maxVersions); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testFilterAndColumnTracker")); - htd.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1); + familyDescriptor.setMaxVersions(maxVersions); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf("testFilterAndColumnTracker")); + tableDescriptor.setColumnFamily(familyDescriptor); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log"); final WAL wal = HBaseTestingUtility.createWal(TEST_UTIL.getConfiguration(), logDir, info); - this.region = TEST_UTIL.createLocalHRegion(info, htd, wal); + this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor, wal); // Put 4 version to memstore long ts = 0; @@ -4101,13 +4102,16 @@ public class TestHRegion { byte[] qf1 = Bytes.toBytes("col"); byte[] val1 = Bytes.toBytes("value1"); // Create Table - HColumnDescriptor hcd = new HColumnDescriptor(fam1).setMaxVersions(Integer.MAX_VALUE) + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1) + .setMaxVersions(Integer.MAX_VALUE) .setBloomFilterType(BloomType.ROWCOL); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(hcd); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - this.region = TEST_UTIL.createLocalHRegion(info, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily(familyDescriptor); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); + this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor); int num_unique_rows = 10; int duplicate_multiplier = 2; int num_storefiles = 4; @@ -4155,12 +4159,15 @@ public class TestHRegion { byte[] FAMILY = Bytes.toBytes("family"); // Create table - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY).setMaxVersions(Integer.MAX_VALUE) + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY) + .setMaxVersions(Integer.MAX_VALUE) .setBloomFilterType(BloomType.ROWCOL); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE)); - htd.addFamily(hcd); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - this.region = TEST_UTIL.createLocalHRegion(info, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TABLE)); + tableDescriptor.setColumnFamily(familyDescriptor); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); + this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor); // For row:0, col:0: insert versions 1 through 5. byte[] row = Bytes.toBytes("row:" + 0); byte[] column = Bytes.toBytes("column:" + 0); @@ -4197,13 +4204,16 @@ public class TestHRegion { byte[] familyName = Bytes.toBytes("familyName"); // Create Table - HColumnDescriptor hcd = new HColumnDescriptor(familyName).setMaxVersions(Integer.MAX_VALUE) + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName) + .setMaxVersions(Integer.MAX_VALUE) .setBloomFilterType(BloomType.ROWCOL); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(hcd); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - this.region = TEST_UTIL.createLocalHRegion(info, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily(familyDescriptor); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); + this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor); // Insert some data byte[] row = Bytes.toBytes("row1"); byte[] col = Bytes.toBytes("col1"); @@ -4743,16 +4753,19 @@ public class TestHRegion { Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3") }; byte[] cq = Bytes.toBytes("cq"); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); for (byte[] family : families) { - htd.addFamily(new HColumnDescriptor(family)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } long time = System.currentTimeMillis(); - HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(), + HRegionInfo primaryHri = new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, time, 0); - HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(), + HRegionInfo secondaryHri = new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, time, 1); @@ -4760,7 +4773,7 @@ public class TestHRegion { try { primaryRegion = HBaseTestingUtility.createRegionAndWAL(primaryHri, - rootDir, TEST_UTIL.getConfiguration(), htd); + rootDir, TEST_UTIL.getConfiguration(), tableDescriptor); // load some data putData(primaryRegion, 0, 1000, cq, families); @@ -4769,7 +4782,7 @@ public class TestHRegion { primaryRegion.flush(true); // open secondary region - secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF); + secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, tableDescriptor, null, CONF); verifyData(secondaryRegion, 0, 1000, cq, families); } finally { @@ -4793,16 +4806,19 @@ public class TestHRegion { Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3") }; byte[] cq = Bytes.toBytes("cq"); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); for (byte[] family : families) { - htd.addFamily(new HColumnDescriptor(family)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } long time = System.currentTimeMillis(); - HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(), + HRegionInfo primaryHri = new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, time, 0); - HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(), + HRegionInfo secondaryHri = new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, time, 1); @@ -4810,7 +4826,7 @@ public class TestHRegion { try { primaryRegion = HBaseTestingUtility.createRegionAndWAL(primaryHri, - rootDir, TEST_UTIL.getConfiguration(), htd); + rootDir, TEST_UTIL.getConfiguration(), tableDescriptor); // load some data putData(primaryRegion, 0, 1000, cq, families); @@ -4819,7 +4835,7 @@ public class TestHRegion { primaryRegion.flush(true); // open secondary region - secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF); + secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, tableDescriptor, null, CONF); try { putData(secondaryRegion, 0, 1000, cq, families); @@ -4852,16 +4868,19 @@ public class TestHRegion { Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3") }; byte[] cq = Bytes.toBytes("cq"); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); for (byte[] family : families) { - htd.addFamily(new HColumnDescriptor(family)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } long time = System.currentTimeMillis(); - HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(), + HRegionInfo primaryHri = new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, time, 0); - HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(), + HRegionInfo secondaryHri = new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, time, 1); @@ -4869,7 +4888,7 @@ public class TestHRegion { try { primaryRegion = HBaseTestingUtility.createRegionAndWAL(primaryHri, - rootDir, TEST_UTIL.getConfiguration(), htd); + rootDir, TEST_UTIL.getConfiguration(), tableDescriptor); // load some data putData(primaryRegion, 0, 1000, cq, families); @@ -4878,7 +4897,7 @@ public class TestHRegion { primaryRegion.flush(true); // open secondary region - secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF); + secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, tableDescriptor, null, CONF); // move the file of the primary region to the archive, simulating a compaction Collection storeFiles = primaryRegion.getStore(families[0]).getStorefiles(); @@ -5886,11 +5905,14 @@ public class TestHRegion { @Test public void testFlushedFileWithNoTags() throws Exception { final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(fam1)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1)); HRegionInfo info = new HRegionInfo(tableName, null, null, false); Path path = TEST_UTIL.getDataTestDir(getClass().getSimpleName()); - region = HBaseTestingUtility.createRegionAndWAL(info, path, TEST_UTIL.getConfiguration(), htd); + region = HBaseTestingUtility.createRegionAndWAL(info, path, + TEST_UTIL.getConfiguration(), tableDescriptor); Put put = new Put(Bytes.toBytes("a-b-0-0")); put.addColumn(fam1, qual1, Bytes.toBytes("c1-value")); region.put(put); @@ -6056,17 +6078,20 @@ public class TestHRegion { final byte[] q3 = Bytes.toBytes("q3"); final byte[] q4 = Bytes.toBytes("q4"); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - HColumnDescriptor hcd = new HColumnDescriptor(fam1); - hcd.setTimeToLive(10); // 10 seconds - htd.addFamily(hcd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1); + familyDescriptor.setTimeToLive(10); // 10 seconds + tableDescriptor.setColumnFamily(familyDescriptor); Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS); - region = HBaseTestingUtility.createRegionAndWAL(new HRegionInfo(htd.getTableName(), + region = HBaseTestingUtility.createRegionAndWAL(new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY), - TEST_UTIL.getDataTestDir(), conf, htd); + TEST_UTIL.getDataTestDir(), conf, tableDescriptor); assertNotNull(region); long now = EnvironmentEdgeManager.currentTime(); // Add a cell that will expire in 5 seconds via cell TTL @@ -6477,12 +6502,15 @@ public class TestHRegion { final ServerName serverName = ServerName.valueOf(name.getMethodName(), 100, 42); final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName)); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - htd.addFamily(new HColumnDescriptor(fam1)); - HRegionInfo hri = new HRegionInfo(htd.getTableName(), + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1)); + HRegionInfo hri = new HRegionInfo(tableDescriptor.getTableName(), HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY); - region = HRegion.openHRegion(hri, htd, rss.getWAL(hri), TEST_UTIL.getConfiguration(), - rss, null); + region = HRegion.openHRegion(hri, tableDescriptor, rss.getWAL(hri), + TEST_UTIL.getConfiguration(), rss, null); assertTrue(region.conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, false)); String plugins = region.conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ""); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java index 4ce298fc5a7..7685815c4e7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java @@ -26,11 +26,10 @@ import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionLocator; @@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -81,10 +81,12 @@ public class TestHRegionOnCluster { HMaster master = cluster.getMaster(); // Create table - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)); hbaseAdmin = master.getConnection().getAdmin(); - hbaseAdmin.createTable(desc); + hbaseAdmin.createTable(tableDescriptor); assertTrue(hbaseAdmin.isTableAvailable(tableName)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java index a34daa6b1aa..d11e0469f70 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java @@ -26,8 +26,6 @@ import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.StartMiniClusterOption; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -113,13 +111,15 @@ public class TestJoinedScanners { byte[][] families = {cf_essential, cf_joined}; final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); for (byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family); - hcd.setDataBlockEncoding(blockEncoding); - desc.addFamily(hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family); + familyDescriptor.setDataBlockEncoding(blockEncoding); + tableDescriptor.setColumnFamily(familyDescriptor); } - TEST_UTIL.getAdmin().createTable(desc); + TEST_UTIL.getAdmin().createTable(tableDescriptor); Table ht = TEST_UTIL.getConnection().getTable(tableName); long rows_to_insert = 1000; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMutateRowsRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMutateRowsRecovery.java index 328de3261e5..ebea0af8cfa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMutateRowsRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMutateRowsRecovery.java @@ -24,11 +24,10 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; @@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -101,9 +101,11 @@ public class TestMutateRowsRecovery { try { admin = connection.getAdmin(); hTable = connection.getTable(tableName); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(fam1)); - admin.createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1)); + admin.createTable(tableDescriptor); // Add a multi RowMutations rm = new RowMutations(row1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java index 10c745ea000..11b29acb4f1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestNewVersionBehaviorFromClientSide.java @@ -23,9 +23,8 @@ import static org.junit.Assert.assertNull; import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -77,12 +77,14 @@ public class TestNewVersionBehaviorFromClientSide { private Table createTable() throws IOException { TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor table = new HTableDescriptor(tableName); - HColumnDescriptor fam = new HColumnDescriptor(FAMILY); - fam.setNewVersionBehavior(true); - fam.setMaxVersions(3); - table.addFamily(fam); - TEST_UTIL.getAdmin().createTable(table); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + familyDescriptor.setNewVersionBehavior(true); + familyDescriptor.setMaxVersions(3); + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); return TEST_UTIL.getConnection().getTable(tableName); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java index 6a79267eb24..ea3d431b568 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java @@ -26,15 +26,15 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -180,12 +180,14 @@ public class TestParallelPut { private HRegion initHRegion(byte [] tableName, String callingMethod, byte[] ... families) throws IOException { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName)); for(byte [] family : families) { - htd.addFamily(new HColumnDescriptor(family)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - return HBTU.createLocalHRegion(info, htd); + HRegionInfo info = new HRegionInfo(tableDescriptor.getTableName(), null, null, false); + return HBTU.createLocalHRegion(info, tableDescriptor); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java index 2fa50ee7017..05e72e6d2e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java @@ -31,7 +31,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -40,12 +39,14 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -89,13 +90,15 @@ public class TestPerColumnFamilyFlush { public static final byte[] FAMILY3 = FAMILIES[2]; private HRegion initHRegion(String callingMethod, Configuration conf) throws IOException { - HTableDescriptor htd = new HTableDescriptor(TABLENAME); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLENAME); for (byte[] family : FAMILIES) { - htd.addFamily(new HColumnDescriptor(family)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); } HRegionInfo info = new HRegionInfo(TABLENAME, null, null, false); Path path = new Path(DIR, callingMethod); - return HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + return HBaseTestingUtility.createRegionAndWAL(info, path, conf, tableDescriptor); } // A helper function to create puts. @@ -558,11 +561,15 @@ public class TestPerColumnFamilyFlush { conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, ConstantSizeRegionSplitPolicy.class.getName()); - HTableDescriptor htd = new HTableDescriptor(TABLENAME); - htd.setCompactionEnabled(false); - htd.addFamily(new HColumnDescriptor(FAMILY1)); - htd.addFamily(new HColumnDescriptor(FAMILY2)); - htd.addFamily(new HColumnDescriptor(FAMILY3)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLENAME); + tableDescriptor.setCompactionEnabled(false); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY1)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY2)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY3)); LOG.info("==============Test with selective flush disabled==============="); int cf1StoreFileCount = -1; @@ -575,7 +582,7 @@ public class TestPerColumnFamilyFlush { TEST_UTIL.startMiniCluster(1); TEST_UTIL.getAdmin().createNamespace( NamespaceDescriptor.create(TABLENAME.getNamespaceAsString()).build()); - TEST_UTIL.getAdmin().createTable(htd); + TEST_UTIL.getAdmin().createTable(tableDescriptor); TEST_UTIL.waitTableAvailable(TABLENAME); Connection conn = ConnectionFactory.createConnection(conf); Table table = conn.getTable(TABLENAME); @@ -599,7 +606,7 @@ public class TestPerColumnFamilyFlush { TEST_UTIL.startMiniCluster(1); TEST_UTIL.getAdmin().createNamespace( NamespaceDescriptor.create(TABLENAME.getNamespaceAsString()).build()); - TEST_UTIL.getAdmin().createTable(htd); + TEST_UTIL.getAdmin().createTable(tableDescriptor); Connection conn = ConnectionFactory.createConnection(conf); Table table = conn.getTable(TABLENAME); doPut(table, memstoreFlushSize); @@ -629,12 +636,17 @@ public class TestPerColumnFamilyFlush { int numRegions = Integer.parseInt(args[0]); long numRows = Long.parseLong(args[1]); - HTableDescriptor htd = new HTableDescriptor(TABLENAME); - htd.setMaxFileSize(10L * 1024 * 1024 * 1024); - htd.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName()); - htd.addFamily(new HColumnDescriptor(FAMILY1)); - htd.addFamily(new HColumnDescriptor(FAMILY2)); - htd.addFamily(new HColumnDescriptor(FAMILY3)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLENAME); + tableDescriptor.setMaxFileSize(10L * 1024 * 1024 * 1024); + tableDescriptor.setValue(HTableDescriptor.SPLIT_POLICY, + ConstantSizeRegionSplitPolicy.class.getName()); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY1)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY2)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY3)); Configuration conf = HBaseConfiguration.create(); Connection conn = ConnectionFactory.createConnection(conf); @@ -647,9 +659,9 @@ public class TestPerColumnFamilyFlush { byte[] startKey = new byte[16]; byte[] endKey = new byte[16]; Arrays.fill(endKey, (byte) 0xFF); - admin.createTable(htd, startKey, endKey, numRegions); + admin.createTable(tableDescriptor, startKey, endKey, numRegions); } else { - admin.createTable(htd); + admin.createTable(tableDescriptor); } admin.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java index 7d525086d81..fcf372c6722 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java @@ -28,15 +28,15 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -88,12 +88,15 @@ public class TestRegionOpen { .getExecutorThreadPool(ExecutorType.RS_OPEN_PRIORITY_REGION); long completed = exec.getCompletedTaskCount(); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.setPriority(HConstants.HIGH_QOS); - htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setPriority(HConstants.HIGH_QOS); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + HConstants.CATALOG_FAMILY)); try (Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration()); Admin admin = connection.getAdmin()) { - admin.createTable(htd); + admin.createTable(tableDescriptor); } assertEquals(completed + 1, exec.getCompletedTaskCount()); @@ -108,8 +111,10 @@ public class TestRegionOpen { Configuration conf = HTU.getConfiguration(); Path rootDir = HTU.getDataTestDirOnTestFS(); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(FAMILYNAME)); + TableDescriptorBuilder.ModifyableTableDescriptor htd = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + htd.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILYNAME)); admin.createTable(htd); HTU.waitUntilNoRegionsInTransition(60000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java index 06331ff1694..cfcb93f46ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java @@ -28,14 +28,14 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -73,10 +73,14 @@ public class TestResettingCounters { byte [][] rows = new byte [numRows][]; for (int i=0; i regions = new ArrayList<>(1); regions.add(region); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java index 812ee4b416e..3c0f06c87da 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java @@ -38,10 +38,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -97,11 +97,13 @@ public class TestDateTieredCompactor { conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders); final Scanner scanner = new Scanner(input); // Create store mock that is satisfactory for compactor. - HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS); - ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparatorImpl.COMPARATOR); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(NAME_OF_THINGS); + ScanInfo si = new ScanInfo(conf, familyDescriptor, Long.MAX_VALUE, 0, + CellComparatorImpl.COMPARATOR); HStore store = mock(HStore.class); when(store.getStorefiles()).thenReturn(storefiles); - when(store.getColumnFamilyDescriptor()).thenReturn(col); + when(store.getColumnFamilyDescriptor()).thenReturn(familyDescriptor); when(store.getScanInfo()).thenReturn(si); when(store.areWritesEnabled()).thenReturn(true); when(store.getFileSystem()).thenReturn(mock(FileSystem.class)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java index 1a736f4ffab..c97e814793c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java @@ -48,9 +48,9 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HStore; @@ -782,12 +782,13 @@ public class TestStripeCompactionPolicy { } private StripeCompactor createCompactor() throws Exception { - HColumnDescriptor col = new HColumnDescriptor(Bytes.toBytes("foo")); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("foo")); StoreFileWritersCapture writers = new StoreFileWritersCapture(); HStore store = mock(HStore.class); HRegionInfo info = mock(HRegionInfo.class); when(info.getRegionNameAsString()).thenReturn("testRegion"); - when(store.getColumnFamilyDescriptor()).thenReturn(col); + when(store.getColumnFamilyDescriptor()).thenReturn(familyDescriptor); when(store.getRegionInfo()).thenReturn(info); when( store.createWriterInTmp(anyLong(), any(), anyBoolean(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java index 02212743a78..27ed074404e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java @@ -37,10 +37,10 @@ import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -198,10 +198,12 @@ public class TestStripeCompactor { final Scanner scanner = new Scanner(input); // Create store mock that is satisfactory for compactor. - HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS); - ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparatorImpl.COMPARATOR); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(NAME_OF_THINGS); + ScanInfo si = new ScanInfo(conf, familyDescriptor, Long.MAX_VALUE, 0, + CellComparatorImpl.COMPARATOR); HStore store = mock(HStore.class); - when(store.getColumnFamilyDescriptor()).thenReturn(col); + when(store.getColumnFamilyDescriptor()).thenReturn(familyDescriptor); when(store.getScanInfo()).thenReturn(si); when(store.areWritesEnabled()).thenReturn(true); when(store.getFileSystem()).thenReturn(mock(FileSystem.class)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java index 44214537c0d..67789995caf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java @@ -60,6 +60,8 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -67,6 +69,8 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher; @@ -278,8 +282,9 @@ public abstract class AbstractTestWALReplay { Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName); deleteDir(basedir); - HTableDescriptor htd = createBasic3FamilyHTD(tableName); - Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); + TableDescriptor tableDescriptor = createBasic3FamilyHTD(tableName); + Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, + tableDescriptor); HBaseTestingUtility.closeRegionAndWAL(region2); final byte [] rowName = tableName.getName(); @@ -288,28 +293,29 @@ public abstract class AbstractTestWALReplay { final int countPerFamily = 1000; NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + for(byte[] fam : tableDescriptor.getColumnFamilyNames()) { scopes.put(fam, 0); } - for (HColumnDescriptor hcd: htd.getFamilies()) { - addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, - wal1, htd, mvcc, scopes); + for (ColumnFamilyDescriptor familyDescriptor: tableDescriptor.getColumnFamilies()) { + addWALEdits(tableName, hri, rowName, familyDescriptor.getName(), countPerFamily, ee, + wal1, mvcc, scopes); } wal1.shutdown(); runWALSplit(this.conf); WAL wal2 = createWAL(this.conf, hbaseRootDir, logName); // Add 1k to each family. - for (HColumnDescriptor hcd: htd.getFamilies()) { - addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, - ee, wal2, htd, mvcc, scopes); + for (ColumnFamilyDescriptor familyDescriptor: tableDescriptor.getColumnFamilies()) { + addWALEdits(tableName, hri, rowName, familyDescriptor.getName(), countPerFamily, + ee, wal2, mvcc, scopes); } wal2.shutdown(); runWALSplit(this.conf); WAL wal3 = createWAL(this.conf, hbaseRootDir, logName); try { - HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal3); + HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, + tableDescriptor, wal3); long seqid = region.getOpenSeqNum(); // The regions opens with sequenceId as 1. With 6k edits, its sequence number reaches 6k + 1. // When opened, this region would apply 6k edits, and increment the sequenceId by 1 @@ -343,7 +349,7 @@ public abstract class AbstractTestWALReplay { final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName); final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString()); deleteDir(basedir); - final HTableDescriptor htd = createBasic3FamilyHTD(tableName); + final HTableDescriptor htd = new HTableDescriptor(createBasic3FamilyHTD(tableName)); Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region2); WAL wal = createWAL(this.conf, hbaseRootDir, logName); @@ -409,7 +415,7 @@ public abstract class AbstractTestWALReplay { final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName); final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString()); deleteDir(basedir); - final HTableDescriptor htd = createBasic3FamilyHTD(tableName); + final HTableDescriptor htd = new HTableDescriptor(createBasic3FamilyHTD(tableName)); HRegion region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region2); WAL wal = createWAL(this.conf, hbaseRootDir, logName); @@ -481,7 +487,7 @@ public abstract class AbstractTestWALReplay { deleteDir(basedir); final byte[] rowName = tableName.getName(); final int countPerFamily = 10; - final HTableDescriptor htd = createBasic3FamilyHTD(tableName); + final HTableDescriptor htd = new HTableDescriptor(createBasic3FamilyHTD(tableName)); HRegion region3 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region3); // Write countPerFamily edits into the three families. Do a flush on one @@ -589,7 +595,7 @@ public abstract class AbstractTestWALReplay { deleteDir(basedir); final byte[] rowName = tableName.getName(); final int countPerFamily = 10; - final HTableDescriptor htd = createBasic3FamilyHTD(tableName); + final HTableDescriptor htd = new HTableDescriptor(createBasic3FamilyHTD(tableName)); HRegion region3 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region3); // Write countPerFamily edits into the three families. Do a flush on one @@ -672,7 +678,7 @@ public abstract class AbstractTestWALReplay { final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName); final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName); deleteDir(basedir); - final HTableDescriptor htd = createBasic3FamilyHTD(tableName); + final HTableDescriptor htd = new HTableDescriptor(createBasic3FamilyHTD(tableName)); HRegion region3 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region3); // Write countPerFamily edits into the three families. Do a flush on one @@ -772,7 +778,7 @@ public abstract class AbstractTestWALReplay { final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName); deleteDir(basedir); - final HTableDescriptor htd = createBasic3FamilyHTD(tableName); + final HTableDescriptor htd = new HTableDescriptor(createBasic3FamilyHTD(tableName)); HRegion region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region2); final WAL wal = createWAL(this.conf, hbaseRootDir, logName); @@ -788,7 +794,7 @@ public abstract class AbstractTestWALReplay { } for (HColumnDescriptor hcd: htd.getFamilies()) { addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, - ee, wal, htd, mvcc, scopes); + ee, wal, mvcc, scopes); familyNames.add(hcd.getName()); } @@ -1088,10 +1094,12 @@ public abstract class AbstractTestWALReplay { } private HTableDescriptor createBasic1FamilyHTD(final TableName tableName) { - HTableDescriptor htd = new HTableDescriptor(tableName); - HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a")); - htd.addFamily(a); - return htd; + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("a")); + tableDescriptor.setColumnFamily(familyDescriptor); + return new HTableDescriptor(tableDescriptor); } private MockWAL createMockWAL() throws IOException { @@ -1164,7 +1172,7 @@ public abstract class AbstractTestWALReplay { private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName, final byte[] family, final int count, EnvironmentEdge ee, final WAL wal, - final HTableDescriptor htd, final MultiVersionConcurrencyControl mvcc, + final MultiVersionConcurrencyControl mvcc, NavigableMap scopes) throws IOException { for (int j = 0; j < count; j++) { wal.appendData(hri, createWALKey(tableName, hri, mvcc, scopes), @@ -1212,15 +1220,20 @@ public abstract class AbstractTestWALReplay { return splits.get(0); } - private HTableDescriptor createBasic3FamilyHTD(final TableName tableName) { - HTableDescriptor htd = new HTableDescriptor(tableName); - HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a")); - htd.addFamily(a); - HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b")); - htd.addFamily(b); - HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c")); - htd.addFamily(c); - return htd; + private TableDescriptorBuilder.ModifyableTableDescriptor createBasic3FamilyHTD( + final TableName tableName) { + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor a = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("a")); + tableDescriptor.setColumnFamily(a); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor b = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("b")); + tableDescriptor.setColumnFamily(b); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor c = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("c")); + tableDescriptor.setColumnFamily(c); + return tableDescriptor; } private void writerWALFile(Path file, List entries) throws IOException, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index e7a1691f5ea..a7b9e157f3d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -30,18 +30,18 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; @@ -84,7 +84,7 @@ public class TestMultiSlaveReplication { private static final byte[] row3 = Bytes.toBytes("row3"); private static final byte[] noRepfamName = Bytes.toBytes("norep"); - private static HTableDescriptor table; + private static TableDescriptorBuilder.ModifyableTableDescriptor table; @BeforeClass public static void setUpBeforeClass() throws Exception { @@ -122,12 +122,14 @@ public class TestMultiSlaveReplication { utility3.setZkCluster(miniZK); new ZKWatcher(conf3, "cluster3", null, true); - table = new HTableDescriptor(tableName); - HColumnDescriptor fam = new HColumnDescriptor(famName); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - table.addFamily(fam); - fam = new HColumnDescriptor(noRepfamName); - table.addFamily(fam); + table = new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + table.setColumnFamily(familyDescriptor); + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(noRepfamName); + table.setColumnFamily(familyDescriptor); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java index bc90563216b..8fe187b2dc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java @@ -28,11 +28,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.testclassification.FlakeyTests; @@ -91,10 +91,10 @@ public class TestPerTableCFReplication { private static final byte[] noRepfamName = Bytes.toBytes("norep"); private static final byte[] val = Bytes.toBytes("myval"); - private static HTableDescriptor table; - private static HTableDescriptor tabA; - private static HTableDescriptor tabB; - private static HTableDescriptor tabC; + private static TableDescriptorBuilder.ModifyableTableDescriptor table; + private static TableDescriptorBuilder.ModifyableTableDescriptor tabA; + private static TableDescriptorBuilder.ModifyableTableDescriptor tabB; + private static TableDescriptorBuilder.ModifyableTableDescriptor tabC; @Rule public TestName name = new TestName(); @@ -133,45 +133,47 @@ public class TestPerTableCFReplication { utility3.setZkCluster(miniZK); new ZKWatcher(conf3, "cluster3", null, true); - table = new HTableDescriptor(tableName); - HColumnDescriptor fam = new HColumnDescriptor(famName); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - table.addFamily(fam); - fam = new HColumnDescriptor(noRepfamName); - table.addFamily(fam); + table = new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + table.setColumnFamily(familyDescriptor); + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(noRepfamName); + table.setColumnFamily(familyDescriptor); - tabA = new HTableDescriptor(tabAName); - fam = new HColumnDescriptor(f1Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabA.addFamily(fam); - fam = new HColumnDescriptor(f2Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabA.addFamily(fam); - fam = new HColumnDescriptor(f3Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabA.addFamily(fam); + tabA = new TableDescriptorBuilder.ModifyableTableDescriptor(tabAName); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f1Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabA.setColumnFamily(familyDescriptor); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f2Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabA.setColumnFamily(familyDescriptor); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f3Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabA.setColumnFamily(familyDescriptor); - tabB = new HTableDescriptor(tabBName); - fam = new HColumnDescriptor(f1Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabB.addFamily(fam); - fam = new HColumnDescriptor(f2Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabB.addFamily(fam); - fam = new HColumnDescriptor(f3Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabB.addFamily(fam); + tabB = new TableDescriptorBuilder.ModifyableTableDescriptor(tabBName); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f1Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabB.setColumnFamily(familyDescriptor); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f2Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabB.setColumnFamily(familyDescriptor); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f3Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabB.setColumnFamily(familyDescriptor); - tabC = new HTableDescriptor(tabCName); - fam = new HColumnDescriptor(f1Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabC.addFamily(fam); - fam = new HColumnDescriptor(f2Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabC.addFamily(fam); - fam = new HColumnDescriptor(f3Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabC.addFamily(fam); + tabC = new TableDescriptorBuilder.ModifyableTableDescriptor(tabCName); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f1Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabC.setColumnFamily(familyDescriptor); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f2Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabC.setColumnFamily(familyDescriptor); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(f3Name); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tabC.setColumnFamily(familyDescriptor); utility1.startMiniCluster(); utility2.startMiniCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java index 69d7aca1cad..5d1bafa57ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java @@ -31,15 +31,14 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; @@ -47,6 +46,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -144,18 +144,20 @@ public class TestReplicationWithTags { rpc.setClusterKey(utility2.getClusterKey()); replicationAdmin.addReplicationPeer("2", rpc); - HTableDescriptor table = new HTableDescriptor(TABLE_NAME); - HColumnDescriptor fam = new HColumnDescriptor(FAMILY); - fam.setMaxVersions(3); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - table.addFamily(fam); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY); + familyDescriptor.setMaxVersions(3); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tableDescriptor.setColumnFamily(familyDescriptor); try (Connection conn = ConnectionFactory.createConnection(conf1); Admin admin = conn.getAdmin()) { - admin.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); + admin.createTable(tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); } try (Connection conn = ConnectionFactory.createConnection(conf2); Admin admin = conn.getAdmin()) { - admin.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); + admin.createTable(tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); } htable1 = utility1.getConnection().getTable(TABLE_NAME); htable2 = utility2.getConnection().getTable(TABLE_NAME); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java index f86daf565d6..c07312a19df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java @@ -23,18 +23,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTestConst; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; @@ -124,12 +124,14 @@ public class TestGlobalReplicationThrottler { @Test public void testQuota() throws IOException { final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor table = new HTableDescriptor(tableName); - HColumnDescriptor fam = new HColumnDescriptor(famName); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - table.addFamily(fam); - utility1.getAdmin().createTable(table); - utility2.getAdmin().createTable(table); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tableDescriptor.setColumnFamily(familyDescriptor); + utility1.getAdmin().createTable(tableDescriptor); + utility2.getAdmin().createTable(tableDescriptor); Thread watcher = new Thread(()->{ Replication replication = (Replication)utility1.getMiniHBaseCluster() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 723a4765a8e..2142b5299f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; @@ -281,12 +280,14 @@ public class TestAccessController extends SecureTestUtil { } private static void setUpTableAndUserPermissions() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TEST_TABLE); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY); - hcd.setMaxVersions(100); - htd.addFamily(hcd); - htd.setOwner(USER_OWNER); - createTable(TEST_UTIL, htd, new byte[][] { Bytes.toBytes("s") }); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); + familyDescriptor.setMaxVersions(100); + tableDescriptor.setColumnFamily(familyDescriptor); + tableDescriptor.setOwner(USER_OWNER); + createTable(TEST_UTIL, tableDescriptor, new byte[][] { Bytes.toBytes("s") }); HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0); RegionCoprocessorHost rcpHost = region.getCoprocessorHost(); @@ -393,9 +394,13 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction createTable = new AccessTestAction() { @Override public Object run() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); - ACCESS_CONTROLLER.preCreateTable(ObserverContextImpl.createAndPrepare(CP_ENV), htd, null); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); + ACCESS_CONTROLLER.preCreateTable(ObserverContextImpl.createAndPrepare(CP_ENV), + tableDescriptor, null); return null; } }; @@ -1312,10 +1317,13 @@ public class TestAccessController extends SecureTestUtil { if (admin.tableExists(tableName)) { deleteTable(TEST_UTIL, tableName); } - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(family1)); - htd.addFamily(new HColumnDescriptor(family2)); - createTable(TEST_UTIL, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family1)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family2)); + createTable(TEST_UTIL, tableDescriptor); try { // create temp users User tblUser = @@ -1565,10 +1573,13 @@ public class TestAccessController extends SecureTestUtil { if (admin.tableExists(tableName)) { deleteTable(TEST_UTIL, tableName); } - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(family1)); - htd.addFamily(new HColumnDescriptor(family2)); - createTable(TEST_UTIL, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family1)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family2)); + createTable(TEST_UTIL, tableDescriptor); try { // create temp users @@ -1668,11 +1679,14 @@ public class TestAccessController extends SecureTestUtil { if (admin.tableExists(tableName)) { deleteTable(TEST_UTIL, tableName); } - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(family1)); - htd.addFamily(new HColumnDescriptor(family2)); - htd.setOwner(USER_OWNER); - createTable(TEST_UTIL, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family1)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family2)); + tableDescriptor.setOwner(USER_OWNER); + createTable(TEST_UTIL, tableDescriptor); try { List perms = admin.getUserPermissions(GetUserPermissionsRequest.newBuilder(tableName).build()); @@ -1729,8 +1743,8 @@ public class TestAccessController extends SecureTestUtil { admin.disableTable(tableName); User newOwner = User.createUserForTesting(conf, "new_owner", new String[] {}); - htd.setOwner(newOwner); - admin.modifyTable(htd); + tableDescriptor.setOwner(newOwner); + admin.modifyTable(tableDescriptor); perms = admin.getUserPermissions(GetUserPermissionsRequest.newBuilder(tableName).build()); UserPermission newOwnerperm = new UserPermission(newOwner.getName(), @@ -2119,9 +2133,11 @@ public class TestAccessController extends SecureTestUtil { MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster(); final Admin admin = TEST_UTIL.getAdmin(); - HTableDescriptor htd = new HTableDescriptor(TEST_TABLE2); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); - createTable(TEST_UTIL, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE2); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); + createTable(TEST_UTIL, tableDescriptor); // Starting a new RegionServer. JVMClusterUtil.RegionServerThread newRsThread = hbaseCluster @@ -2276,12 +2292,14 @@ public class TestAccessController extends SecureTestUtil { } private void createTestTable(TableName tname, byte[] cf) throws Exception { - HTableDescriptor htd = new HTableDescriptor(tname); - HColumnDescriptor hcd = new HColumnDescriptor(cf); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tname); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cf); hcd.setMaxVersions(100); - htd.addFamily(hcd); - htd.setOwner(USER_OWNER); - createTable(TEST_UTIL, htd, new byte[][] { Bytes.toBytes("s") }); + tableDescriptor.setColumnFamily(hcd); + tableDescriptor.setOwner(USER_OWNER); + createTable(TEST_UTIL, tableDescriptor, new byte[][] { Bytes.toBytes("s") }); } @Test @@ -2846,18 +2864,21 @@ public class TestAccessController extends SecureTestUtil { // create table in default ns Admin admin = TEST_UTIL.getAdmin(); - HTableDescriptor htd = new HTableDescriptor(table1); - htd.addFamily(new HColumnDescriptor(family)); - createTable(TEST_UTIL, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(table1); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); + createTable(TEST_UTIL, tableDescriptor); // creating the ns and table in it String ns = "testNamespace"; NamespaceDescriptor desc = NamespaceDescriptor.create(ns).build(); final TableName table2 = TableName.valueOf(ns, tableName); createNamespace(TEST_UTIL, desc); - htd = new HTableDescriptor(table2); - htd.addFamily(new HColumnDescriptor(family)); - createTable(TEST_UTIL, htd); + tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor(table2); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); + createTable(TEST_UTIL, tableDescriptor); // Verify that we can read sys-tables String aclTableName = PermissionStorage.ACL_TABLE_NAME.getNameAsString(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java index 411022c2b13..bd82780f734 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java @@ -28,14 +28,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; @@ -186,12 +186,14 @@ public class TestAccessController2 extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTableDescriptor desc = new HTableDescriptor(testTable.getTableName()); - desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(testTable.getTableName()); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), testUser)) { try (Admin admin = connection.getAdmin()) { - createTable(TEST_UTIL, admin, desc); + createTable(TEST_UTIL, admin, tableDescriptor); } } return null; @@ -219,12 +221,14 @@ public class TestAccessController2 extends SecureTestUtil { AccessTestAction createAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTableDescriptor desc = new HTableDescriptor(testTable.getTableName()); - desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(testTable.getTableName()); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { try (Admin admin = connection.getAdmin()) { - admin.createTable(desc); + admin.createTable(tableDescriptor); } } return null; @@ -507,9 +511,11 @@ public class TestAccessController2 extends SecureTestUtil { final TableName table = TableName.valueOf(ns, "testACLZNodeDeletionTable"); final byte[] family = Bytes.toBytes("f1"); - HTableDescriptor htd = new HTableDescriptor(table); - htd.addFamily(new HColumnDescriptor(family)); - createTable(TEST_UTIL, htd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(table); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); + createTable(TEST_UTIL, tableDescriptor); // Namespace needs this, as they follow the lazy creation of ACL znode. grantOnNamespace(TEST_UTIL, TESTGROUP1_USER1.getShortName(), ns, Action.ADMIN); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java index d4ae32f06e2..5f7a45b9e8a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java @@ -26,11 +26,11 @@ import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.ObserverContextImpl; @@ -206,12 +206,14 @@ public class TestAccessController3 extends SecureTestUtil { } private static void setUpTableAndUserPermissions() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TEST_TABLE); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY); - hcd.setMaxVersions(100); - htd.addFamily(hcd); - htd.setOwner(USER_OWNER); - createTable(TEST_UTIL, htd, new byte[][] { Bytes.toBytes("s") }); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); + familyDescriptor.setMaxVersions(100); + tableDescriptor.setColumnFamily(familyDescriptor); + tableDescriptor.setOwner(USER_OWNER); + createTable(TEST_UTIL, tableDescriptor, new byte[][] { Bytes.toBytes("s") }); HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0); RegionCoprocessorHost rcpHost = region.getCoprocessorHost(); @@ -281,9 +283,13 @@ public class TestAccessController3 extends SecureTestUtil { AccessTestAction createTable = new AccessTestAction() { @Override public Object run() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); - ACCESS_CONTROLLER.preCreateTable(ObserverContextImpl.createAndPrepare(CP_ENV), htd, null); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor( + TableName.valueOf(name.getMethodName())); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); + ACCESS_CONTROLLER.preCreateTable(ObserverContextImpl.createAndPrepare(CP_ENV), + tableDescriptor, null); return null; } }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java index 1bf5bcb0cac..96e60126b4c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java @@ -28,11 +28,10 @@ import org.apache.hadoop.hbase.AuthUtil; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.security.User; @@ -132,19 +132,22 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { @Before public void setUp() throws Exception { - HTableDescriptor htd = new HTableDescriptor(testTable.getTableName()); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY1); - hcd.setMaxVersions(4); - htd.setOwner(USER_OWNER); - htd.addFamily(hcd); - hcd = new HColumnDescriptor(TEST_FAMILY2); - hcd.setMaxVersions(4); - htd.setOwner(USER_OWNER); - htd.addFamily(hcd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(testTable.getTableName()); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY1); + familyDescriptor.setMaxVersions(4); + tableDescriptor.setOwner(USER_OWNER); + tableDescriptor.setColumnFamily(familyDescriptor); + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY2); + familyDescriptor.setMaxVersions(4); + tableDescriptor.setOwner(USER_OWNER); + tableDescriptor.setColumnFamily(familyDescriptor); // Create the test table (owner added to the _acl_ table) try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { try (Admin admin = connection.getAdmin()) { - admin.createTable(htd, new byte[][] { Bytes.toBytes("s") }); + admin.createTable(tableDescriptor, new byte[][] { Bytes.toBytes("s") }); } } TEST_UTIL.waitTableEnabled(testTable.getTableName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java index 44d29ac4f15..f15aa3d287b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java @@ -27,12 +27,11 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.security.User; @@ -135,12 +135,14 @@ public class TestCellACLs extends SecureTestUtil { public void setUp() throws Exception { // Create the test table (owner added to the _acl_ table) Admin admin = TEST_UTIL.getAdmin(); - HTableDescriptor htd = new HTableDescriptor(testTable.getTableName()); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY); - hcd.setMaxVersions(4); - htd.setOwner(USER_OWNER); - htd.addFamily(hcd); - admin.createTable(htd, new byte[][] { Bytes.toBytes("s") }); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(testTable.getTableName()); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); + familyDescriptor.setMaxVersions(4); + tableDescriptor.setOwner(USER_OWNER); + tableDescriptor.setColumnFamily(familyDescriptor); + admin.createTable(tableDescriptor, new byte[][] { Bytes.toBytes("s") }); TEST_UTIL.waitTableEnabled(testTable.getTableName()); LOG.info("Sleeping a second because of HBASE-12581"); Threads.sleep(1000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java index a13b189c73e..71c19293cab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import java.io.IOException; +import java.util.Collections; import java.util.Optional; import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; @@ -28,15 +29,17 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionObserver; @@ -265,17 +268,22 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil { // set retries low to raise exception quickly conf.setInt("hbase.client.retries.number", 5); UTIL.startMiniCluster(); - HTableDescriptor htd = new HTableDescriptor(TEST_TABLE); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY); - htd.addFamily(hcd); - htd.addCoprocessor("net.clayb.hbase.coprocessor.NotWhitelisted", - new Path("file:///notpermitted/couldnotpossiblyexist.jar"), - Coprocessor.PRIORITY_USER, null); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); + tableDescriptor.setColumnFamily(familyDescriptor); + tableDescriptor.setCoprocessor( + CoprocessorDescriptorBuilder.newBuilder("net.clayb.hbase.coprocessor.NotWhitelisted") + .setJarPath("file:///notpermitted/couldnotpossiblyexist.jar") + .setPriority(Coprocessor.PRIORITY_USER) + .setProperties(Collections.emptyMap()) + .build()); Connection connection = ConnectionFactory.createConnection(conf); Admin admin = connection.getAdmin(); LOG.info("Creating Table"); try { - admin.createTable(htd); + admin.createTable(tableDescriptor); fail("Expected coprocessor to raise IOException"); } catch (IOException e) { // swallow exception from coprocessor @@ -310,14 +318,16 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil { // set retries low to raise exception quickly conf.setInt("hbase.client.retries.number", 5); UTIL.startMiniCluster(); - HTableDescriptor htd = new HTableDescriptor(TEST_TABLE); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY); - htd.addFamily(hcd); - htd.addCoprocessor(TestRegionObserver.class.getName()); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); + tableDescriptor.setColumnFamily(familyDescriptor); + tableDescriptor.setCoprocessor(TestRegionObserver.class.getName()); Connection connection = ConnectionFactory.createConnection(conf); Admin admin = connection.getAdmin(); LOG.info("Creating Table"); - admin.createTable(htd); + admin.createTable(tableDescriptor); // ensure table was created and coprocessor is added to table LOG.info("Done Creating Table"); Table t = connection.getTable(TEST_TABLE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java index 752b4beccef..417c5f82405 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java @@ -29,17 +29,17 @@ import java.util.Objects; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.ObserverContextImpl; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService; @@ -511,9 +511,12 @@ public class TestNamespaceCommands extends SecureTestUtil { AccessTestAction createTable = new AccessTestAction() { @Override public Object run() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TEST_TABLE)); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); - ACCESS_CONTROLLER.preCreateTable(ObserverContextImpl.createAndPrepare(CP_ENV), htd, null); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); + ACCESS_CONTROLLER.preCreateTable(ObserverContextImpl.createAndPrepare(CP_ENV), + tableDescriptor, null); return null; } }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java index 8eb0605e345..7bcfdc43511 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java @@ -24,18 +24,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.security.User; @@ -115,20 +115,23 @@ public class TestScanEarlyTermination extends SecureTestUtil { @Before public void setUp() throws Exception { Admin admin = TEST_UTIL.getAdmin(); - HTableDescriptor htd = new HTableDescriptor(testTable.getTableName()); - htd.setOwner(USER_OWNER); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY1); - hcd.setMaxVersions(10); - htd.addFamily(hcd); - hcd = new HColumnDescriptor(TEST_FAMILY2); - hcd.setMaxVersions(10); - htd.addFamily(hcd); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(testTable.getTableName()); + tableDescriptor.setOwner(USER_OWNER); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY1); + familyDescriptor.setMaxVersions(10); + tableDescriptor.setColumnFamily(familyDescriptor); + familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY2); + familyDescriptor.setMaxVersions(10); + tableDescriptor.setColumnFamily(familyDescriptor); // Enable backwards compatible early termination behavior in the HTD. We // want to confirm that the per-table configuration is properly picked up. - htd.setConfiguration(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT, "true"); + tableDescriptor.setValue(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT, "true"); - admin.createTable(htd); + admin.createTable(tableDescriptor); TEST_UTIL.waitUntilAllRegionsAssigned(testTable.getTableName()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java index b967f4dd20d..62f186d0459 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.ObserverContextImpl; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; @@ -175,12 +176,14 @@ public class TestWithDisabledAuthorization extends SecureTestUtil { public void setUp() throws Exception { // Create the test table (owner added to the _acl_ table) Admin admin = TEST_UTIL.getAdmin(); - HTableDescriptor htd = new HTableDescriptor(testTable.getTableName()); - HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY); - hcd.setMaxVersions(100); - htd.addFamily(hcd); - htd.setOwner(USER_OWNER); - admin.createTable(htd, new byte[][] { Bytes.toBytes("s") }); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(testTable.getTableName()); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY); + familyDescriptor.setMaxVersions(100); + tableDescriptor.setColumnFamily(familyDescriptor); + tableDescriptor.setOwner(USER_OWNER); + admin.createTable(tableDescriptor, new byte[][] { Bytes.toBytes("s") }); TEST_UTIL.waitUntilAllRegionsAssigned(testTable.getTableName()); HRegion region = TEST_UTIL.getHBaseCluster().getRegions(testTable.getTableName()).get(0); @@ -470,10 +473,12 @@ public class TestWithDisabledAuthorization extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTableDescriptor htd = new HTableDescriptor(testTable.getTableName()); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); - ACCESS_CONTROLLER.preCreateTable(ObserverContextImpl.createAndPrepare(CP_ENV), htd, - null); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(testTable.getTableName()); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); + ACCESS_CONTROLLER.preCreateTable(ObserverContextImpl.createAndPrepare(CP_ENV), + tableDescriptor, null); return null; } }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE); @@ -482,13 +487,16 @@ public class TestWithDisabledAuthorization extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTableDescriptor htd = new HTableDescriptor(testTable.getTableName()); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); - htd.addFamily(new HColumnDescriptor(TEST_FAMILY2)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(testTable.getTableName()); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY)); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY2)); ACCESS_CONTROLLER.preModifyTable(ObserverContextImpl.createAndPrepare(CP_ENV), testTable.getTableName(), null, // not needed by AccessController - htd); + tableDescriptor); return null; } }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java index 0a13df06ced..49170a39969 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java @@ -21,16 +21,16 @@ import static org.junit.Assert.assertArrayEquals; import java.util.Arrays; import java.util.Collection; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.ipc.NettyRpcClient; import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -103,8 +103,10 @@ public class TestDelegationTokenWithEncryption extends SecureTestCluster { byte[] value = Bytes.toBytes("data"); try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { Admin admin = conn.getAdmin(); - HTableDescriptor tableDescriptor = new HTableDescriptor(new HTableDescriptor(tableName)); - tableDescriptor.addFamily(new HColumnDescriptor(family)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); admin.createTable(tableDescriptor); try (Table table = conn.getTable(tableName)) { table.put(new Put(row).addColumn(family, qualifier, value)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java index 5efc6cb559e..fdef0703727 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java @@ -31,16 +31,16 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; @@ -140,15 +140,17 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit rpc.setClusterKey(TEST_UTIL1.getClusterKey()); admin.addReplicationPeer("2", rpc); - HTableDescriptor table = new HTableDescriptor(TABLE_NAME); - HColumnDescriptor desc = new HColumnDescriptor(fam); - desc.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - table.addFamily(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tableDescriptor.setColumnFamily(familyDescriptor); try (Admin hBaseAdmin = TEST_UTIL.getAdmin()) { - hBaseAdmin.createTable(table); + hBaseAdmin.createTable(tableDescriptor); } try (Admin hBaseAdmin1 = TEST_UTIL1.getAdmin()){ - hBaseAdmin1.createTable(table); + hBaseAdmin1.createTable(tableDescriptor); } addLabels(); setAuths(conf); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java index 133bab194dd..d68f532ade8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java @@ -39,9 +39,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; @@ -677,9 +675,11 @@ public abstract class TestVisibilityLabels { } catch (Exception e) { } try { - HColumnDescriptor hcd = new HColumnDescriptor(VisibilityConstants.LABELS_TABLE_FAMILY); - hcd.setBloomFilterType(BloomType.ROWCOL); - admin.modifyColumnFamily(LABELS_TABLE_NAME, hcd); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor( + VisibilityConstants.LABELS_TABLE_FAMILY); + familyDescriptor.setBloomFilterType(BloomType.ROWCOL); + admin.modifyColumnFamily(LABELS_TABLE_NAME, familyDescriptor); fail("Lables table should not get altered by user."); } catch (Exception e) { } @@ -707,13 +707,17 @@ public abstract class TestVisibilityLabels { final byte[] fam2 = Bytes.toBytes("info2"); final byte[] qual2 = Bytes.toBytes("qual2"); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor col = new HColumnDescriptor(fam);// Default max versions is 1. - desc.addFamily(col); - col = new HColumnDescriptor(fam2); - col.setMaxVersions(5); - desc.addFamily(col); - TEST_UTIL.getAdmin().createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + // Default max versions is 1. + + tableDescriptor.setColumnFamily(familyDescriptor); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2); + familyDescriptor.setMaxVersions(5); + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put put = new Put(r1); put.addColumn(fam, qual, 3L, v1); @@ -794,10 +798,12 @@ public abstract class TestVisibilityLabels { public void testMutateRow() throws Exception { final byte[] qual2 = Bytes.toBytes("qual2"); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor col = new HColumnDescriptor(fam); - desc.addFamily(col); - TEST_UTIL.getAdmin().createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); try (Table table = TEST_UTIL.getConnection().getTable(tableName)){ Put p1 = new Put(row1); p1.addColumn(fam, qual, value); @@ -830,10 +836,12 @@ public abstract class TestVisibilityLabels { public void testFlushedFileWithVisibilityTags() throws Exception { final byte[] qual2 = Bytes.toBytes("qual2"); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor col = new HColumnDescriptor(fam); - desc.addFamily(col); - TEST_UTIL.getAdmin().createTable(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + tableDescriptor.setColumnFamily(familyDescriptor); + TEST_UTIL.getAdmin().createTable(tableDescriptor); try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put p1 = new Put(row1); p1.addColumn(fam, qual, value); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java index 52809b1e8ae..ec397c31d6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java @@ -36,9 +36,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.PrivateCellUtil; @@ -46,6 +44,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; @@ -55,6 +54,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -193,12 +193,14 @@ public class TestVisibilityLabelsReplication { admin.addReplicationPeer("2", rpc); Admin hBaseAdmin = TEST_UTIL.getAdmin(); - HTableDescriptor table = new HTableDescriptor(TABLE_NAME); - HColumnDescriptor desc = new HColumnDescriptor(fam); - desc.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - table.addFamily(desc); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + tableDescriptor.setColumnFamily(familyDescriptor); try { - hBaseAdmin.createTable(table); + hBaseAdmin.createTable(tableDescriptor); } finally { if (hBaseAdmin != null) { hBaseAdmin.close(); @@ -206,7 +208,7 @@ public class TestVisibilityLabelsReplication { } Admin hBaseAdmin1 = TEST_UTIL1.getAdmin(); try { - hBaseAdmin1.createTable(table); + hBaseAdmin1.createTable(tableDescriptor); } finally { if (hBaseAdmin1 != null) { hBaseAdmin1.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java index 153a892f114..d3defb11812 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java @@ -25,16 +25,16 @@ import java.security.PrivilegedExceptionAction; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -128,11 +128,13 @@ public class TestVisibilityWithCheckAuths { SUPERUSER.runAs(action); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Admin hBaseAdmin = TEST_UTIL.getAdmin(); - HColumnDescriptor colDesc = new HColumnDescriptor(fam); - colDesc.setMaxVersions(5); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(colDesc); - hBaseAdmin.createTable(desc); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + familyDescriptor.setMaxVersions(5); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setColumnFamily(familyDescriptor); + hBaseAdmin.createTable(tableDescriptor); try { TEST_UTIL.getAdmin().flush(tableName); PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java index 6166472287d..916f034106b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java @@ -28,13 +28,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -150,10 +151,10 @@ public class TestMiniClusterLoadSequential { assertEquals(numKeys, readerThreads.getNumKeysVerified()); } - protected void createPreSplitLoadTestTable(HTableDescriptor htd, HColumnDescriptor hcd) - throws IOException { - HBaseTestingUtility.createPreSplitLoadTestTable(conf, htd, hcd); - TEST_UTIL.waitUntilAllRegionsAssigned(htd.getTableName()); + protected void createPreSplitLoadTestTable(TableDescriptor tableDescriptor, + ColumnFamilyDescriptor familyDescriptor) throws IOException { + HBaseTestingUtility.createPreSplitLoadTestTable(conf, tableDescriptor, familyDescriptor); + TEST_UTIL.waitUntilAllRegionsAssigned(tableDescriptor.getTableName()); } protected void prepareForLoadTest() throws IOException { @@ -168,11 +169,13 @@ public class TestMiniClusterLoadSequential { } admin.close(); - HTableDescriptor htd = new HTableDescriptor(TABLE); - HColumnDescriptor hcd = new HColumnDescriptor(CF) + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(CF) .setCompressionType(compression) .setDataBlockEncoding(dataBlockEncoding); - createPreSplitLoadTestTable(htd, hcd); + createPreSplitLoadTestTable(tableDescriptor, familyDescriptor); LoadTestDataGenerator dataGen = new MultiThreadedAction.DefaultDataGenerator(CF); writerThreads = prepareWriterThreads(dataGen, conf, TABLE); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java index fe5e15775ac..d7a6945ab55 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; @@ -59,6 +60,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.ParseFilter; import org.apache.hadoop.hbase.filter.PrefixFilter; @@ -608,12 +610,14 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements Hb if (getAdmin().tableExists(tableName)) { throw new AlreadyExists("table name already in use"); } - HTableDescriptor desc = new HTableDescriptor(tableName); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); for (ColumnDescriptor col : columnFamilies) { - HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col); - desc.addFamily(colDesc); + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + ThriftUtilities.colDescFromThrift(col); + tableDescriptor.setColumnFamily(familyDescriptor); } - getAdmin().createTable(desc); + getAdmin().createTable(tableDescriptor); } catch (IOException e) { LOG.warn(e.getMessage(), e); throw getIOError(e); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java index 90f11adc35e..f3e743cceaa 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.compress.Compression; @@ -55,11 +56,11 @@ public final class ThriftUtilities { * Thrift ColumnDescriptor "struct". * * @param in Thrift ColumnDescriptor object - * @return HColumnDescriptor + * @return ModifyableColumnFamilyDescriptor * @throws IllegalArgument if the column name is empty */ - static public HColumnDescriptor colDescFromThrift(ColumnDescriptor in) - throws IllegalArgument { + static public ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor colDescFromThrift( + ColumnDescriptor in) throws IllegalArgument { Compression.Algorithm comp = Compression.getCompressionAlgorithmByName(in.compression.toLowerCase(Locale.ROOT)); BloomType bt = @@ -69,14 +70,15 @@ public final class ThriftUtilities { throw new IllegalArgument("column name is empty"); } byte [] parsedName = CellUtil.parseColumn(Bytes.getBytes(in.name))[0]; - HColumnDescriptor col = new HColumnDescriptor(parsedName) + ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(parsedName) .setMaxVersions(in.maxVersions) .setCompressionType(comp) .setInMemory(in.inMemory) .setBlockCacheEnabled(in.blockCacheEnabled) .setTimeToLive(in.timeToLive > 0 ? in.timeToLive : Integer.MAX_VALUE) .setBloomFilterType(bt); - return col; + return familyDescriptor; } /** diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java index 38b08de5200..b8d9269c5da 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java @@ -51,10 +51,9 @@ import org.apache.hadoop.hbase.CompatibilityFactory; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; @@ -64,6 +63,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; @@ -147,10 +147,13 @@ public class TestThriftHBaseServiceHandler { private static byte[] qualifierBname = Bytes.toBytes("qualifierB"); private static byte[] valueAname = Bytes.toBytes("valueA"); private static byte[] valueBname = Bytes.toBytes("valueB"); - private static HColumnDescriptor[] families = new HColumnDescriptor[] { - new HColumnDescriptor(familyAname).setMaxVersions(3), - new HColumnDescriptor(familyBname).setMaxVersions(2) - }; + private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor[] families = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor[]{ + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyAname) + .setMaxVersions(3), + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyBname) + .setMaxVersions(2) + }; private static final MetricsAssertHelper metricsHelper = @@ -190,9 +193,10 @@ public class TestThriftHBaseServiceHandler { public static void beforeClass() throws Exception { UTIL.getConfiguration().set("hbase.client.retries.number", "3"); UTIL.startMiniCluster(); - HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(tableAname)); - for (HColumnDescriptor family : families) { - tableDescriptor.addFamily(family); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableAname)); + for (ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor family : families) { + tableDescriptor.setColumnFamily(family); } try (Admin admin = UTIL.getAdmin()) { admin.createTable(tableDescriptor); @@ -1250,9 +1254,11 @@ public class TestThriftHBaseServiceHandler { byte[] col = Bytes.toBytes("c"); // create a table which will throw exceptions for requests TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor tableDesc = new HTableDescriptor(tableName); - tableDesc.addCoprocessor(ErrorThrowingGetObserver.class.getName()); - tableDesc.addFamily(new HColumnDescriptor(family)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDesc = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDesc.setCoprocessor(ErrorThrowingGetObserver.class.getName()); + tableDesc.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); Table table = UTIL.createTable(tableDesc, null); table.put(new Put(rowkey).addColumn(family, col, Bytes.toBytes("val1"))); @@ -1321,13 +1327,15 @@ public class TestThriftHBaseServiceHandler { byte[] col = Bytes.toBytes("c"); // create a table which will throw exceptions for requests TableName tableName = TableName.valueOf("testMetricsPrecision"); - HTableDescriptor tableDesc = new HTableDescriptor(tableName); - tableDesc.addCoprocessor(DelayingRegionObserver.class.getName()); - tableDesc.addFamily(new HColumnDescriptor(family)); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(tableName); + tableDescriptor.setCoprocessor(DelayingRegionObserver.class.getName()); + tableDescriptor.setColumnFamily( + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)); Table table = null; try { - table = UTIL.createTable(tableDesc, null); + table = UTIL.createTable(tableDescriptor, null); table.put(new Put(rowkey).addColumn(family, col, Bytes.toBytes("val1"))); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java index 135ad12aab2..9f92cdfde93 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java @@ -33,12 +33,12 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; @@ -92,9 +92,12 @@ public class TestThriftHBaseServiceHandlerWithLabels { private static byte[] qualifierBname = Bytes.toBytes("qualifierB"); private static byte[] valueAname = Bytes.toBytes("valueA"); private static byte[] valueBname = Bytes.toBytes("valueB"); - private static HColumnDescriptor[] families = new HColumnDescriptor[] { - new HColumnDescriptor(familyAname).setMaxVersions(3), - new HColumnDescriptor(familyBname).setMaxVersions(2) }; + private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor[] families = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor[]{ + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyAname) + .setMaxVersions(3), + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyBname) + .setMaxVersions(2)}; private final static String TOPSECRET = "topsecret"; private final static String PUBLIC = "public"; @@ -141,10 +144,10 @@ public class TestThriftHBaseServiceHandlerWithLabels { UTIL.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000); createLabels(); Admin admin = UTIL.getAdmin(); - HTableDescriptor tableDescriptor = new HTableDescriptor( - TableName.valueOf(tableAname)); - for (HColumnDescriptor family : families) { - tableDescriptor.addFamily(family); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableAname)); + for (ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor family : families) { + tableDescriptor.setColumnFamily(family); } admin.createTable(tableDescriptor); admin.close(); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithReadOnly.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithReadOnly.java index b338e0fe82a..ef9cae1a342 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithReadOnly.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithReadOnly.java @@ -31,10 +31,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -76,10 +76,13 @@ public class TestThriftHBaseServiceHandlerWithReadOnly { private static byte[] qualifierBname = Bytes.toBytes("qualifierB"); private static byte[] valueAname = Bytes.toBytes("valueA"); private static byte[] valueBname = Bytes.toBytes("valueB"); - private static HColumnDescriptor[] families = new HColumnDescriptor[] { - new HColumnDescriptor(familyAname).setMaxVersions(3), - new HColumnDescriptor(familyBname).setMaxVersions(2) - }; + private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor[] families = + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor[]{ + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyAname) + .setMaxVersions(3), + new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyBname) + .setMaxVersions(2) + }; @BeforeClass public static void beforeClass() throws Exception { @@ -87,9 +90,10 @@ public class TestThriftHBaseServiceHandlerWithReadOnly { UTIL.getConfiguration().set("hbase.client.retries.number", "3"); UTIL.startMiniCluster(); Admin admin = UTIL.getAdmin(); - HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(tableAname)); - for (HColumnDescriptor family : families) { - tableDescriptor.addFamily(family); + TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = + new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableAname)); + for (ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor family : families) { + tableDescriptor.setColumnFamily(family); } admin.createTable(tableDescriptor); admin.close();