diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index 5c01d936628..45a561af96c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -872,8 +872,21 @@ public class MetaTableAccessor { * @return Count or regions in table tableName * @throws IOException */ + @Deprecated public static int getRegionCount(final Configuration c, final String tableName) throws IOException { + return getRegionCount(c, TableName.valueOf(tableName)); + } + + /** + * Count regions in hbase:meta for passed table. + * @param c Configuration object + * @param tableName table name to count regions for + * @return Count or regions in table tableName + * @throws IOException + */ + public static int getRegionCount(final Configuration c, final TableName tableName) + throws IOException { HTable t = new HTable(c, tableName); try { return t.getRegionLocations().size(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 722b5c26839..5f2e5153412 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -128,7 +128,7 @@ public interface Admin extends Abortable, Closeable { /** * Method for getting the tableDescriptor * - * @param tableName as a byte [] + * @param tableName as a {@link TableName} * @return the tableDescriptor * @throws org.apache.hadoop.hbase.TableNotFoundException * @throws IOException if a remote or network exception occurs diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java index 98dca5999a7..b28c247acb7 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.coprocessor.example; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; @@ -60,7 +61,7 @@ public class TestRowCountEndpoint { RowCountEndpoint.class.getName()); TEST_UTIL.startMiniCluster(); - TEST_UTIL.createTable(TEST_TABLE, TEST_FAMILY); + TEST_UTIL.createTable(TableName.valueOf(TEST_TABLE), new byte[][]{TEST_FAMILY}); } // @Ignore @AfterClass diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java index b6c1f2e77cd..ffeab0e10b1 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java @@ -24,6 +24,7 @@ import java.util.Collection; import org.apache.commons.lang.math.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; /** @@ -43,7 +44,7 @@ public class RestartRsHoldingTableAction extends RestartActionBaseAction { HTable table = null; try { Configuration conf = context.getHBaseIntegrationTestingUtility().getConfiguration(); - table = new HTable(conf, tableName); + table = new HTable(conf, TableName.valueOf(tableName)); } catch (IOException e) { LOG.debug("Error creating HTable used to get list of region locations.", e); return; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java index b13d670d606..4b07f8ff8c2 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; @@ -129,10 +130,10 @@ public class IntegrationTestImportTsv implements Configurable, Tool { * Verify the data described by simple_tsv matches * simple_expected. */ - protected void doLoadIncrementalHFiles(Path hfiles, String tableName) + protected void doLoadIncrementalHFiles(Path hfiles, TableName tableName) throws Exception { - String[] args = { hfiles.toString(), tableName }; + String[] args = { hfiles.toString(), tableName.getNameAsString() }; LOG.info(format("Running LoadIncrememntalHFiles with args: %s", Arrays.asList(args))); assertEquals("Loading HFiles failed.", 0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args)); @@ -181,9 +182,10 @@ public class IntegrationTestImportTsv implements Configurable, Tool { @Test public void testGenerateAndLoad() throws Exception { LOG.info("Running test testGenerateAndLoad."); - String table = NAME + "-" + UUID.randomUUID(); + TableName table = TableName.valueOf(NAME + "-" + UUID.randomUUID()); String cf = "d"; - Path hfiles = new Path(util.getDataTestDirOnTestFS(table), "hfiles"); + Path hfiles = new Path( + util.getDataTestDirOnTestFS(table.getNameAsString()), "hfiles"); String[] args = { format("-D%s=%s", ImportTsv.BULK_OUTPUT_CONF_KEY, hfiles), @@ -192,11 +194,11 @@ public class IntegrationTestImportTsv implements Configurable, Tool { // configure the test harness to NOT delete the HFiles after they're // generated. We need those for doLoadIncrementalHFiles format("-D%s=false", TestImportTsv.DELETE_AFTER_LOAD_CONF), - table + table.getNameAsString() }; // run the job, complete the load. - util.createTable(table, cf); + util.createTable(table, new String[]{cf}); Tool t = TestImportTsv.doMROnTableTest(util, cf, simple_tsv, args); doLoadIncrementalHFiles(hfiles, table); @@ -205,7 +207,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool { // clean up after ourselves. util.deleteTable(table); - util.cleanupDataTestDirOnTestFS(table); + util.cleanupDataTestDirOnTestFS(table.getNameAsString()); LOG.info("testGenerateAndLoad completed successfully."); } 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 44f9e4ad8fa..5c9a9ad222b 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 @@ -179,7 +179,7 @@ public void cleanUpCluster() throws Exception { recordsToWrite = conf.getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT); String tableName = conf.get(TABLE_NAME_KEY, TABLE_NAME_DEFAULT); numBackReferencesPerRow = conf.getInt(NUM_BACKREFS_KEY, NUM_BACKREFS_DEFAULT); - table = new HTable(conf, tableName); + table = new HTable(conf, TableName.valueOf(tableName)); table.setWriteBufferSize(4*1024*1024); table.setAutoFlush(false, true); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java index d572839b48e..f88d959e17a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -122,8 +123,9 @@ implements Configurable { public void setConf(Configuration configuration) { this.conf = HBaseConfiguration.create(configuration); try { - this.table = new HTable(this.conf, - configuration.get(TableOutputFormat.OUTPUT_TABLE)); + TableName tableName = TableName.valueOf(configuration + .get(TableOutputFormat.OUTPUT_TABLE)); + this.table = new HTable(this.conf, tableName); } catch (IOException e) { LOG.error(e); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java index dbac5687546..c2453fa57e1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; @@ -401,8 +402,8 @@ public class Import extends Configured implements Tool { */ public static Job createSubmittableJob(Configuration conf, String[] args) throws IOException { - String tableName = args[0]; - conf.set(TABLE_NAME, tableName); + TableName tableName = TableName.valueOf(args[0]); + conf.set(TABLE_NAME, tableName.getNameAsString()); Path inputDir = new Path(args[1]); Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName)); job.setJarByClass(Importer.class); @@ -435,7 +436,7 @@ public class Import extends Configured implements Tool { // No reducers. Just write straight to table. Call initTableReducerJob // because it sets up the TableOutputFormat. job.setMapperClass(Importer.class); - TableMapReduceUtil.initTableReducerJob(tableName, null, job); + TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job); job.setNumReduceTasks(0); } return job; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java index 12a2d4caa13..a953c3ec33b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java @@ -414,9 +414,9 @@ public class ImportTsv extends Configured implements Tool { Class mapperClass = mapperClassName != null ? Class.forName(mapperClassName) : DEFAULT_MAPPER; - String tableName = args[0]; + TableName tableName = TableName.valueOf(args[0]); Path inputDir = new Path(args[1]); - String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName); + String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName.getNameAsString()); Job job = Job.getInstance(conf, jobName); job.setJarByClass(mapperClass); FileInputFormat.setInputPaths(job, inputDir); @@ -460,7 +460,8 @@ public class ImportTsv extends Configured implements Tool { } // No reducers. Just write straight to table. Call initTableReducerJob // to set up the TableOutputFormat. - TableMapReduceUtil.initTableReducerJob(tableName, null, job); + TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, + job); job.setNumReduceTasks(0); } @@ -470,9 +471,9 @@ public class ImportTsv extends Configured implements Tool { return job; } - private static void createTable(Admin admin, String tableName, String[] columns) + private static void createTable(Admin admin, TableName tableName, String[] columns) throws IOException { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); + HTableDescriptor htd = new HTableDescriptor(tableName); Set cfSet = new HashSet(); for (String aColumn : columns) { if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java index 40fb21ec589..c7fa29e20e5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; @@ -96,7 +97,7 @@ implements Configurable { @Override public void setConf(Configuration configuration) { this.conf = configuration; - String tableName = conf.get(INPUT_TABLE); + TableName tableName = TableName.valueOf(conf.get(INPUT_TABLE)); try { setHTable(new HTable(new Configuration(conf), tableName)); } catch (Exception e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java index 454bc28ff46..52b8e450a24 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java @@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Mutation; @@ -204,7 +205,7 @@ implements Configurable { if (zkClientPort != 0) { this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort); } - this.table = new HTable(this.conf, tableName); + this.table = new HTable(this.conf, TableName.valueOf(tableName)); this.table.setAutoFlush(false, true); LOG.info("Created table instance for " + tableName); } catch(IOException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java index 290cf8c8422..cf9dc56adb5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java @@ -247,7 +247,7 @@ public class WALPlayer extends Configured implements Tool { if (tables.length != 1) { throw new IOException("Exactly one table must be specified for the bulk export option"); } - HTable table = new HTable(conf, tables[0]); + HTable table = new HTable(conf, TableName.valueOf(tables[0])); job.setMapperClass(HLogKeyValueMapper.class); job.setReducerClass(KeyValueSortReducer.class); Path outputDir = new Path(hfileOutPath); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java index 8f6b217c1de..f425ba80c1c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java @@ -123,7 +123,8 @@ public class VerifyReplication extends Configured implements Tool { Configuration peerConf = HBaseConfiguration.create(conf); ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey); - Table replicatedTable = new HTable(peerConf, conf.get(NAME + ".tableName")); + TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName")); + Table replicatedTable = new HTable(peerConf, tableName); scan.setStartRow(value.getRow()); replicatedScanner = replicatedTable.getScanner(scan); return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java index 9b11291549f..d5cdb391ce4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java @@ -606,7 +606,7 @@ public final class Canary implements Tool { private void monitorRegionServers(Map> rsAndRMap) { String serverName = null; - String tableName = null; + TableName tableName = null; HRegionInfo region = null; Table table = null; Get get = null; @@ -620,7 +620,7 @@ public final class Canary implements Tool { // always get the first region region = entry.getValue().get(0); try { - tableName = region.getTable().getNameAsString(); + tableName = region.getTable(); table = new HTable(this.admin.getConfiguration(), tableName); startKey = region.getStartKey(); // Can't do a get on empty start row so do a Scan of first element if any instead. @@ -638,17 +638,18 @@ public final class Canary implements Tool { s.close(); stopWatch.stop(); } - this.getSink().publishReadTiming(tableName, serverName, stopWatch.getTime()); + this.getSink().publishReadTiming(tableName.getNameAsString(), + serverName, stopWatch.getTime()); } catch (TableNotFoundException tnfe) { // This is ignored because it doesn't imply that the regionserver is dead } catch (TableNotEnabledException tnee) { // This is considered a success since we got a response. LOG.debug("The targeted table was disabled. Assuming success."); } catch (DoNotRetryIOException dnrioe) { - this.getSink().publishReadFailure(tableName, serverName); + this.getSink().publishReadFailure(tableName.getNameAsString(), serverName); LOG.error(dnrioe); } catch (IOException e) { - this.getSink().publishReadFailure(tableName, serverName); + this.getSink().publishReadFailure(tableName.getNameAsString(), serverName); LOG.error(e); this.errorCode = ERROR_EXIT_CODE; } finally { 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 70ef0ba11aa..de530aad526 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 @@ -1071,9 +1071,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @return An HTable instance for the created table. * @throws IOException */ - public HTable createTable(String tableName, String family) + public HTable createTable(TableName tableName, String family) throws IOException{ - return createTable(TableName.valueOf(tableName), new String[]{family}); + return createTable(tableName, new String[]{family}); } /** @@ -2226,20 +2226,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @throws IOException * @throws InterruptedException */ - public HRegionServer getRSForFirstRegionInTable(byte[] tableName) - throws IOException, InterruptedException { - return getRSForFirstRegionInTable(TableName.valueOf(tableName)); - } - /** - * Tool to get the reference to the region server object that holds the - * region of the specified user table. - * It first searches for the meta rows that contain the region of the - * specified table, then gets the index of that RS, and finally retrieves - * the RS's reference. - * @param tableName user table to lookup in hbase:meta - * @return region server that holds it, null if the row doesn't exist - * @throws IOException - */ public HRegionServer getRSForFirstRegionInTable(TableName tableName) throws IOException, InterruptedException { List metaRows = getMetaTableRows(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java index f2ac593f806..59ddfd71586 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java @@ -39,7 +39,7 @@ public class TestFullLogReconstruction { private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private final static byte[] TABLE_NAME = Bytes.toBytes("tabletest"); + private final static TableName TABLE_NAME = TableName.valueOf("tabletest"); private final static byte[] FAMILY = Bytes.toBytes("family"); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java index 4db8f308c27..7be5074dc25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java @@ -73,7 +73,7 @@ public class TestGlobalMemStoreSize { byte [] table = Bytes.toBytes("TestGlobalMemStoreSize"); byte [] family = Bytes.toBytes("family"); LOG.info("Creating table with " + regionNum + " regions"); - HTable ht = TEST_UTIL.createTable(table, family); + HTable ht = TEST_UTIL.createTable(TableName.valueOf(table), family); int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family, regionNum); assertEquals(regionNum,numRegions); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java index 4eac224f620..abbcb4cd07e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java @@ -83,7 +83,7 @@ public class TestHBaseTestingUtility { htu2.startMiniCluster(); htu3.startMiniCluster(); - final byte[] TABLE_NAME = Bytes.toBytes("test"); + final TableName TABLE_NAME = TableName.valueOf("test"); final byte[] FAM_NAME = Bytes.toBytes("fam"); final byte[] ROW = Bytes.toBytes("row"); final byte[] QUAL_NAME = Bytes.toBytes("qual"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java index ec9cc766ea3..62b00d8d24e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java @@ -99,17 +99,16 @@ public class TestInfoServers { @Test public void testMasterServerReadOnly() throws Exception { - String sTableName = "testMasterServerReadOnly"; - byte[] tableName = Bytes.toBytes(sTableName); + TableName tableName = TableName.valueOf("testMasterServerReadOnly"); byte[] cf = Bytes.toBytes("d"); UTIL.createTable(tableName, cf); new HTable(UTIL.getConfiguration(), tableName).close(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); assertDoesNotContainContent( - new URL("http://localhost:" + port + "/table.jsp?name=" + sTableName + "&action=split&key="), + new URL("http://localhost:" + port + "/table.jsp?name=" + tableName + "&action=split&key="), "Table action request accepted"); assertDoesNotContainContent( - new URL("http://localhost:" + port + "/table.jsp?name=" + sTableName), + new URL("http://localhost:" + port + "/table.jsp?name=" + tableName), "Actions:"); } 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 8333954f64f..6c70384c3a7 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 @@ -144,7 +144,7 @@ public class TestMultiVersions { this.admin.createTable(desc); Put put = new Put(row, timestamp1); put.add(contents, contents, value1); - Table table = new HTable(UTIL.getConfiguration(), tableName); + Table table = new HTable(UTIL.getConfiguration(), desc.getTableName()); table.put(put); // Shut down and restart the HBase cluster table.close(); @@ -153,7 +153,7 @@ public class TestMultiVersions { UTIL.startMiniHBaseCluster(1, NUM_SLAVES); // Make a new connection. Use new Configuration instance because old one // is tied to an HConnection that has since gone stale. - table = new HTable(new Configuration(UTIL.getConfiguration()), tableName); + table = new HTable(new Configuration(UTIL.getConfiguration()), desc.getTableName()); // Overwrite previous value put = new Put(row, timestamp2); put.add(contents, contents, value2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index a0a810eef3b..1944b611190 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -261,7 +261,7 @@ public class TestZooKeeper { } Table table = - new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); + new HTable(new Configuration(TEST_UTIL.getConfiguration()), desc.getTableName()); Put put = new Put(Bytes.toBytes("testrow")); put.add(Bytes.toBytes("fam"), Bytes.toBytes("col"), Bytes.toBytes("testdata")); @@ -543,7 +543,7 @@ public class TestZooKeeper { htd.addFamily(hcd); admin.createTable(htd, SPLIT_KEYS); TEST_UTIL.waitUntilNoRegionsInTransition(60000); - table = new HTable(TEST_UTIL.getConfiguration(), tableName); + table = new HTable(TEST_UTIL.getConfiguration(), htd.getTableName()); Put p; int numberOfPuts; for (numberOfPuts = 0; numberOfPuts < 6; numberOfPuts++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 937ebe7bea5..7f93780810d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -300,8 +300,8 @@ public class TestAdmin { final byte [] row = Bytes.toBytes("row"); final byte [] qualifier = Bytes.toBytes("qualifier"); final byte [] value = Bytes.toBytes("value"); - final byte [] table1 = Bytes.toBytes("testDisableAndEnableTable1"); - final byte [] table2 = Bytes.toBytes("testDisableAndEnableTable2"); + final TableName table1 = TableName.valueOf("testDisableAndEnableTable1"); + final TableName table2 = TableName.valueOf("testDisableAndEnableTable2"); Table ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY); Table ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY); Put put = new Put(row); @@ -350,8 +350,7 @@ public class TestAdmin { public void testCreateTable() throws IOException { HTableDescriptor [] tables = admin.listTables(); int numTables = tables.length; - TEST_UTIL.createTable(Bytes.toBytes("testCreateTable"), - HConstants.CATALOG_FAMILY).close(); + TEST_UTIL.createTable(TableName.valueOf("testCreateTable"), HConstants.CATALOG_FAMILY).close(); tables = this.admin.listTables(); assertEquals(numTables + 1, tables.length); assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster() @@ -411,7 +410,7 @@ public class TestAdmin { htd.addFamily(fam2); htd.addFamily(fam3); this.admin.createTable(htd); - Table table = new HTable(TEST_UTIL.getConfiguration(), "myTestTable"); + Table table = new HTable(TEST_UTIL.getConfiguration(), htd.getTableName()); HTableDescriptor confirmedHtd = table.getTableDescriptor(); assertEquals(htd.compareTo(confirmedHtd), 0); table.close(); @@ -1227,7 +1226,7 @@ public class TestAdmin { admin.createTable(htd1); admin.createTable(htd2); // Before fix, below would fail throwing a NoServerForRegionException. - new HTable(TEST_UTIL.getConfiguration(), name).close(); + new HTable(TEST_UTIL.getConfiguration(), htd2.getTableName()).close(); } /*** @@ -1264,7 +1263,7 @@ public class TestAdmin { */ @Test (timeout=300000) public void testReadOnlyTable() throws Exception { - byte [] name = Bytes.toBytes("testReadOnlyTable"); + TableName name = TableName.valueOf("testReadOnlyTable"); Table table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY); byte[] value = Bytes.toBytes("somedata"); // This used to use an empty row... That must have been a bug @@ -1310,7 +1309,7 @@ public class TestAdmin { */ @Test (expected=TableExistsException.class, timeout=300000) public void testTableExistsExceptionWithATable() throws IOException { - final byte [] name = Bytes.toBytes("testTableExistsExceptionWithATable"); + final TableName name = TableName.valueOf("testTableExistsExceptionWithATable"); TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY).close(); TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY); } @@ -1348,8 +1347,9 @@ public class TestAdmin { */ @Test (expected=TableNotFoundException.class, timeout=300000) public void testTableNotFoundExceptionWithoutAnyTables() throws IOException { - Table ht = - new HTable(TEST_UTIL.getConfiguration(),"testTableNotFoundExceptionWithoutAnyTables"); + TableName tableName = TableName + .valueOf("testTableNotFoundExceptionWithoutAnyTables"); + Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); ht.get(new Get("e".getBytes())); } @@ -1390,7 +1390,7 @@ public class TestAdmin { createTableWithDefaultConf(TABLENAME); HRegionInfo info = null; - HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME); + HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME)); List onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()); for (HRegionInfo regionInfo : onlineRegions) { if (!regionInfo.isMetaTable()) { @@ -1448,7 +1448,7 @@ public class TestAdmin { byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion3"); createTableWithDefaultConf(TABLENAME); - HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME); + HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME)); try { List onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()); @@ -1472,7 +1472,7 @@ public class TestAdmin { byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegionWhenServerNameIsEmpty"); createTableWithDefaultConf(TABLENAME); - HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME); + HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME)); try { List onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()); @@ -1496,7 +1496,7 @@ public class TestAdmin { createTableWithDefaultConf(TABLENAME); HRegionInfo info = null; - HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME); + HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME)); List onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()); for (HRegionInfo regionInfo : onlineRegions) { @@ -1579,7 +1579,7 @@ public class TestAdmin { v.append(className); } byte[] value = Bytes.toBytes(v.toString()); - HRegionServer regionServer = startAndWriteData("TestLogRolling", value); + HRegionServer regionServer = startAndWriteData(TableName.valueOf("TestLogRolling"), value); LOG.info("after writing there are " + HLogUtilsForTests.getNumRolledLogFiles(regionServer.getWAL()) + " log files"); @@ -1659,19 +1659,19 @@ public class TestAdmin { "hbase.regionserver.hlog.lowreplication.rolllimit", 3); } - private HRegionServer startAndWriteData(String tableName, byte[] value) + private HRegionServer startAndWriteData(TableName tableName, byte[] value) throws IOException, InterruptedException { // When the hbase:meta table can be opened, the region servers are running new HTable( TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME).close(); // Create the test table and open it - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); - HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName); for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i))); put.add(HConstants.CATALOG_FAMILY, null, value); @@ -1772,10 +1772,9 @@ public class TestAdmin { // here because makes use of an internal HBA method (TODO: Fix.). HBaseAdmin rawAdmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); - final String name = "testGetRegion"; - LOG.info("Started " + name); - final byte [] nameBytes = Bytes.toBytes(name); - HTable t = TEST_UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY); + final TableName tableName = TableName.valueOf("testGetRegion"); + LOG.info("Started " + tableName); + HTable t = TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); TEST_UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY); HRegionLocation regionLocation = t.getRegionLocation("mmm"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java index f8f7bb312b2..f8c093fc980 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java @@ -84,7 +84,7 @@ public class TestClientScannerRPCTimeout { @Test public void testScannerNextRPCTimesout() throws Exception { - final byte[] TABLE_NAME = Bytes.toBytes("testScannerNextRPCTimesout"); + final TableName TABLE_NAME = TableName.valueOf("testScannerNextRPCTimesout"); Table ht = TEST_UTIL.createTable(TABLE_NAME, FAMILY); byte[] r1 = Bytes.toBytes("row-1"); byte[] r2 = Bytes.toBytes("row-2"); 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 610c815c145..aa44647f8ca 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 @@ -182,7 +182,7 @@ public class TestFromClientSide { HColumnDescriptor hcd = new HColumnDescriptor(FAMILY) .setKeepDeletedCells(true).setMaxVersions(3); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TABLENAME)); + HTableDescriptor desc = new HTableDescriptor(TABLENAME); desc.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(desc); Configuration c = TEST_UTIL.getConfiguration(); @@ -385,7 +385,7 @@ public class TestFromClientSide { */ @Test public void testGetConfiguration() throws Exception { - byte[] TABLE = Bytes.toBytes("testGetConfiguration"); + TableName TABLE = TableName.valueOf("testGetConfiguration"); byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") }; Configuration conf = TEST_UTIL.getConfiguration(); Table table = TEST_UTIL.createTable(TABLE, FAMILIES, conf); @@ -533,7 +533,7 @@ public class TestFromClientSide { @Test public void testFilterAcrossMultipleRegions() throws IOException, InterruptedException { - byte [] name = Bytes.toBytes("testFilterAcrossMutlipleRegions"); + TableName name = TableName.valueOf("testFilterAcrossMutlipleRegions"); HTable t = TEST_UTIL.createTable(name, FAMILY); int rowCount = TEST_UTIL.loadTable(t, FAMILY, false); assertRowCount(t, rowCount); @@ -1208,7 +1208,7 @@ public class TestFromClientSide { // Null family (should NOT work) try { - TEST_UTIL.createTable(TABLE, (byte[])null); + TEST_UTIL.createTable(TABLE, new byte[][]{(byte[])null}); fail("Creating a table with a null family passed, should fail"); } catch(Exception e) {} @@ -1243,7 +1243,7 @@ public class TestFromClientSide { // Use a new table byte [] TABLE2 = Bytes.toBytes("testNull2"); - ht = TEST_UTIL.createTable(TABLE2, FAMILY); + ht = TEST_UTIL.createTable(TableName.valueOf(TABLE2), FAMILY); // Empty qualifier, byte[0] instead of null (should work) try { @@ -3597,8 +3597,7 @@ public class TestFromClientSide { @Test public void testUpdatesWithMajorCompaction() throws Exception { - String tableName = "testUpdatesWithMajorCompaction"; - byte [] TABLE = Bytes.toBytes(tableName); + TableName TABLE = TableName.valueOf("testUpdatesWithMajorCompaction"); Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); @@ -3630,8 +3629,8 @@ public class TestFromClientSide { assertEquals("BBB", Bytes.toString(navigableMap.get(2L))); // Trigger a major compaction - admin.flush(tableName); - admin.majorCompact(tableName); + admin.flush(TABLE); + admin.majorCompact(TABLE); Thread.sleep(6000); // Update the value at timestamp 1 @@ -3645,8 +3644,8 @@ public class TestFromClientSide { hTable.put(put); // Trigger a major compaction - admin.flush(tableName); - admin.majorCompact(tableName); + admin.flush(TABLE); + admin.majorCompact(TABLE); Thread.sleep(6000); // Check that the values at timestamp 2 and 1 got updated @@ -3726,7 +3725,7 @@ public class TestFromClientSide { @Test public void testGet_EmptyTable() throws IOException { - Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_EmptyTable"), FAMILY); + Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_EmptyTable"), FAMILY); Get get = new Get(ROW); get.addFamily(FAMILY); Result r = table.get(get); @@ -3735,7 +3734,7 @@ public class TestFromClientSide { @Test public void testGet_NullQualifier() throws IOException { - Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NullQualifier"), FAMILY); + Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_NullQualifier"), FAMILY); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -3758,7 +3757,7 @@ public class TestFromClientSide { @Test public void testGet_NonExistentRow() throws IOException { - Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NonExistentRow"), FAMILY); + Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_NonExistentRow"), FAMILY); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -3822,7 +3821,7 @@ public class TestFromClientSide { public void testPutNoCF() throws IOException { final byte[] BAD_FAM = Bytes.toBytes("BAD_CF"); final byte[] VAL = Bytes.toBytes(100); - Table table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), new byte[][]{FAMILY}); + Table table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), FAMILY); boolean caughtNSCFE = false; @@ -4064,10 +4063,10 @@ public class TestFromClientSide { @Test public void testListTables() throws IOException, InterruptedException { - byte [] t1 = Bytes.toBytes("testListTables1"); - byte [] t2 = Bytes.toBytes("testListTables2"); - byte [] t3 = Bytes.toBytes("testListTables3"); - byte [][] tables = new byte[][] { t1, t2, t3 }; + TableName t1 = TableName.valueOf("testListTables1"); + TableName t2 = TableName.valueOf("testListTables2"); + TableName t3 = TableName.valueOf("testListTables3"); + TableName [] tables = new TableName[] { t1, t2, t3 }; for (int i = 0; i < tables.length; i++) { TEST_UTIL.createTable(tables[i], FAMILY); } @@ -4080,12 +4079,12 @@ public class TestFromClientSide { for (int i = 0; i < tables.length && i < size; i++) { boolean found = false; for (int j = 0; j < ts.length; j++) { - if (Bytes.equals(ts[j].getTableName().getName(), tables[i])) { + if (ts[j].getTableName().equals(tables[i])) { found = true; break; } } - assertTrue("Not found: " + Bytes.toString(tables[i]), found); + assertTrue("Not found: " + tables[i], found); } } @@ -4096,7 +4095,7 @@ public class TestFromClientSide { * @return the created HTable object * @throws IOException */ - HTable createUnmangedHConnectionHTable(final byte [] tableName) throws IOException { + HTable createUnmangedHConnectionHTable(final TableName tableName) throws IOException { TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); return (HTable)conn.getTable(tableName); @@ -4110,7 +4109,7 @@ public class TestFromClientSide { */ @Test public void testUnmanagedHConnection() throws IOException { - final byte[] tableName = Bytes.toBytes("testUnmanagedHConnection"); + final TableName tableName = TableName.valueOf("testUnmanagedHConnection"); HTable t = createUnmangedHConnectionHTable(tableName); HBaseAdmin ha = new HBaseAdmin(t.getConnection()); assertTrue(ha.tableExists(tableName)); @@ -4125,7 +4124,7 @@ public class TestFromClientSide { */ @Test public void testUnmanagedHConnectionReconnect() throws Exception { - final byte[] tableName = Bytes.toBytes("testUnmanagedHConnectionReconnect"); + final TableName tableName = TableName.valueOf("testUnmanagedHConnectionReconnect"); HTable t = createUnmangedHConnectionHTable(tableName); Connection conn = t.getConnection(); HBaseAdmin ha = new HBaseAdmin(conn); @@ -4150,8 +4149,8 @@ public class TestFromClientSide { @Test public void testMiscHTableStuff() throws IOException { - final byte[] tableAname = Bytes.toBytes("testMiscHTableStuffA"); - final byte[] tableBname = Bytes.toBytes("testMiscHTableStuffB"); + final TableName tableAname = TableName.valueOf("testMiscHTableStuffA"); + final TableName tableBname = TableName.valueOf("testMiscHTableStuffB"); final byte[] attrName = Bytes.toBytes("TESTATTR"); final byte[] attrValue = Bytes.toBytes("somevalue"); byte[] value = Bytes.toBytes("value"); @@ -4193,7 +4192,7 @@ public class TestFromClientSide { // to be reloaded. // Test user metadata - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); // make a modifiable descriptor HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor()); // offline the table @@ -4211,7 +4210,7 @@ public class TestFromClientSide { // Test that attribute changes were applied desc = a.getTableDescriptor(); assertTrue("wrong table descriptor returned", - Bytes.compareTo(desc.getTableName().getName(), tableAname) == 0); + desc.getTableName().equals(tableAname)); // check HTD attribute value = desc.getValue(attrName); assertFalse("missing HTD attribute value", value == null); @@ -4228,7 +4227,7 @@ public class TestFromClientSide { @Test public void testGetClosestRowBefore() throws IOException, InterruptedException { - final byte[] tableAname = Bytes.toBytes("testGetClosestRowBefore"); + final TableName tableAname = TableName.valueOf("testGetClosestRowBefore"); final byte[] firstRow = Bytes.toBytes("row111"); final byte[] secondRow = Bytes.toBytes("row222"); final byte[] thirdRow = Bytes.toBytes("row333"); @@ -4342,7 +4341,7 @@ public class TestFromClientSide { @Test public void testMultiRowMutation() throws Exception { LOG.info("Starting testMultiRowMutation"); - final byte [] TABLENAME = Bytes.toBytes("testMultiRowMutation"); + final TableName TABLENAME = TableName.valueOf("testMultiRowMutation"); final byte [] ROW1 = Bytes.toBytes("testRow1"); Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); @@ -4373,7 +4372,7 @@ public class TestFromClientSide { @Test public void testRowMutation() throws Exception { LOG.info("Starting testRowMutation"); - final byte [] TABLENAME = Bytes.toBytes("testRowMutation"); + final TableName TABLENAME = TableName.valueOf("testRowMutation"); Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); byte [][] QUALIFIERS = new byte [][] { Bytes.toBytes("a"), Bytes.toBytes("b") @@ -4405,7 +4404,7 @@ public class TestFromClientSide { @Test public void testAppend() throws Exception { LOG.info("Starting testAppend"); - final byte [] TABLENAME = Bytes.toBytes("testAppend"); + final TableName TABLENAME = TableName.valueOf("testAppend"); Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); byte[] v1 = Bytes.toBytes("42"); byte[] v2 = Bytes.toBytes("23"); @@ -4456,7 +4455,7 @@ public class TestFromClientSide { @Test public void testIncrementingInvalidValue() throws Exception { LOG.info("Starting testIncrementingInvalidValue"); - final byte [] TABLENAME = Bytes.toBytes("testIncrementingInvalidValue"); + final TableName TABLENAME = TableName.valueOf("testIncrementingInvalidValue"); Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); final byte[] COLUMN = Bytes.toBytes("column"); Put p = new Put(ROW); @@ -4482,7 +4481,7 @@ public class TestFromClientSide { @Test public void testIncrementInvalidArguments() throws Exception { LOG.info("Starting testIncrementInvalidArguments"); - final byte[] TABLENAME = Bytes.toBytes("testIncrementInvalidArguments"); + final TableName TABLENAME = TableName.valueOf("testIncrementInvalidArguments"); Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); final byte[] COLUMN = Bytes.toBytes("column"); try { @@ -4537,7 +4536,7 @@ public class TestFromClientSide { @Test public void testIncrementOutOfOrder() throws Exception { LOG.info("Starting testIncrementOutOfOrder"); - final byte [] TABLENAME = Bytes.toBytes("testIncrementOutOfOrder"); + final TableName TABLENAME = TableName.valueOf("testIncrementOutOfOrder"); Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); byte [][] QUALIFIERS = new byte [][] { @@ -4577,7 +4576,7 @@ public class TestFromClientSide { @Test public void testIncrement() throws Exception { LOG.info("Starting testIncrement"); - final byte [] TABLENAME = Bytes.toBytes("testIncrement"); + final TableName TABLENAME = TableName.valueOf("testIncrement"); Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); byte [][] ROWS = new byte [][] { @@ -4648,7 +4647,7 @@ public class TestFromClientSide { @Test public void testClientPoolRoundRobin() throws IOException { - final byte[] tableName = Bytes.toBytes("testClientPoolRoundRobin"); + final TableName tableName = TableName.valueOf("testClientPoolRoundRobin"); int poolSize = 3; int numVersions = poolSize * 2; @@ -4656,8 +4655,7 @@ public class TestFromClientSide { conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin"); conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); - Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, - conf, Integer.MAX_VALUE); + Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, conf, Integer.MAX_VALUE); final long ts = EnvironmentEdgeManager.currentTime(); Get get = new Get(ROW); @@ -4685,7 +4683,7 @@ public class TestFromClientSide { @Ignore ("Flakey: HBASE-8989") @Test public void testClientPoolThreadLocal() throws IOException { - final byte[] tableName = Bytes.toBytes("testClientPoolThreadLocal"); + final TableName tableName = TableName.valueOf("testClientPoolThreadLocal"); int poolSize = Integer.MAX_VALUE; int numVersions = 3; @@ -4771,8 +4769,7 @@ public class TestFromClientSide { final byte [] anotherrow = Bytes.toBytes("anotherrow"); final byte [] value2 = Bytes.toBytes("abcd"); - Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"), - new byte [][] {FAMILY}); + Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndPut"), FAMILY); Put put1 = new Put(ROW); put1.add(FAMILY, QUALIFIER, VALUE); @@ -4813,8 +4810,7 @@ public class TestFromClientSide { final byte [] value3 = Bytes.toBytes("cccc"); final byte [] value4 = Bytes.toBytes("dddd"); - Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPutWithCompareOp"), - new byte [][] {FAMILY}); + Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndPutWithCompareOp"), FAMILY); Put put2 = new Put(ROW); put2.add(FAMILY, QUALIFIER, value2); @@ -4879,8 +4875,8 @@ public class TestFromClientSide { final byte [] value3 = Bytes.toBytes("cccc"); final byte [] value4 = Bytes.toBytes("dddd"); - Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndDeleteWithCompareOp"), - new byte [][] {FAMILY}); + Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndDeleteWithCompareOp"), + FAMILY); Put put2 = new Put(ROW); put2.add(FAMILY, QUALIFIER, value2); @@ -4953,7 +4949,7 @@ public class TestFromClientSide { @Test @SuppressWarnings ("unused") public void testScanMetrics() throws Exception { - byte [] TABLENAME = Bytes.toBytes("testScanMetrics"); + TableName TABLENAME = TableName.valueOf("testScanMetrics"); Configuration conf = TEST_UTIL.getConfiguration(); TEST_UTIL.createTable(TABLENAME, FAMILY); @@ -5057,13 +5053,12 @@ public class TestFromClientSide { */ @Test public void testCacheOnWriteEvictOnClose() throws Exception { - byte [] tableName = Bytes.toBytes("testCOWEOCfromClient"); + TableName tableName = TableName.valueOf("testCOWEOCfromClient"); byte [] data = Bytes.toBytes("data"); - HTable table = TEST_UTIL.createTable(tableName, new byte [][] {FAMILY}); + HTable table = TEST_UTIL.createTable(tableName, FAMILY); // get the block cache and region String regionName = table.getRegionLocations().firstKey().getEncodedName(); - HRegion region = TEST_UTIL.getRSForFirstRegionInTable( - tableName).getFromOnlineRegions(regionName); + HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName); Store store = region.getStores().values().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); @@ -5171,8 +5166,7 @@ public class TestFromClientSide { */ public void testNonCachedGetRegionLocation() throws Exception { // Test Initialization. - String tableName = "testNonCachedGetRegionLocation"; - byte [] TABLE = Bytes.toBytes(tableName); + TableName TABLE = TableName.valueOf("testNonCachedGetRegionLocation"); byte [] family1 = Bytes.toBytes("f1"); byte [] family2 = Bytes.toBytes("f2"); HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10); @@ -5220,7 +5214,7 @@ public class TestFromClientSide { // Test Initialization. byte [] startKey = Bytes.toBytes("ddc"); byte [] endKey = Bytes.toBytes("mmm"); - byte [] TABLE = Bytes.toBytes("testGetRegionsInRange"); + TableName TABLE = TableName.valueOf("testGetRegionsInRange"); HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10); int numOfRegions = TEST_UTIL.createMultiRegions(table, FAMILY); assertEquals(25, numOfRegions); @@ -5271,7 +5265,7 @@ public class TestFromClientSide { @Test public void testJira6912() throws Exception { - byte [] TABLE = Bytes.toBytes("testJira6912"); + TableName TABLE = TableName.valueOf("testJira6912"); Table foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10); List puts = new ArrayList(); @@ -5297,7 +5291,7 @@ public class TestFromClientSide { @Test public void testScan_NullQualifier() throws IOException { - Table table = TEST_UTIL.createTable(Bytes.toBytes("testScan_NullQualifier"), FAMILY); + Table table = TEST_UTIL.createTable(TableName.valueOf("testScan_NullQualifier"), FAMILY); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -5326,7 +5320,7 @@ public class TestFromClientSide { @Test public void testNegativeTimestamp() throws IOException { - Table table = TEST_UTIL.createTable(Bytes.toBytes("testNegativeTimestamp"), FAMILY); + Table table = TEST_UTIL.createTable(TableName.valueOf("testNegativeTimestamp"), FAMILY); try { Put put = new Put(ROW, -1); @@ -5470,8 +5464,8 @@ public class TestFromClientSide { @Test public void testRawScanRespectsVersions() throws Exception { - byte[] TABLE = Bytes.toBytes("testRawScan"); - Table table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }); + TableName TABLE = TableName.valueOf("testRawScan"); + Table table = TEST_UTIL.createTable(TABLE, FAMILY); byte[] row = Bytes.toBytes("row"); // put the same row 4 times, with different values @@ -5546,7 +5540,7 @@ public class TestFromClientSide { @Test public void testSmallScan() throws Exception { // Test Initialization. - byte[] TABLE = Bytes.toBytes("testSmallScan"); + TableName TABLE = TableName.valueOf("testSmallScan"); Table table = TEST_UTIL.createTable(TABLE, FAMILY); // Insert one row each region @@ -5582,7 +5576,7 @@ public class TestFromClientSide { @Test public void testSuperSimpleWithReverseScan() throws Exception { - byte[] TABLE = Bytes.toBytes("testSuperSimpleWithReverseScan"); + TableName TABLE = TableName.valueOf("testSuperSimpleWithReverseScan"); Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000")); put.add(FAMILY, QUALIFIER, VALUE); @@ -5628,7 +5622,7 @@ public class TestFromClientSide { @Test public void testFiltersWithReverseScan() throws Exception { - byte[] TABLE = Bytes.toBytes("testFiltersWithReverseScan"); + TableName TABLE = TableName.valueOf("testFiltersWithReverseScan"); Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte[][] ROWS = makeN(ROW, 10); byte[][] QUALIFIERS = { Bytes.toBytes("col0--"), @@ -5668,7 +5662,7 @@ public class TestFromClientSide { @Test public void testKeyOnlyFilterWithReverseScan() throws Exception { - byte[] TABLE = Bytes.toBytes("testKeyOnlyFilterWithReverseScan"); + TableName TABLE = TableName.valueOf("testKeyOnlyFilterWithReverseScan"); Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte[][] ROWS = makeN(ROW, 10); byte[][] QUALIFIERS = { Bytes.toBytes("col0--"), @@ -5709,7 +5703,7 @@ public class TestFromClientSide { */ @Test public void testSimpleMissingWithReverseScan() throws Exception { - byte[] TABLE = Bytes.toBytes("testSimpleMissingWithReverseScan"); + TableName TABLE = TableName.valueOf("testSimpleMissingWithReverseScan"); Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte[][] ROWS = makeN(ROW, 4); @@ -5774,7 +5768,7 @@ public class TestFromClientSide { @Test public void testNullWithReverseScan() throws Exception { - byte[] TABLE = Bytes.toBytes("testNullWithReverseScan"); + TableName TABLE = TableName.valueOf("testNullWithReverseScan"); Table ht = TEST_UTIL.createTable(TABLE, FAMILY); // Null qualifier (should work) Put put = new Put(ROW); @@ -5786,7 +5780,7 @@ public class TestFromClientSide { ht.delete(delete); // Use a new table byte[] TABLE2 = Bytes.toBytes("testNull2WithReverseScan"); - ht = TEST_UTIL.createTable(TABLE2, FAMILY); + ht = TEST_UTIL.createTable(TableName.valueOf(TABLE2), FAMILY); // Empty qualifier, byte[0] instead of null (should work) put = new Put(ROW); put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE); @@ -5811,13 +5805,12 @@ public class TestFromClientSide { @Test public void testDeletesWithReverseScan() throws Exception { - byte[] TABLE = Bytes.toBytes("testDeletesWithReverseScan"); + TableName TABLE = TableName.valueOf("testDeletesWithReverseScan"); byte[][] ROWS = makeNAscii(ROW, 6); byte[][] FAMILIES = makeNAscii(FAMILY, 3); byte[][] VALUES = makeN(VALUE, 5); long[] ts = { 1000, 2000, 3000, 4000, 5000 }; - Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, - TEST_UTIL.getConfiguration(), 3); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, TEST_UTIL.getConfiguration(), 3); Put put = new Put(ROW); put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); @@ -5998,7 +5991,7 @@ public class TestFromClientSide { @Test public void testReversedScanUnderMultiRegions() throws Exception { // Test Initialization. - byte[] TABLE = Bytes.toBytes("testReversedScanUnderMultiRegions"); + TableName TABLE = TableName.valueOf("testReversedScanUnderMultiRegions"); byte[] maxByteArray = ReversedClientScanner.MAX_BYTE_ARRAY; byte[][] splitRows = new byte[][] { Bytes.toBytes("005"), Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)), @@ -6055,7 +6048,7 @@ public class TestFromClientSide { @Test public void testSmallReversedScanUnderMultiRegions() throws Exception { // Test Initialization. - byte[] TABLE = Bytes.toBytes("testSmallReversedScanUnderMultiRegions"); + TableName TABLE = TableName.valueOf("testSmallReversedScanUnderMultiRegions"); byte[][] splitRows = new byte[][]{ Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"), Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")}; 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 7b155441913..31bc1359583 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 @@ -338,8 +338,8 @@ public class TestFromClientSide3 { public void testHTableExistsMethodMultipleRegionsSingleGet() throws Exception { Table table = TEST_UTIL.createTable( - Bytes.toBytes("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY }, 1, - new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255); + TableName.valueOf("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY }, + 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); @@ -357,8 +357,8 @@ public class TestFromClientSide3 { @Test public void testHTableExistsMethodMultipleRegionsMultipleGets() throws Exception { HTable table = TEST_UTIL.createTable( - Bytes.toBytes("testHTableExistsMethodMultipleRegionsMultipleGets"), new byte[][] { FAMILY }, - 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255); + TableName.valueOf("testHTableExistsMethodMultipleRegionsMultipleGets"), + new byte[][] { FAMILY }, 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); table.put (put); @@ -410,7 +410,7 @@ public class TestFromClientSide3 { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test"))); desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table table = new HTable(TEST_UTIL.getConfiguration(), "test"); + Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); Put put = new Put(ROW_BYTES); put.add(FAMILY, COL_QUAL, VAL_BYTES); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index 1558eaf1c4b..e4bf256d2e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.exceptions.OperationConflictException; import org.apache.hadoop.hbase.ipc.RpcClient; @@ -67,7 +68,7 @@ public class TestMultiParallel { private static final byte[] VALUE = Bytes.toBytes("value"); private static final byte[] QUALIFIER = Bytes.toBytes("qual"); private static final String FAMILY = "family"; - private static final String TEST_TABLE = "multi_test_table"; + private static final TableName TEST_TABLE = TableName.valueOf("multi_test_table"); private static final byte[] BYTES_FAMILY = Bytes.toBytes(FAMILY); private static final byte[] ONE_ROW = Bytes.toBytes("xxx"); private static final byte [][] KEYS = makeKeys(); @@ -79,9 +80,9 @@ public class TestMultiParallel { ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); UTIL.startMiniCluster(slaves); - HTable t = UTIL.createTable(Bytes.toBytes(TEST_TABLE), Bytes.toBytes(FAMILY)); + HTable t = UTIL.createTable(TEST_TABLE, Bytes.toBytes(FAMILY)); UTIL.createMultiRegions(t, Bytes.toBytes(FAMILY)); - UTIL.waitTableEnabled(Bytes.toBytes(TEST_TABLE)); + UTIL.waitTableEnabled(TEST_TABLE.getName()); t.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java index 61de8518bc8..b46312f91cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -48,7 +49,7 @@ public class TestScannerTimeout { final Log LOG = LogFactory.getLog(getClass()); private final static byte[] SOME_BYTES = Bytes.toBytes("f"); - private final static byte[] TABLE_NAME = Bytes.toBytes("t"); + private final static TableName TABLE_NAME = TableName.valueOf("t"); private final static int NB_ROWS = 10; // Be careful w/ what you set this timer to... it can get in the way of // the mini cluster coming up -- the verification in particular. 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 0165bac939c..a6c1cfef3e8 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 @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTestConst; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.filter.ColumnPrefixFilter; import org.apache.hadoop.hbase.filter.ColumnRangeFilter; import org.apache.hadoop.hbase.master.HMaster; @@ -101,7 +102,7 @@ public class TestScannersFromClientSide { */ @Test public void testScanBatch() throws Exception { - byte [] TABLE = Bytes.toBytes("testScanBatch"); + TableName TABLE = TableName.valueOf("testScanBatch"); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 8); Table ht = TEST_UTIL.createTable(TABLE, FAMILY); @@ -442,7 +443,7 @@ public class TestScannersFromClientSide { */ @Test public void testScanOnReopenedRegion() throws Exception { - byte [] TABLE = Bytes.toBytes("testScanOnReopenedRegion"); + TableName TABLE = TableName.valueOf("testScanOnReopenedRegion"); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 2); HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); 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 a1ead55d7cc..1d9ff1e53b0 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 @@ -58,7 +58,7 @@ public class TestSnapshotCloneIndependence { private static final String STRING_TABLE_NAME = "test"; private static final String TEST_FAM_STR = "fam"; private static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR); - private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME); + private static final TableName TABLE_NAME = TableName.valueOf(STRING_TABLE_NAME); /** * Setup the config for the cluster and start it diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java index 12a04570d10..4843715d05f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java @@ -98,7 +98,7 @@ public class TestTimestampsFilter { Cell kvs[]; // create table; set versions to max... - Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TableName.valueOf(TABLE), FAMILIES, Integer.MAX_VALUE); for (int rowIdx = 0; rowIdx < 5; rowIdx++) { for (int colIdx = 0; colIdx < 5; colIdx++) { @@ -173,7 +173,7 @@ public class TestTimestampsFilter { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TableName.valueOf(TABLE), FAMILIES, Integer.MAX_VALUE); Put p = new Put(Bytes.toBytes("row")); p.add(FAMILY, Bytes.toBytes("column0"), 3, Bytes.toBytes("value0-3")); @@ -233,7 +233,7 @@ public class TestTimestampsFilter { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TableName.valueOf(TABLE), FAMILIES, Integer.MAX_VALUE); // For row:0, col:0: insert versions 1 through 5. putNVersions(ht, FAMILY, 0, 0, 1, 5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java index af50b526d37..de0057cb899 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java @@ -66,7 +66,7 @@ public class TestHTableWrapper { private static final HBaseTestingUtility util = new HBaseTestingUtility(); - private static final byte[] TEST_TABLE = Bytes.toBytes("test"); + private static final TableName TEST_TABLE = TableName.valueOf("test"); private static final byte[] TEST_FAMILY = Bytes.toBytes("f1"); private static final byte[] ROW_A = Bytes.toBytes("aaa"); @@ -136,7 +136,7 @@ public class TestHTableWrapper { CoprocessorEnvironment env = cpHost.findCoprocessorEnvironment(implClazz.getName()); assertEquals(Coprocessor.VERSION, env.getVersion()); assertEquals(VersionInfo.getVersion(), env.getHBaseVersion()); - hTableInterface = env.getTable(TableName.valueOf(TEST_TABLE)); + hTableInterface = env.getTable(TEST_TABLE); checkHTableInterfaceMethods(); cpHost.shutdown(env); } @@ -170,7 +170,7 @@ public class TestHTableWrapper { } private void checkNameAndDescriptor() throws IOException { - assertArrayEquals(TEST_TABLE, hTableInterface.getTableName()); + assertEquals(TEST_TABLE, hTableInterface.getName()); assertEquals(table.getTableDescriptor(), hTableInterface.getTableDescriptor()); } 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 f6e7437ecb8..57db176f308 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 @@ -162,7 +162,7 @@ public class TestOpenTableInCoprocessor { admin.createTable(primary); admin.createTable(other); - Table table = new HTable(UTIL.getConfiguration(), "primary"); + Table table = new HTable(UTIL.getConfiguration(), TableName.valueOf("primary")); Put p = new Put(new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' }); table.put(p); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java index dc96d502297..1c81adfb071 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java @@ -160,7 +160,7 @@ public class TestColumnRangeFilter { public void TestColumnRangeFilterClient() throws Exception { String family = "Family"; String table = "TestColumnRangeFilterClient"; - Table ht = TEST_UTIL.createTable(Bytes.toBytes(table), + Table ht = TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(family), Integer.MAX_VALUE); List rows = generateRandomWords(10, 8); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java index 83958179a37..565c7db52b4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java @@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -89,7 +90,7 @@ public class TestFuzzyRowAndColumnRangeFilter { public void Test() throws Exception { String cf = "f"; String table = "TestFuzzyAndColumnRangeFilterClient"; - Table ht = TEST_UTIL.createTable(Bytes.toBytes(table), + Table ht = TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(cf), Integer.MAX_VALUE); // 10 byte row key - (2 bytes 4 bytes 4 bytes) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java index f21176b15a2..409ac8ce76e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -260,7 +261,7 @@ public class TestBlockReorder { // We use the regionserver file system & conf as we expect it to have the hook. conf = targetRs.getConfiguration(); HFileSystem rfs = (HFileSystem) targetRs.getFileSystem(); - Table h = htu.createTable("table".getBytes(), sb); + Table h = htu.createTable(TableName.valueOf("table"), sb); // Now, we have 4 datanodes and a replication count of 3. So we don't know if the datanode // with the same node will be used. We can't really stop an existing datanode, this would diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java index c961e4eafa2..0be8931806a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java @@ -74,7 +74,7 @@ public class TestChangingEncoding { private HBaseAdmin admin; private HColumnDescriptor hcd; - private String tableName; + private TableName tableName; private static final List ENCODINGS_TO_ITERATE = createEncodingsToIterate(); @@ -89,8 +89,8 @@ public class TestChangingEncoding { private int numBatchesWritten; private void prepareTest(String testId) throws IOException { - tableName = "test_table_" + testId; - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); + tableName = TableName.valueOf("test_table_" + testId); + HTableDescriptor htd = new HTableDescriptor(tableName); hcd = new HColumnDescriptor(CF); htd.addFamily(hcd); admin.createTable(htd); @@ -135,7 +135,7 @@ public class TestChangingEncoding { + "_col" + j); } - static void writeTestDataBatch(Configuration conf, String tableName, + static void writeTestDataBatch(Configuration conf, TableName tableName, int batchId) throws Exception { LOG.debug("Writing test data batch " + batchId); Table table = new HTable(conf, tableName); @@ -153,7 +153,7 @@ public class TestChangingEncoding { table.close(); } - static void verifyTestDataBatch(Configuration conf, String tableName, + static void verifyTestDataBatch(Configuration conf, TableName tableName, int batchId) throws Exception { LOG.debug("Verifying test data batch " + batchId); Table table = new HTable(conf, tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java index 6a6da1ffab2..9fb4eb8dd26 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java @@ -91,7 +91,7 @@ public class TestTableInputFormat { * @throws IOException */ public static Table createTable(byte[] tableName) throws IOException { - Table table = UTIL.createTable(tableName, FAMILY); + Table table = UTIL.createTable(TableName.valueOf(tableName), new byte[][]{FAMILY}); Put p = new Put("aaa".getBytes()); p.add(FAMILY, null, "value aaa".getBytes()); table.put(p); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java index c8277519394..107837e6e21 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java @@ -80,19 +80,19 @@ public class TestTableMapReduce extends TestTableMapReduceBase { jobConf = new JobConf(UTIL.getConfiguration(), TestTableMapReduce.class); jobConf.setJobName("process column contents"); jobConf.setNumReduceTasks(1); - TableMapReduceUtil.initTableMapJob(Bytes.toString(table.getTableName()), + TableMapReduceUtil.initTableMapJob(table.getName().getNameAsString(), Bytes.toString(INPUT_FAMILY), ProcessContentsMapper.class, ImmutableBytesWritable.class, Put.class, jobConf); - TableMapReduceUtil.initTableReduceJob(Bytes.toString(table.getTableName()), + TableMapReduceUtil.initTableReduceJob(table.getName().getNameAsString(), IdentityTableReduce.class, jobConf); - LOG.info("Started " + Bytes.toString(table.getTableName())); + LOG.info("Started " + table.getName()); RunningJob job = JobClient.runJob(jobConf); assertTrue(job.isSuccessful()); LOG.info("After map/reduce completion"); // verify map-reduce results - verify(Bytes.toString(table.getTableName())); + verify(table.getName()); } finally { if (jobConf != null) { FileUtil.fullyDelete(new File(jobConf.get("hadoop.tmp.dir"))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java index 451fce55204..6e9f8d8f06f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java @@ -33,6 +33,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.client.Put; @@ -88,7 +89,7 @@ public class TestTableMapReduceUtil { @BeforeClass public static void beforeClass() throws Exception { UTIL.startMiniCluster(); - presidentsTable = createAndFillTable(Bytes.toBytes(TABLE_NAME)); + presidentsTable = createAndFillTable(TableName.valueOf(TABLE_NAME)); UTIL.startMiniMapReduceCluster(); } @@ -105,7 +106,7 @@ public class TestTableMapReduceUtil { LOG.info("before done"); } - public static Table createAndFillTable(byte[] tableName) throws IOException { + public static Table createAndFillTable(TableName tableName) throws IOException { Table table = UTIL.createTable(tableName, COLUMN_FAMILY); createPutCommand(table); return table; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java index 73a0d1e6a5e..fac90a892d0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java @@ -30,6 +30,7 @@ import java.io.PrintStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -80,8 +81,8 @@ public class TestCopyTable { */ @Test public void testCopyTable() throws Exception { - final byte[] TABLENAME1 = Bytes.toBytes("testCopyTable1"); - final byte[] TABLENAME2 = Bytes.toBytes("testCopyTable2"); + final TableName TABLENAME1 = TableName.valueOf("testCopyTable1"); + final TableName TABLENAME2 = TableName.valueOf("testCopyTable2"); final byte[] FAMILY = Bytes.toBytes("family"); final byte[] COLUMN1 = Bytes.toBytes("c1"); @@ -99,8 +100,8 @@ public class TestCopyTable { assertEquals( 0, - copy.run(new String[] { "--new.name=" + Bytes.toString(TABLENAME2), - Bytes.toString(TABLENAME1) })); + copy.run(new String[] { "--new.name=" + TABLENAME2.getNameAsString(), + TABLENAME1.getNameAsString() })); // verify the data was copied into table 2 for (int i = 0; i < 10; i++) { @@ -118,8 +119,8 @@ public class TestCopyTable { @Test public void testStartStopRow() throws Exception { - final byte[] TABLENAME1 = Bytes.toBytes("testStartStopRow1"); - final byte[] TABLENAME2 = Bytes.toBytes("testStartStopRow2"); + final TableName TABLENAME1 = TableName.valueOf("testStartStopRow1"); + final TableName TABLENAME2 = TableName.valueOf("testStartStopRow2"); final byte[] FAMILY = Bytes.toBytes("family"); final byte[] COLUMN1 = Bytes.toBytes("c1"); final byte[] ROW0 = Bytes.toBytes("row0"); @@ -143,8 +144,8 @@ public class TestCopyTable { CopyTable copy = new CopyTable(TEST_UTIL.getConfiguration()); assertEquals( 0, - copy.run(new String[] { "--new.name=" + Bytes.toString(TABLENAME2), "--startrow=row1", - "--stoprow=row2", Bytes.toString(TABLENAME1) })); + copy.run(new String[] { "--new.name=" + TABLENAME2, "--startrow=row1", + "--stoprow=row2", TABLENAME1.getNameAsString() })); // verify the data was copied into table 2 // row1 exist, row0, row2 do not exist diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java index bf4d8a02dcc..33d0e749523 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java @@ -17,6 +17,7 @@ package org.apache.hadoop.hbase.mapreduce; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -52,8 +53,8 @@ public class TestHRegionPartitioner { byte[][] families = { Bytes.toBytes("familyA"), Bytes.toBytes("familyB") }; - UTIL.createTable(Bytes.toBytes("out_table"), families, 1, Bytes.toBytes("aa"), - Bytes.toBytes("cc"), 3); + UTIL.createTable(TableName.valueOf("out_table"), families, 1, + Bytes.toBytes("aa"), Bytes.toBytes("cc"), 3); HRegionPartitioner partitioner = new HRegionPartitioner(); Configuration configuration = UTIL.getConfiguration(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 91ff718a57e..04b1a924a89 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -161,7 +161,7 @@ public class TestImportExport { @Test public void testSimpleCase() throws Exception { String EXPORT_TABLE = "exportSimpleCase"; - Table t = UTIL.createTable(Bytes.toBytes(EXPORT_TABLE), FAMILYA, 3); + Table t = UTIL.createTable(TableName.valueOf(EXPORT_TABLE), FAMILYA, 3); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); p.add(FAMILYA, QUAL, now+1, QUAL); @@ -181,7 +181,7 @@ public class TestImportExport { assertTrue(runExport(args)); String IMPORT_TABLE = "importTableSimpleCase"; - t = UTIL.createTable(Bytes.toBytes(IMPORT_TABLE), FAMILYB, 3); + t = UTIL.createTable(TableName.valueOf(IMPORT_TABLE), FAMILYB, 3); args = new String[] { "-D" + Import.CF_RENAME_PROP + "="+FAMILYA_STRING+":"+FAMILYB_STRING, IMPORT_TABLE, @@ -224,7 +224,7 @@ public class TestImportExport { fs.copyFromLocalFile(importPath, new Path(FQ_OUTPUT_DIR + Path.SEPARATOR + "exportedTableIn94Format")); String IMPORT_TABLE = "importTableExportedFrom94"; - Table t = UTIL.createTable(Bytes.toBytes(IMPORT_TABLE), Bytes.toBytes("f1"), 3); + Table t = UTIL.createTable(TableName.valueOf(IMPORT_TABLE), Bytes.toBytes("f1"), 3); String[] args = new String[] { "-Dhbase.import.version=0.94" , IMPORT_TABLE, FQ_OUTPUT_DIR @@ -254,7 +254,7 @@ public class TestImportExport { .setMaxVersions(1) ); UTIL.getHBaseAdmin().createTable(desc); - Table t = new HTable(UTIL.getConfiguration(), BATCH_TABLE); + Table t = new HTable(UTIL.getConfiguration(), desc.getTableName()); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); @@ -285,7 +285,7 @@ public class TestImportExport { .setKeepDeletedCells(true) ); UTIL.getHBaseAdmin().createTable(desc); - Table t = new HTable(UTIL.getConfiguration(), EXPORT_TABLE); + Table t = new HTable(UTIL.getConfiguration(), desc.getTableName()); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); @@ -317,7 +317,7 @@ public class TestImportExport { ); UTIL.getHBaseAdmin().createTable(desc); t.close(); - t = new HTable(UTIL.getConfiguration(), IMPORT_TABLE); + t = new HTable(UTIL.getConfiguration(), desc.getTableName()); args = new String[] { IMPORT_TABLE, FQ_OUTPUT_DIR @@ -351,7 +351,7 @@ public class TestImportExport { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE)); desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); UTIL.getHBaseAdmin().createTable(desc); - Table exportTable = new HTable(UTIL.getConfiguration(), EXPORT_TABLE); + Table exportTable = new HTable(UTIL.getConfiguration(), desc.getTableName()); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); @@ -378,7 +378,7 @@ public class TestImportExport { desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); UTIL.getHBaseAdmin().createTable(desc); - Table importTable = new HTable(UTIL.getConfiguration(), IMPORT_TABLE); + Table importTable = new HTable(UTIL.getConfiguration(), desc.getTableName()); args = new String[] { "-D" + Import.FILTER_CLASS_CONF_KEY + "=" + PrefixFilter.class.getName(), "-D" + Import.FILTER_ARGS_CONF_KEY + "=" + Bytes.toString(ROW1), IMPORT_TABLE, FQ_OUTPUT_DIR, "1000" }; @@ -544,7 +544,7 @@ public class TestImportExport { public void testDurability() throws IOException, InterruptedException, ClassNotFoundException { // Create an export table. String exportTableName = "exporttestDurability"; - Table exportTable = UTIL.createTable(Bytes.toBytes(exportTableName), FAMILYA, 3); + Table exportTable = UTIL.createTable(TableName.valueOf(exportTableName), FAMILYA, 3); // Insert some data Put put = new Put(ROW1); @@ -565,7 +565,7 @@ public class TestImportExport { // Create the table for import String importTableName = "importTestDurability1"; - Table importTable = UTIL.createTable(Bytes.toBytes(importTableName), FAMILYA, 3); + Table importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3); // Register the hlog listener for the import table TableWALActionListener walListener = new TableWALActionListener(importTableName); @@ -584,7 +584,7 @@ public class TestImportExport { // Run the import with the default durability option importTableName = "importTestDurability2"; - importTable = UTIL.createTable(Bytes.toBytes(importTableName), FAMILYA, 3); + importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3); hLog.unregisterWALActionsListener(walListener); walListener = new TableWALActionListener(importTableName); hLog.registerWALActionsListener(walListener); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java index f31a916ad7a..eddee5a12ad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; @@ -120,7 +121,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable { "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY", "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; String data = "KEY\u001bVALUE1\u001bVALUE2\u001btest=>myvalue\n"; - util.createTable(tableName, FAMILY); + util.createTable(TableName.valueOf(tableName), FAMILY); doMROnTableTest(util, FAMILY, data, args, 1, true); util.deleteTable(tableName); } @@ -136,7 +137,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable { "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY", "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; String data = "KEY\u001bVALUE1\u001bVALUE2\u001btest1=>myvalue\n"; - util.createTable(tableName, FAMILY); + util.createTable(TableName.valueOf(tableName), FAMILY); doMROnTableTest(util, FAMILY, data, args, 1, false); util.deleteTable(tableName); } @@ -177,7 +178,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable { LOG.debug("Running ImportTsv with arguments: " + argv); assertEquals(0, ToolRunner.run(conf, tool, argv.toArray(args))); - validateTable(conf, table, family, valueMultiplier, dataAvailable); + validateTable(conf, TableName.valueOf(table), family, valueMultiplier, dataAvailable); if (conf.getBoolean(DELETE_AFTER_LOAD_CONF, true)) { LOG.debug("Deleting test subdirectory"); @@ -191,7 +192,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable { * * @param dataAvailable */ - private static void validateTable(Configuration conf, String tableName, String family, + private static void validateTable(Configuration conf, TableName tableName, String family, int valueMultiplier, boolean dataAvailable) throws IOException { LOG.debug("Validating table."); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java index b15d04e57f8..0ca0f8ff449 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; @@ -161,20 +162,20 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY", "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n"; - util.createTable(tableName, FAMILY); + util.createTable(TableName.valueOf(tableName), FAMILY); doMROnTableTest(util, FAMILY, data, args, 1); util.deleteTable(tableName); } @Test public void testMROnTableWithDeletes() throws Exception { - String tableName = "test-" + UUID.randomUUID(); + TableName tableName = TableName.valueOf("test-" + UUID.randomUUID()); // Prepare the arguments required for the test. String[] args = new String[] { "-D" + ImportTsv.MAPPER_CONF_KEY + "=org.apache.hadoop.hbase.mapreduce.TsvImporterMapper", "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY", - "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; + "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() }; String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n"; util.createTable(tableName, FAMILY); doMROnTableTest(util, FAMILY, data, args, 1); @@ -182,7 +183,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { util.deleteTable(tableName); } - private void issueDeleteAndVerifyData(String tableName) throws IOException { + private void issueDeleteAndVerifyData(TableName tableName) throws IOException { LOG.debug("Validating table after delete."); Table table = new HTable(conf, tableName); boolean verified = false; @@ -229,7 +230,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY", "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n"; - util.createTable(tableName, FAMILY); + util.createTable(TableName.valueOf(tableName), FAMILY); doMROnTableTest(util, FAMILY, data, args, 1); util.deleteTable(tableName); } @@ -266,7 +267,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY", "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; String data = "KEY\u001bVALUE4\u001bVALUE8\u001bsecret&private\n"; - util.createTable(tableName, FAMILY); + util.createTable(TableName.valueOf(tableName), FAMILY); doMROnTableTest(util, FAMILY, data, args, 1); util.deleteTable(tableName); } @@ -283,12 +284,13 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { */ protected static Tool doMROnTableTest(HBaseTestingUtility util, String family, String data, String[] args, int valueMultiplier) throws Exception { - String table = args[args.length - 1]; + TableName table = TableName.valueOf(args[args.length - 1]); Configuration conf = new Configuration(util.getConfiguration()); // populate input file FileSystem fs = FileSystem.get(conf); - Path inputPath = fs.makeQualified(new Path(util.getDataTestDirOnTestFS(table), "input.dat")); + Path inputPath = fs.makeQualified(new Path(util + .getDataTestDirOnTestFS(table.getNameAsString()), "input.dat")); FSDataOutputStream op = fs.create(inputPath, true); if (data == null) { data = "KEY\u001bVALUE1\u001bVALUE2\n"; @@ -330,7 +332,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { if (conf.getBoolean(DELETE_AFTER_LOAD_CONF, true)) { LOG.debug("Deleting test subdirectory"); - util.cleanupDataTestDirOnTestFS(table); + util.cleanupDataTestDirOnTestFS(table.getNameAsString()); } return tool; } @@ -364,7 +366,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { /** * Confirm ImportTsv via data in online table. */ - private static void validateTable(Configuration conf, String tableName, String family, + private static void validateTable(Configuration conf, TableName tableName, String family, int valueMultiplier) throws IOException { LOG.debug("Validating table."); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java index 8706d9cfd11..e3b34958749 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -111,7 +112,7 @@ public class TestImportTsv implements Configurable { table }; - util.createTable(table, FAMILY); + util.createTable(TableName.valueOf(table), FAMILY); doMROnTableTest(util, FAMILY, null, args, 1); util.deleteTable(table); } @@ -129,7 +130,7 @@ public class TestImportTsv implements Configurable { }; String data = "KEY,1234,VALUE1,VALUE2\n"; - util.createTable(table, FAMILY); + util.createTable(TableName.valueOf(table), FAMILY); doMROnTableTest(util, FAMILY, data, args, 1); util.deleteTable(table); } @@ -146,7 +147,7 @@ public class TestImportTsv implements Configurable { table }; - util.createTable(table, FAMILY); + util.createTable(TableName.valueOf(table), FAMILY); doMROnTableTest(util, FAMILY, null, args, 3); util.deleteTable(table); } @@ -181,7 +182,7 @@ public class TestImportTsv implements Configurable { table }; - util.createTable(table, FAMILY); + util.createTable(TableName.valueOf(table), FAMILY); doMROnTableTest(util, FAMILY, null, args, 3); util.deleteTable(table); } @@ -288,7 +289,7 @@ public class TestImportTsv implements Configurable { if (createdHFiles) validateHFiles(fs, outputPath, family); else - validateTable(conf, table, family, valueMultiplier); + validateTable(conf, TableName.valueOf(table), family, valueMultiplier); if (conf.getBoolean(DELETE_AFTER_LOAD_CONF, true)) { LOG.debug("Deleting test subdirectory"); @@ -300,7 +301,7 @@ public class TestImportTsv implements Configurable { /** * Confirm ImportTsv via data in online table. */ - private static void validateTable(Configuration conf, String tableName, + private static void validateTable(Configuration conf, TableName tableName, String family, int valueMultiplier) throws IOException { LOG.debug("Validating table."); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java index 338202c05f5..f44909a7974 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java @@ -121,10 +121,10 @@ public class TestLoadIncrementalHFilesSplitRecovery { * Creates a table with given table name and specified number of column * families if the table does not already exist. */ - private void setupTable(String table, int cfs) throws IOException { + private void setupTable(TableName table, int cfs) throws IOException { try { LOG.info("Creating table " + table); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); + HTableDescriptor htd = new HTableDescriptor(table); for (int i = 0; i < cfs; i++) { htd.addFamily(new HColumnDescriptor(family(i))); } @@ -142,11 +142,11 @@ public class TestLoadIncrementalHFilesSplitRecovery { * @param cfs * @param SPLIT_KEYS */ - private void setupTableWithSplitkeys(String table, int cfs, byte[][] SPLIT_KEYS) + private void setupTableWithSplitkeys(TableName table, int cfs, byte[][] SPLIT_KEYS) throws IOException { try { LOG.info("Creating table " + table); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); + HTableDescriptor htd = new HTableDescriptor(table); for (int i = 0; i < cfs; i++) { htd.addFamily(new HColumnDescriptor(family(i))); } @@ -157,9 +157,9 @@ public class TestLoadIncrementalHFilesSplitRecovery { } } - private Path buildBulkFiles(String table, int value) throws Exception { - Path dir = util.getDataTestDirOnTestFS(table); - Path bulk1 = new Path(dir, table+value); + private Path buildBulkFiles(TableName table, int value) throws Exception { + Path dir = util.getDataTestDirOnTestFS(table.getNameAsString()); + Path bulk1 = new Path(dir, table.getNameAsString() + value); FileSystem fs = util.getTestFileSystem(); buildHFiles(fs, bulk1, value); return bulk1; @@ -168,26 +168,25 @@ public class TestLoadIncrementalHFilesSplitRecovery { /** * Populate table with known values. */ - private void populateTable(String table, int value) throws Exception { + private void populateTable(TableName table, int value) throws Exception { // create HFiles for different column families LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()); Path bulk1 = buildBulkFiles(table, value); - HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table)); + HTable t = new HTable(util.getConfiguration(), table); lih.doBulkLoad(bulk1, t); } /** * Split the known table in half. (this is hard coded for this test suite) */ - private void forceSplit(String table) { + private void forceSplit(TableName table) { try { // need to call regions server to by synchronous but isn't visible. - HRegionServer hrs = util.getRSForFirstRegionInTable(Bytes - .toBytes(table)); + HRegionServer hrs = util.getRSForFirstRegionInTable(table); for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) { - if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) { + if (hri.getTable().equals(table)) { // splitRegion doesn't work if startkey/endkey are null ProtobufUtil.split(hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2)); // hard code split } @@ -199,7 +198,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { regions = 0; for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) { - if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) { + if (hri.getTable().equals(table)) { regions++; } } @@ -232,10 +231,11 @@ public class TestLoadIncrementalHFilesSplitRecovery { * expected number of rows. * @throws IOException */ - void assertExpectedTable(String table, int count, int value) throws IOException { + void assertExpectedTable(TableName table, int count, int value) throws IOException { Table t = null; try { - assertEquals(util.getHBaseAdmin().listTables(table).length, 1); + assertEquals( + util.getHBaseAdmin().listTables(table.getNameAsString()).length, 1); t = new HTable(util.getConfiguration(), table); Scan s = new Scan(); ResultScanner sr = t.getScanner(s); @@ -262,7 +262,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { */ @Test(expected=IOException.class) public void testBulkLoadPhaseFailure() throws Exception { - String table = "bulkLoadPhaseFailure"; + TableName table = TableName.valueOf("bulkLoadPhaseFailure"); setupTable(table, 10); final AtomicInteger attmptedCalls = new AtomicInteger(); @@ -293,7 +293,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { // create HFiles for different column families Path dir = buildBulkFiles(table, 1); - HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table)); + HTable t = new HTable(util.getConfiguration(), table); lih.doBulkLoad(dir, t); } finally { util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, @@ -334,7 +334,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { */ @Test public void testSplitWhileBulkLoadPhase() throws Exception { - final String table = "splitWhileBulkloadPhase"; + final TableName table = TableName.valueOf("splitWhileBulkloadPhase"); setupTable(table, 10); populateTable(table,1); assertExpectedTable(table, ROWCOUNT, 1); @@ -359,7 +359,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { }; // create HFiles for different column families - HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table)); + HTable t = new HTable(util.getConfiguration(), table); Path bulk = buildBulkFiles(table, 2); lih2.doBulkLoad(bulk, t); @@ -376,7 +376,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { */ @Test public void testGroupOrSplitPresplit() throws Exception { - final String table = "groupOrSplitPresplit"; + final TableName table = TableName.valueOf("groupOrSplitPresplit"); setupTable(table, 10); populateTable(table, 1); assertExpectedTable(table, ROWCOUNT, 1); @@ -399,7 +399,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { // create HFiles for different column families Path bulk = buildBulkFiles(table, 2); - HTable ht = new HTable(util.getConfiguration(), Bytes.toBytes(table)); + HTable ht = new HTable(util.getConfiguration(), table); lih.doBulkLoad(bulk, ht); assertExpectedTable(table, ROWCOUNT, 2); @@ -412,7 +412,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { */ @Test(expected = IOException.class) public void testGroupOrSplitFailure() throws Exception { - String table = "groupOrSplitFailure"; + TableName table = TableName.valueOf("groupOrSplitFailure"); setupTable(table, 10); LoadIncrementalHFiles lih = new LoadIncrementalHFiles( @@ -434,7 +434,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { // create HFiles for different column families Path dir = buildBulkFiles(table,1); - HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table)); + HTable t = new HTable(util.getConfiguration(), table); lih.doBulkLoad(dir, t); fail("doBulkLoad should have thrown an exception"); @@ -442,9 +442,9 @@ public class TestLoadIncrementalHFilesSplitRecovery { @Test public void testGroupOrSplitWhenRegionHoleExistsInMeta() throws Exception { - String tableName = "testGroupOrSplitWhenRegionHoleExistsInMeta"; + TableName tableName = TableName.valueOf("testGroupOrSplitWhenRegionHoleExistsInMeta"); byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000100") }; - HTable table = new HTable(util.getConfiguration(), Bytes.toBytes(tableName)); + HTable table = new HTable(util.getConfiguration(), tableName); setupTableWithSplitkeys(tableName, 10, SPLIT_KEYS); Path dir = buildBulkFiles(tableName, 2); @@ -479,7 +479,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { // Mess it up by leaving a hole in the hbase:meta HConnection hConnection = HConnectionManager.getConnection(util.getConfiguration()); List regionInfos = MetaTableAccessor.getTableRegions( - hConnection, TableName.valueOf(tableName)); + hConnection, tableName); for (HRegionInfo regionInfo : regionInfos) { if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { MetaTableAccessor.deleteRegion(hConnection, regionInfo); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java index a78b859e192..ae1ac8ffdc4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; @@ -76,8 +77,7 @@ public class TestMultiTableInputFormat { // create and fill table for (int i = 0; i < 3; i++) { HTable table = - TEST_UTIL.createTable(Bytes.toBytes(TABLE_NAME + String.valueOf(i)), - INPUT_FAMILY); + TEST_UTIL.createTable(TableName.valueOf(TABLE_NAME + String.valueOf(i)), INPUT_FAMILY); TEST_UTIL.createMultiRegions(TEST_UTIL.getConfiguration(), table, INPUT_FAMILY, 4); TEST_UTIL.loadTable(table, INPUT_FAMILY, false); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java index e63e746e55d..e42d135d8c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java @@ -59,7 +59,7 @@ public class TestMultithreadedTableMapper { private static final Log LOG = LogFactory.getLog(TestMultithreadedTableMapper.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - static final byte[] MULTI_REGION_TABLE_NAME = Bytes.toBytes("mrtest"); + static final TableName MULTI_REGION_TABLE_NAME = TableName.valueOf("mrtest"); static final byte[] INPUT_FAMILY = Bytes.toBytes("contents"); static final byte[] OUTPUT_FAMILY = Bytes.toBytes("text"); static final int NUMBER_OF_THREADS = 10; @@ -139,7 +139,7 @@ public class TestMultithreadedTableMapper { Scan scan = new Scan(); scan.addFamily(INPUT_FAMILY); TableMapReduceUtil.initTableMapperJob( - Bytes.toString(table.getTableName()), scan, + table.getTableName(), scan, MultithreadedTableMapper.class, ImmutableBytesWritable.class, Put.class, job); MultithreadedTableMapper.setMapperClass(job, ProcessContentsMapper.class); @@ -148,11 +148,11 @@ public class TestMultithreadedTableMapper { Bytes.toString(table.getTableName()), IdentityTableReducer.class, job); FileOutputFormat.setOutputPath(job, new Path("test")); - LOG.info("Started " + Bytes.toString(table.getTableName())); + LOG.info("Started " + table.getTableName()); assertTrue(job.waitForCompletion(true)); LOG.info("After map/reduce completion"); // verify map-reduce results - verify(Bytes.toString(table.getTableName())); + verify(table.getName()); } finally { table.close(); if (job != null) { @@ -162,7 +162,7 @@ public class TestMultithreadedTableMapper { } } - private void verify(String tableName) throws IOException { + private void verify(TableName tableName) throws IOException { Table table = new HTable(new Configuration(UTIL.getConfiguration()), tableName); boolean verified = false; long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java index 592992fc262..99fdfd44afe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java @@ -31,6 +31,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.client.Put; @@ -68,8 +69,7 @@ public class TestRowCounter { public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(); TEST_UTIL.startMiniMapReduceCluster(); - Table table = TEST_UTIL.createTable(Bytes.toBytes(TABLE_NAME), - Bytes.toBytes(COL_FAM)); + Table table = TEST_UTIL.createTable(TableName.valueOf(TABLE_NAME), Bytes.toBytes(COL_FAM)); writeRows(table); table.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java index 697289e236e..750ea39f293 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java @@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; @@ -77,7 +78,7 @@ public abstract class TestTableInputFormatScanBase { // start mini hbase cluster TEST_UTIL.startMiniCluster(3); // create and fill table - table = TEST_UTIL.createTable(TABLE_NAME, INPUT_FAMILY); + table = TEST_UTIL.createTable(TableName.valueOf(TABLE_NAME), INPUT_FAMILY); TEST_UTIL.createMultiRegions(table, INPUT_FAMILY); TEST_UTIL.loadTable(table, INPUT_FAMILY, false); // start MR cluster diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java index fa345f2211b..11a35f0174c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java @@ -111,7 +111,7 @@ public class TestTableMapReduce extends TestTableMapReduceBase { LOG.info("After map/reduce completion"); // verify map-reduce results - verify(Bytes.toString(table.getTableName())); + verify(table.getName()); } catch (InterruptedException e) { throw new IOException(e); } catch (ClassNotFoundException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java index 789c87455bf..c218bc926ba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -132,7 +133,7 @@ public abstract class TestTableMapReduceBase { return outval; } - protected void verify(String tableName) throws IOException { + protected void verify(TableName tableName) throws IOException { Table table = new HTable(UTIL.getConfiguration(), tableName); boolean verified = false; long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java index f227692bbbc..bde3bc6fc29 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java @@ -87,8 +87,8 @@ public class TestWALPlayer { */ @Test public void testWALPlayer() throws Exception { - final byte[] TABLENAME1 = Bytes.toBytes("testWALPlayer1"); - final byte[] TABLENAME2 = Bytes.toBytes("testWALPlayer2"); + final TableName TABLENAME1 = TableName.valueOf("testWALPlayer1"); + final TableName TABLENAME2 = TableName.valueOf("testWALPlayer2"); final byte[] FAMILY = Bytes.toBytes("family"); final byte[] COLUMN1 = Bytes.toBytes("c1"); final byte[] COLUMN2 = Bytes.toBytes("c2"); @@ -118,8 +118,8 @@ public class TestWALPlayer { configuration.set(optionName, "1000"); player.setupTime(configuration, optionName); assertEquals(1000,configuration.getLong(optionName,0)); - assertEquals(0, player.run(new String[] { walInputDir, Bytes.toString(TABLENAME1), - Bytes.toString(TABLENAME2) })); + assertEquals(0, player.run(new String[] {walInputDir, TABLENAME1.getNameAsString(), + TABLENAME2.getNameAsString() })); // verify the WAL was player into table 2 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java index 0af95b9001e..7ab47ee437d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java @@ -218,7 +218,7 @@ public class TestMasterFailover { assertTrue(master.isInitialized()); // Create a table with a region online - RegionLocator onlineTable = TEST_UTIL.createTable("onlineTable", "family"); + RegionLocator onlineTable = TEST_UTIL.createTable(TableName.valueOf("onlineTable"), "family"); // Create a table in META, so it has a region offline HTableDescriptor offlineTable = new HTableDescriptor( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index 5f9f0b8d15d..374366ef1e6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -52,7 +52,7 @@ import org.junit.experimental.categories.Category; public class TestMasterTransitions { private static final Log LOG = LogFactory.getLog(TestMasterTransitions.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static final String TABLENAME = "master_transitions"; + private static final TableName TABLENAME = TableName.valueOf("master_transitions"); private static final byte [][] FAMILIES = new byte [][] {Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")}; @@ -64,11 +64,10 @@ public class TestMasterTransitions { TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true); TEST_UTIL.startMiniCluster(2); // Create a table of three families. This will assign a region. - TableName tableName = TableName.valueOf(TABLENAME); - TEST_UTIL.createTable(tableName, FAMILIES); + TEST_UTIL.createTable(TABLENAME, FAMILIES); HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME); int countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily()); - TEST_UTIL.waitUntilAllRegionsAssigned(tableName); + TEST_UTIL.waitUntilAllRegionsAssigned(TABLENAME); addToEachStartKey(countOfRegions); t.close(); } @@ -491,12 +490,12 @@ public class TestMasterTransitions { for (Result r = null; (r = s.next()) != null;) { HRegionInfo hri = HRegionInfo.getHRegionInfo(r); if (hri == null) break; - if (!hri.getTable().getNameAsString().equals(TABLENAME)) { + if (!hri.getTable().equals(TABLENAME)) { continue; } // If start key, add 'aaa'. - if(!hri.getTable().getNameAsString().equals(TABLENAME)) { + if(!hri.getTable().equals(TABLENAME)) { continue; } byte [] row = getStartKey(hri); @@ -529,4 +528,4 @@ public class TestMasterTransitions { private static byte [] getTestQualifier() { return getTestFamily(); } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java index 1f672d3643a..0b1ccb2e840 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java @@ -50,10 +50,10 @@ public class TestRestartCluster { private static final Log LOG = LogFactory.getLog(TestRestartCluster.class); private HBaseTestingUtility UTIL = new HBaseTestingUtility(); - private static final byte [][] TABLES = { - Bytes.toBytes("restartTableOne"), - Bytes.toBytes("restartTableTwo"), - Bytes.toBytes("restartTableThree") + private static final TableName[] TABLES = { + TableName.valueOf("restartTableOne"), + TableName.valueOf("restartTableTwo"), + TableName.valueOf("restartTableThree") }; private static final byte [] FAMILY = Bytes.toBytes("family"); @@ -68,11 +68,11 @@ public class TestRestartCluster { Threads.sleep(1); } LOG.info("\n\nCreating tables"); - for(byte [] TABLE : TABLES) { + for(TableName TABLE : TABLES) { UTIL.createTable(TABLE, FAMILY); } - for(byte [] TABLE : TABLES) { - UTIL.waitTableEnabled(TABLE); + for(TableName TABLE : TABLES) { + UTIL.waitTableEnabled(TABLE.getName()); } List allRegions = @@ -94,14 +94,14 @@ public class TestRestartCluster { allRegions = MetaScanner.listAllRegions(new Configuration(UTIL.getConfiguration()), true); assertEquals(4, allRegions.size()); LOG.info("\n\nWaiting for tables to be available"); - for(byte [] TABLE: TABLES) { + for(TableName TABLE: TABLES) { try { UTIL.createTable(TABLE, FAMILY); assertTrue("Able to create table that should already exist", false); } catch(TableExistsException tee) { LOG.info("Table already exists as expected"); } - UTIL.waitTableAvailable(TABLE); + UTIL.waitTableAvailable(TABLE.getName()); } } @@ -118,11 +118,11 @@ public class TestRestartCluster { UTIL.getMiniHBaseCluster().getMaster(). getMasterRpcServices().synchronousBalanceSwitch(false); LOG.info("\n\nCreating tables"); - for(byte [] TABLE : TABLES) { + for(TableName TABLE : TABLES) { UTIL.createTable(TABLE, FAMILY); } - for(byte [] TABLE : TABLES) { - UTIL.waitTableEnabled(TABLE); + for(TableName TABLE : TABLES) { + UTIL.waitTableEnabled(TABLE.getName()); } HMaster master = UTIL.getMiniHBaseCluster().getMaster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java index 0f3db18ae1e..4311b2982ed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java @@ -81,7 +81,7 @@ public class TestCompactionState { @Test public void testInvalidColumnFamily() throws IOException, InterruptedException { - byte [] table = Bytes.toBytes("testInvalidColumnFamily"); + TableName table = TableName.valueOf("testInvalidColumnFamily"); byte [] family = Bytes.toBytes("family"); byte [] fakecf = Bytes.toBytes("fakecf"); boolean caughtMinorCompact = false; @@ -124,8 +124,7 @@ public class TestCompactionState { final CompactionState expectedState, boolean singleFamily) throws IOException, InterruptedException { // Create a table with regions - TableName table = - TableName.valueOf(tableName); + TableName table = TableName.valueOf(tableName); byte [] family = Bytes.toBytes("family"); byte [][] families = {family, Bytes.add(family, Bytes.toBytes("2")), Bytes.add(family, Bytes.toBytes("3"))}; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java index d0b7ea37882..efae472b24f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java @@ -52,7 +52,7 @@ public class TestEncryptionRandomKeying { private static Configuration conf = TEST_UTIL.getConfiguration(); private static HTableDescriptor htd; - private static List findStorefilePaths(byte[] tableName) throws Exception { + private static List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); for (HRegion region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { @@ -103,7 +103,7 @@ public class TestEncryptionRandomKeying { TEST_UTIL.waitTableAvailable(htd.getName(), 5000); // Create a store file - Table table = new HTable(conf, htd.getName()); + Table table = new HTable(conf, htd.getTableName()); try { table.put(new Put(Bytes.toBytes("testrow")) .add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value"))); @@ -121,7 +121,7 @@ public class TestEncryptionRandomKeying { @Test public void testRandomKeying() throws Exception { // Verify we have store file(s) with a random key - final List initialPaths = findStorefilePaths(htd.getName()); + final List initialPaths = findStorefilePaths(htd.getTableName()); assertTrue(initialPaths.size() > 0); for (Path path: initialPaths) { assertNotNull("Store file " + path + " is not encrypted", extractHFileKey(path)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java index 91c62eda524..0e94e68cfb2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java @@ -133,9 +133,9 @@ public class TestHRegionServerBulkLoad { public static class AtomicHFileLoader extends RepeatingTestThread { final AtomicLong numBulkLoads = new AtomicLong(); final AtomicLong numCompactions = new AtomicLong(); - private String tableName; + private TableName tableName; - public AtomicHFileLoader(String tableName, TestContext ctx, + public AtomicHFileLoader(TableName tableName, TestContext ctx, byte targetFamilies[][]) throws IOException { super(ctx); this.tableName = tableName; @@ -160,9 +160,8 @@ public class TestHRegionServerBulkLoad { // bulk load HFiles final HConnection conn = UTIL.getHBaseAdmin().getConnection(); - TableName tbl = TableName.valueOf(tableName); RegionServerCallable callable = - new RegionServerCallable(conn, tbl, Bytes.toBytes("aaa")) { + new RegionServerCallable(conn, tableName, Bytes.toBytes("aaa")) { @Override public Void call(int callTimeout) throws Exception { LOG.debug("Going to connect to server " + getLocation() + " for row " @@ -181,7 +180,7 @@ public class TestHRegionServerBulkLoad { // Periodically do compaction to reduce the number of open file handles. if (numBulkLoads.get() % 10 == 0) { // 10 * 50 = 500 open file handles! - callable = new RegionServerCallable(conn, tbl, Bytes.toBytes("aaa")) { + callable = new RegionServerCallable(conn, tableName, Bytes.toBytes("aaa")) { @Override public Void call(int callTimeout) throws Exception { LOG.debug("compacting " + getLocation() + " for row " @@ -210,9 +209,9 @@ public class TestHRegionServerBulkLoad { HTable table; AtomicLong numScans = new AtomicLong(); AtomicLong numRowsScanned = new AtomicLong(); - String TABLE_NAME; + TableName TABLE_NAME; - public AtomicScanReader(String TABLE_NAME, TestContext ctx, + public AtomicScanReader(TableName TABLE_NAME, TestContext ctx, byte targetFamilies[][]) throws IOException { super(ctx); this.TABLE_NAME = TABLE_NAME; @@ -264,10 +263,10 @@ public class TestHRegionServerBulkLoad { * Creates a table with given table name and specified number of column * families if the table does not already exist. */ - private void setupTable(String table, int cfs) throws IOException { + private void setupTable(TableName table, int cfs) throws IOException { try { LOG.info("Creating table " + table); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); + HTableDescriptor htd = new HTableDescriptor(table); for (int i = 0; i < 10; i++) { htd.addFamily(new HColumnDescriptor(family(i))); } @@ -283,7 +282,7 @@ public class TestHRegionServerBulkLoad { */ @Test public void testAtomicBulkLoad() throws Exception { - String TABLE_NAME = "atomicBulkLoad"; + TableName TABLE_NAME = TableName.valueOf("atomicBulkLoad"); int millisToRun = 30000; int numScanners = 50; @@ -296,7 +295,7 @@ public class TestHRegionServerBulkLoad { } } - void runAtomicBulkloadTest(String tableName, int millisToRun, int numScanners) + void runAtomicBulkloadTest(TableName tableName, int millisToRun, int numScanners) throws Exception { setupTable(tableName, 10); @@ -336,7 +335,7 @@ public class TestHRegionServerBulkLoad { Configuration c = HBaseConfiguration.create(); TestHRegionServerBulkLoad test = new TestHRegionServerBulkLoad(); test.setConf(c); - test.runAtomicBulkloadTest("atomicTableTest", 5 * 60 * 1000, 50); + test.runAtomicBulkloadTest(TableName.valueOf("atomicTableTest"), 5 * 60 * 1000, 50); } finally { System.exit(0); // something hangs (believe it is lru threadpool) } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 9388d5fb1c4..6e4030b32fa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -77,7 +77,7 @@ public class TestRegionReplicas { @BeforeClass public static void before() throws Exception { HTU.startMiniCluster(NB_SERVERS); - final byte[] tableName = Bytes.toBytes(TestRegionReplicas.class.getSimpleName()); + final TableName tableName = TableName.valueOf(TestRegionReplicas.class.getSimpleName()); // Create table then get the single region for our new table. table = HTU.createTable(tableName, f); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index 36baa05a9bf..3ae82ee3b62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -89,7 +89,7 @@ public class TestRegionServerMetrics { String regionMetricsKey = "regionCount"; long regions = metricsHelper.getGaugeLong(regionMetricsKey, serverSource); // Creating a table should add one region - TEST_UTIL.createTable(Bytes.toBytes("table"), Bytes.toBytes("cf")); + TEST_UTIL.createTable(TableName.valueOf("table"), Bytes.toBytes("cf")); metricsHelper.assertGaugeGt(regionMetricsKey, regions, serverSource); } @@ -101,7 +101,7 @@ public class TestRegionServerMetrics { @Test public void testRequestCount() throws Exception { String tableNameString = "testRequestCount"; - byte[] tName = Bytes.toBytes(tableNameString); + TableName tName = TableName.valueOf(tableNameString); byte[] cfName = Bytes.toBytes("d"); byte[] row = Bytes.toBytes("rk"); byte[] qualifier = Bytes.toBytes("qual"); @@ -181,7 +181,7 @@ public class TestRegionServerMetrics { @Test public void testMutationsWithoutWal() throws Exception { - byte[] tableName = Bytes.toBytes("testMutationsWithoutWal"); + TableName tableName = TableName.valueOf("testMutationsWithoutWal"); byte[] cf = Bytes.toBytes("d"); byte[] row = Bytes.toBytes("rk"); byte[] qualifier = Bytes.toBytes("qual"); @@ -240,7 +240,7 @@ public class TestRegionServerMetrics { @Test public void testCheckAndPutCount() throws Exception { String tableNameString = "testCheckAndPutCount"; - byte[] tableName = Bytes.toBytes(tableNameString); + TableName tableName = TableName.valueOf(tableNameString); byte[] cf = Bytes.toBytes("d"); byte[] row = Bytes.toBytes("rk"); byte[] qualifier = Bytes.toBytes("qual"); @@ -276,7 +276,7 @@ public class TestRegionServerMetrics { @Test public void testIncrement() throws Exception { String tableNameString = "testIncrement"; - byte[] tableName = Bytes.toBytes(tableNameString); + TableName tableName = TableName.valueOf(tableNameString); byte[] cf = Bytes.toBytes("d"); byte[] row = Bytes.toBytes("rk"); byte[] qualifier = Bytes.toBytes("qual"); @@ -308,7 +308,7 @@ public class TestRegionServerMetrics { @Test public void testAppend() throws Exception { String tableNameString = "testAppend"; - byte[] tableName = Bytes.toBytes(tableNameString); + TableName tableName = TableName.valueOf(tableNameString); byte[] cf = Bytes.toBytes("d"); byte[] row = Bytes.toBytes("rk"); byte[] qualifier = Bytes.toBytes("qual"); @@ -340,7 +340,7 @@ public class TestRegionServerMetrics { @Test public void testScanNext() throws IOException { String tableNameString = "testScanNext"; - byte[] tableName = Bytes.toBytes(tableNameString); + TableName tableName = TableName.valueOf(tableNameString); byte[] cf = Bytes.toBytes("d"); byte[] qualifier = Bytes.toBytes("qual"); byte[] val = Bytes.toBytes("One"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java index 8c7d36e9155..24d9dd45937 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.master.HMaster; @@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; @@ -70,10 +70,10 @@ public class TestRegionServerNoMaster { @BeforeClass public static void before() throws Exception { HTU.startMiniCluster(NB_SERVERS); - final byte[] tableName = Bytes.toBytes(TestRegionServerNoMaster.class.getSimpleName()); + final TableName tableName = TableName.valueOf(TestRegionServerNoMaster.class.getSimpleName()); // Create table then get the single region for our new table. - table = HTU.createTable(tableName, HConstants.CATALOG_FAMILY); + table = HTU.createTable(tableName,HConstants.CATALOG_FAMILY); Put p = new Put(row); p.add(HConstants.CATALOG_FAMILY, row, row); table.put(p); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java index 1a2fc63ce08..5593d802815 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java @@ -31,6 +31,7 @@ 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.TableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableNotFoundException; @@ -59,7 +60,7 @@ public class TestScannerWithBulkload { TEST_UTIL.startMiniCluster(1); } - private static void createTable(Admin admin, String tableName) throws IOException { + private static void createTable(Admin admin, TableName tableName) throws IOException { HTableDescriptor desc = new HTableDescriptor(tableName); HColumnDescriptor hcd = new HColumnDescriptor("col"); hcd.setMaxVersions(3); @@ -69,7 +70,7 @@ public class TestScannerWithBulkload { @Test public void testBulkLoad() throws Exception { - String tableName = "testBulkLoad"; + TableName tableName = TableName.valueOf("testBulkLoad"); long l = System.currentTimeMillis(); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); createTable(admin, tableName); @@ -164,7 +165,7 @@ public class TestScannerWithBulkload { return hfilePath; } - private HTable init(HBaseAdmin admin, long l, Scan scan, String tableName) throws Exception { + private HTable init(HBaseAdmin admin, long l, Scan scan, TableName tableName) throws Exception { HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); Put put0 = new Put(Bytes.toBytes("row1")); put0.add(new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("col"), Bytes.toBytes("q"), l, Bytes @@ -198,7 +199,7 @@ public class TestScannerWithBulkload { @Test public void testBulkLoadWithParallelScan() throws Exception { - String tableName = "testBulkLoadWithParallelScan"; + TableName tableName = TableName.valueOf("testBulkLoadWithParallelScan"); final long l = System.currentTimeMillis(); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); createTable(admin, tableName); @@ -240,7 +241,7 @@ public class TestScannerWithBulkload { @Test public void testBulkLoadNativeHFile() throws Exception { - String tableName = "testBulkLoadNativeHFile"; + TableName tableName = TableName.valueOf("testBulkLoadNativeHFile"); long l = System.currentTimeMillis(); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); createTable(admin, tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java index 158f05e7c0f..881699d8459 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.ServerName; @@ -134,7 +135,7 @@ public class TestServerCustomProtocol { } } - private static final byte[] TEST_TABLE = Bytes.toBytes("test"); + private static final TableName TEST_TABLE = TableName.valueOf("test"); private static final byte[] TEST_FAMILY = Bytes.toBytes("f1"); private static final byte[] ROW_A = Bytes.toBytes("aaa"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 45dd273c9b1..cde1c6fd799 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -177,10 +177,10 @@ public class TestSplitTransactionOnCluster { TableName.valueOf("testRITStateForRollback"); try { // Create table then get the single region for our new table. - Table t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf")); + Table t = createTableAndWait(tableName, Bytes.toBytes("cf")); final List regions = cluster.getRegions(tableName); final HRegionInfo hri = getAndCheckSingleTableRegion(regions); - insertData(tableName.getName(), admin, t); + insertData(tableName, admin, t); t.close(); // Turn off balancer so it doesn't cut in and mess up our placements. @@ -219,7 +219,7 @@ public class TestSplitTransactionOnCluster { } @Test(timeout = 60000) public void testSplitFailedCompactionAndSplit() throws Exception { - final byte[] tableName = Bytes.toBytes("testSplitFailedCompactionAndSplit"); + final TableName tableName = TableName.valueOf("testSplitFailedCompactionAndSplit"); Configuration conf = TESTING_UTIL.getConfiguration(); HBaseAdmin admin = new HBaseAdmin(conf); // Create table then get the single region for our new table. @@ -284,8 +284,8 @@ public class TestSplitTransactionOnCluster { */ @Test (timeout = 300000) public void testRSSplitDaughtersAreOnlinedAfterShutdownHandling() throws IOException, InterruptedException, ServiceException { - final byte [] tableName = - Bytes.toBytes("testRSSplitDaughtersAreOnlinedAfterShutdownHandling"); + final TableName tableName = + TableName.valueOf("testRSSplitDaughtersAreOnlinedAfterShutdownHandling"); // Create table then get the single region for our new table. HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY); @@ -329,8 +329,8 @@ public class TestSplitTransactionOnCluster { @Test (timeout = 300000) public void testExistingZnodeBlocksSplitAndWeRollback() throws IOException, InterruptedException, NodeExistsException, KeeperException, ServiceException { - final byte [] tableName = - Bytes.toBytes("testExistingZnodeBlocksSplitAndWeRollback"); + final TableName tableName = + TableName.valueOf("testExistingZnodeBlocksSplitAndWeRollback"); // Create table then get the single region for our new table. HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY); @@ -387,8 +387,8 @@ public class TestSplitTransactionOnCluster { */ @Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit() throws IOException, InterruptedException { - final byte [] tableName = - Bytes.toBytes("testShutdownFixupWhenDaughterHasSplit"); + final TableName tableName = + TableName.valueOf("testShutdownFixupWhenDaughterHasSplit"); // Create table then get the single region for our new table. HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY); @@ -554,7 +554,8 @@ public class TestSplitTransactionOnCluster { public void testMasterRestartAtRegionSplitPendingCatalogJanitor() throws IOException, InterruptedException, NodeExistsException, KeeperException, ServiceException { - final byte[] tableName = Bytes.toBytes("testMasterRestartAtRegionSplitPendingCatalogJanitor"); + final TableName tableName = TableName + .valueOf("testMasterRestartAtRegionSplitPendingCatalogJanitor"); // Create table then get the single region for our new table. HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY); @@ -622,13 +623,13 @@ public class TestSplitTransactionOnCluster { final TableName tableName = TableName.valueOf("testTableExistsIfTheSpecifiedTableRegionIsSplitParent"); // Create table then get the single region for our new table. - Table t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf")); + Table t = createTableAndWait(tableName, Bytes.toBytes("cf")); List regions = null; try { regions = cluster.getRegions(tableName); int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); - insertData(tableName.getName(), admin, t); + insertData(tableName, admin, t); // Turn off balancer so it doesn't cut in and mess up our placements. admin.setBalancerRunning(false, true); // Turn off the meta scanner so it don't remove parent on us. @@ -674,7 +675,7 @@ public class TestSplitTransactionOnCluster { try { int regionServerIndex = cluster.getServerWith(oldRegions.get(0).getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); - insertData(tableName.getName(), admin, t); + insertData(tableName, admin, t); // Turn off balancer so it doesn't cut in and mess up our placements. admin.setBalancerRunning(false, true); // Turn off the meta scanner so it don't remove parent on us. @@ -731,7 +732,7 @@ public class TestSplitTransactionOnCluster { } } - private void insertData(final byte[] tableName, HBaseAdmin admin, Table t) throws IOException, + private void insertData(final TableName tableName, HBaseAdmin admin, Table t) throws IOException, InterruptedException { Put p = new Put(Bytes.toBytes("row1")); p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("1")); @@ -758,7 +759,7 @@ public class TestSplitTransactionOnCluster { final TableName tableName = TableName.valueOf("testSplitRegionWithNoStoreFiles"); // Create table then get the single region for our new table. - createTableAndWait(tableName.getName(), HConstants.CATALOG_FAMILY); + createTableAndWait(tableName, HConstants.CATALOG_FAMILY); List regions = cluster.getRegions(tableName); HRegionInfo hri = getAndCheckSingleTableRegion(regions); ensureTableRegionNotOnSameServerAsMeta(admin, hri); @@ -841,7 +842,7 @@ public class TestSplitTransactionOnCluster { LOG.info("Starting testSplitAndRestartingMaster"); final TableName tableName = TableName.valueOf("testSplitAndRestartingMaster"); // Create table then get the single region for our new table. - createTableAndWait(tableName.getName(), HConstants.CATALOG_FAMILY); + createTableAndWait(tableName, HConstants.CATALOG_FAMILY); List regions = cluster.getRegions(tableName); HRegionInfo hri = getAndCheckSingleTableRegion(regions); ensureTableRegionNotOnSameServerAsMeta(admin, hri); @@ -880,19 +881,19 @@ public class TestSplitTransactionOnCluster { @Test(timeout = 180000) public void testSplitHooksBeforeAndAfterPONR() throws Exception { - String firstTable = "testSplitHooksBeforeAndAfterPONR_1"; - String secondTable = "testSplitHooksBeforeAndAfterPONR_2"; - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(firstTable)); + TableName firstTable = TableName.valueOf("testSplitHooksBeforeAndAfterPONR_1"); + TableName secondTable = TableName.valueOf("testSplitHooksBeforeAndAfterPONR_2"); + HTableDescriptor desc = new HTableDescriptor(firstTable); desc.addCoprocessor(MockedRegionObserver.class.getName()); HColumnDescriptor hcd = new HColumnDescriptor("cf"); desc.addFamily(hcd); admin.createTable(desc); - desc = new HTableDescriptor(TableName.valueOf(secondTable)); + desc = new HTableDescriptor(secondTable); hcd = new HColumnDescriptor("cf"); desc.addFamily(hcd); admin.createTable(desc); - List firstTableregions = cluster.getRegions(TableName.valueOf(firstTable)); - List secondTableRegions = cluster.getRegions(TableName.valueOf(secondTable)); + List firstTableregions = cluster.getRegions(firstTable); + List secondTableRegions = cluster.getRegions(secondTable); ServerName serverName = cluster.getServerHoldingRegion(firstTableregions.get(0).getRegionName()); admin.move(secondTableRegions.get(0).getRegionInfo().getEncodedNameAsBytes(), @@ -902,16 +903,16 @@ public class TestSplitTransactionOnCluster { try { table1 = new HTable(TESTING_UTIL.getConfiguration(), firstTable); table2 = new HTable(TESTING_UTIL.getConfiguration(), firstTable); - insertData(Bytes.toBytes(firstTable), admin, table1); - insertData(Bytes.toBytes(secondTable), admin, table2); - admin.split(Bytes.toBytes(firstTable), "row2".getBytes()); - firstTableregions = cluster.getRegions(Bytes.toBytes(firstTable)); + insertData(firstTable, admin, table1); + insertData(secondTable, admin, table2); + admin.split(firstTable, "row2".getBytes()); + firstTableregions = cluster.getRegions(firstTable); while (firstTableregions.size() != 2) { Thread.sleep(1000); - firstTableregions = cluster.getRegions(Bytes.toBytes(firstTable)); + firstTableregions = cluster.getRegions(firstTable); } assertEquals("Number of regions after split should be 2.", 2, firstTableregions.size()); - secondTableRegions = cluster.getRegions(Bytes.toBytes(secondTable)); + secondTableRegions = cluster.getRegions(secondTable); assertEquals("Number of regions after split should be 2.", 2, secondTableRegions.size()); } finally { if (table1 != null) { @@ -926,7 +927,7 @@ public class TestSplitTransactionOnCluster { } private void testSplitBeforeSettingSplittingInZKInternals() throws Exception { - final byte[] tableName = Bytes.toBytes("testSplitBeforeSettingSplittingInZK"); + final TableName tableName = TableName.valueOf("testSplitBeforeSettingSplittingInZK"); try { // Create table then get the single region for our new table. createTableAndWait(tableName, Bytes.toBytes("cf")); @@ -1001,7 +1002,7 @@ public class TestSplitTransactionOnCluster { return(null); } - private List checkAndGetDaughters(byte[] tableName) + private List checkAndGetDaughters(TableName tableName) throws InterruptedException { List daughters = null; // try up to 10s @@ -1126,7 +1127,7 @@ public class TestSplitTransactionOnCluster { || cluster.getLiveRegionServerThreads().size() > NB_SERVERS); } - private void awaitDaughters(byte[] tableName, int numDaughters) throws InterruptedException { + private void awaitDaughters(TableName tableName, int numDaughters) throws InterruptedException { // Wait till regions are back on line again. for (int i=0; cluster.getRegions(tableName).size() < numDaughters && i<60; i++) { LOG.info("Waiting for repair to happen"); @@ -1137,7 +1138,7 @@ public class TestSplitTransactionOnCluster { } } - private List awaitTableRegions(final byte[] tableName) throws InterruptedException { + private List awaitTableRegions(final TableName tableName) throws InterruptedException { List regions = null; for (int i = 0; i < 100; i++) { regions = cluster.getRegions(tableName); @@ -1147,11 +1148,11 @@ public class TestSplitTransactionOnCluster { return regions; } - private HTable createTableAndWait(byte[] tableName, byte[] cf) throws IOException, + private HTable createTableAndWait(TableName tableName, byte[] cf) throws IOException, InterruptedException { HTable t = TESTING_UTIL.createTable(tableName, cf); awaitTableRegions(tableName); - assertTrue("Table not online: " + Bytes.toString(tableName), + assertTrue("Table not online: " + tableName, cluster.getRegions(tableName).size() != 0); return t; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 9b7ce0eb9e0..9bcdd82be47 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -126,15 +126,15 @@ public class TestLogRollAbort { TableName.META_TABLE_NAME).close(); // Create the test table and open it - String tableName = this.getClass().getSimpleName(); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + TableName tableName = TableName.valueOf(this.getClass().getSimpleName()); + HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); try { - HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); HLog log = server.getWAL(); assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java index cb59d17a1fb..3357e08f7df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.client.HTable; @@ -72,13 +73,12 @@ public class TestLogRollPeriod { */ @Test public void testNoEdits() throws Exception { - final String tableName = "TestLogRollPeriodNoEdits"; - + TableName tableName = TableName.valueOf("TestLogRollPeriodNoEdits"); TEST_UTIL.createTable(tableName, "cf"); try { Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); try { - HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); HLog log = server.getWAL(); checkMinLogRolls(log, 5); } finally { @@ -94,12 +94,12 @@ public class TestLogRollPeriod { */ @Test(timeout=60000) public void testWithEdits() throws Exception { - final String tableName = "TestLogRollPeriodWithEdits"; + final TableName tableName = TableName.valueOf("TestLogRollPeriodWithEdits"); final String family = "cf"; TEST_UTIL.createTable(tableName, family); try { - HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); HLog log = server.getWAL(); final Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index fbd60d29c65..a3349a15dae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -179,7 +179,7 @@ public class TestLogRolling { Table table = createTestTable(this.tableName); - server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); this.log = server.getWAL(); for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls doPut(table, i); @@ -332,15 +332,14 @@ public class TestLogRolling { this.log = server.getWAL(); // Create the test table and open it - String tableName = getName(); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getName())); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); assertTrue(table.isAutoFlush()); - server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); this.log = server.getWAL(); assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); @@ -440,14 +439,13 @@ public class TestLogRolling { this.log = server.getWAL(); // Create the test table and open it - String tableName = getName(); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getName())); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + HTable table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); - server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); this.log = server.getWAL(); final List paths = new ArrayList(); final List preLogRolledCalled = new ArrayList(); @@ -599,12 +597,10 @@ public class TestLogRolling { // When the hbase:meta table can be opened, the region servers are running Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); try { - String tableName = getName(); - table = createTestTable(tableName); - String tableName2 = tableName + "1"; - table2 = createTestTable(tableName2); + table = createTestTable(getName()); + table2 = createTestTable(getName() + "1"); - server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); this.log = server.getWAL(); FSHLog fshLog = (FSHLog)log; HRegion region = server.getOnlineRegions(table2.getName()).get(0); @@ -662,7 +658,7 @@ public class TestLogRolling { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - return new HTable(TEST_UTIL.getConfiguration(), tableName); + return new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java index f6ad42236b8..b87e7efd796 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -62,10 +63,10 @@ public class TestReplicationSink { private static ReplicationSink SINK; - private static final byte[] TABLE_NAME1 = - Bytes.toBytes("table1"); - private static final byte[] TABLE_NAME2 = - Bytes.toBytes("table2"); + private static final TableName TABLE_NAME1 = + TableName.valueOf("table1"); + private static final TableName TABLE_NAME2 = + TableName.valueOf("table2"); private static final byte[] FAM_NAME1 = Bytes.toBytes("info1"); private static final byte[] FAM_NAME2 = Bytes.toBytes("info2"); @@ -233,8 +234,8 @@ public class TestReplicationSink { assertEquals(0, res.size()); } - private WALEntry createEntry(byte [] table, int row, KeyValue.Type type, List cells) { - byte[] fam = Bytes.equals(table, TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2; + private WALEntry createEntry(TableName table, int row, KeyValue.Type type, List cells) { + byte[] fam = table.equals(TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2; byte[] rowBytes = Bytes.toBytes(row); // Just make sure we don't get the same ts for two consecutive rows with // same key @@ -262,7 +263,7 @@ public class TestReplicationSink { uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits()); uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits()); keyBuilder.setClusterId(uuidBuilder.build()); - keyBuilder.setTableName(ByteStringer.wrap(table)); + keyBuilder.setTableName(ByteStringer.wrap(table.getName())); keyBuilder.setWriteTime(now); keyBuilder.setEncodedRegionName(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY)); keyBuilder.setLogSequenceNumber(-1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java index c6bdfc2dd09..5b718f064bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java @@ -111,7 +111,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteColumns() throws Throwable { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + TOPSECRET, SECRET); try { @@ -120,7 +120,7 @@ public class TestVisibilityLabelsWithDeletes { public Void run() throws Exception { Table table = null; try { - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(TOPSECRET + "&" + SECRET)); d.deleteColumns(fam, qual); @@ -157,7 +157,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteFamily() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final Table table = createTableAndWriteDataWithLabels(tableName, SECRET, CONFIDENTIAL + "|" + TOPSECRET); try { @@ -165,7 +165,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row2); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.deleteFamily(fam); @@ -199,7 +199,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteFamilyVersion() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); long[] ts = new long[] { 123l, 125l }; final Table table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|" + TOPSECRET, SECRET); @@ -209,7 +209,7 @@ public class TestVisibilityLabelsWithDeletes { public Void run() throws Exception { Table table = null; try { - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.deleteFamilyVersion(fam, 123l); @@ -245,7 +245,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteColumnExactVersion() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); long[] ts = new long[] { 123l, 125l }; final Table table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|" + TOPSECRET, SECRET); @@ -255,7 +255,7 @@ public class TestVisibilityLabelsWithDeletes { public Void run() throws Exception { Table table = null; try { - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.deleteColumn(fam, qual, 123l); @@ -291,7 +291,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersions() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -300,7 +300,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -353,7 +353,7 @@ public class TestVisibilityLabelsWithDeletes { public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersionsNoTimestamp() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -362,7 +362,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumns(fam, qual); @@ -410,7 +410,7 @@ public class TestVisibilityLabelsWithDeletes { testVisibilityLabelsWithDeleteColumnsWithNoMatchVisExpWithMultipleVersionsNoTimestamp() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -419,7 +419,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumns(fam, qual); @@ -471,7 +471,7 @@ public class TestVisibilityLabelsWithDeletes { public void testVisibilityLabelsWithDeleteFamilyWithMultipleVersionsNoTimestamp() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -480,7 +480,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -525,7 +525,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteFamilyWithPutsReAppearing() throws Exception { - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); @@ -548,7 +548,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -575,7 +575,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteFamily(fam); @@ -609,7 +609,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteColumnsWithPutsReAppearing() throws Exception { - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); @@ -632,7 +632,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumns(fam, qual); @@ -659,7 +659,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumns(fam, qual); @@ -693,7 +693,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityCombinations() throws Exception { - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); @@ -716,13 +716,13 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumns(fam, qual, 126l); table.delete(d); - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumn(fam, qual, 123l); @@ -750,7 +750,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteColumnWithSpecificVersionWithPutsReAppearing() throws Exception { - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); @@ -780,13 +780,13 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumn(fam, qual, 123l); table.delete(d); - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumn(fam, qual, 123l); @@ -817,7 +817,7 @@ public class TestVisibilityLabelsWithDeletes { testVisibilityLabelsWithDeleteFamilyWithNoMatchingVisExpWithMultipleVersionsNoTimestamp() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -826,7 +826,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -877,7 +877,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteFamilyAndDeleteColumnsWithAndWithoutVisibilityExp() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -886,7 +886,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteFamily(fam); table.delete(d); @@ -1056,7 +1056,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnWithSpecificTimeStampUsingMultipleVersionsUnMatchingVisExpression() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -1065,7 +1065,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -1127,7 +1127,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnWithLatestTimeStampUsingMultipleVersions() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -1136,7 +1136,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual); @@ -1192,7 +1192,7 @@ public class TestVisibilityLabelsWithDeletes { @Test (timeout=180000) public void testDeleteColumnWithLatestTimeStampWhenNoVersionMatches() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -1205,7 +1205,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET )); d.deleteColumn(fam, qual); @@ -1285,7 +1285,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnWithLatestTimeStampUsingMultipleVersionsAfterCompaction() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -1294,7 +1294,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual); @@ -1357,7 +1357,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteFamilyLatestTimeStampWithMulipleVersions() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -1366,7 +1366,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteFamily(fam); @@ -1412,7 +1412,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnswithMultipleColumnsWithMultipleVersions() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPutsWithDiffCols(tableName); @@ -1421,7 +1421,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumns(fam, qual, 125l); @@ -1475,7 +1475,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnsWithDiffColsAndTags() throws Exception { - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); @@ -1498,7 +1498,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumns(fam, qual, 126l); @@ -1529,7 +1529,7 @@ public class TestVisibilityLabelsWithDeletes { } @Test public void testDeleteColumnsWithDiffColsAndTags1() throws Exception { - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); @@ -1552,7 +1552,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumns(fam, qual, 126l); @@ -1584,7 +1584,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteFamilyWithoutCellVisibilityWithMulipleVersions() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPutsWithoutVisibility(tableName); @@ -1593,7 +1593,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteFamily(fam); table.delete(d); @@ -1629,7 +1629,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteFamilyLatestTimeStampWithMulipleVersionsWithoutCellVisibilityInPuts() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPutsWithoutVisibility(tableName); @@ -1637,7 +1637,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteFamily(fam); @@ -1697,7 +1697,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteFamilySpecificTimeStampWithMulipleVersions() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -1706,7 +1706,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")")); @@ -1758,7 +1758,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testScanAfterCompaction() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -1767,7 +1767,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -1817,7 +1817,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteFamilySpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { // Do not flush here. @@ -1826,7 +1826,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -1878,7 +1878,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -1936,14 +1936,14 @@ public class TestVisibilityLabelsWithDeletes { } }; VisibilityLabelsResponse response = SUPERUSER.runAs(action); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = doPuts(tableName); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamilyVersion(fam, 123l); @@ -1993,14 +1993,14 @@ public class TestVisibilityLabelsWithDeletes { @Test (timeout=180000) public void testSpecificDeletesFollowedByDeleteFamily() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = doPuts(tableName); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")")); @@ -2046,7 +2046,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -2097,14 +2097,14 @@ public class TestVisibilityLabelsWithDeletes { } }; VisibilityLabelsResponse response = SUPERUSER.runAs(action); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = doPuts(tableName); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")")); @@ -2151,7 +2151,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -2191,7 +2191,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { // Do not flush here. @@ -2200,7 +2200,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual, 125l); @@ -2251,7 +2251,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2302,7 +2302,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice1() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { // Do not flush here. @@ -2311,7 +2311,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")" + "|(" + TOPSECRET + "&" + SECRET + ")")); @@ -2363,7 +2363,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual, 127l); @@ -2417,7 +2417,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice2() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { // Do not flush here. @@ -2426,7 +2426,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2483,7 +2483,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2539,7 +2539,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnAndDeleteFamilylSpecificTimeStampWithMulipleVersion() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { // Do not flush here. @@ -2548,7 +2548,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual, 125l); @@ -2599,7 +2599,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2661,7 +2661,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDiffDeleteTypesForTheSameCellUsingMultipleVersions() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { // Do not flush here. @@ -2670,7 +2670,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2722,7 +2722,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2772,7 +2772,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnLatestWithNoCellVisibility() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = doPuts(tableName); @@ -2781,7 +2781,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteColumn(fam, qual, 125l); table.delete(d); @@ -2805,7 +2805,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteColumns(fam, qual, 125l); table.delete(d); @@ -2830,7 +2830,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteFamily(fam, 125l); table.delete(d); @@ -2855,7 +2855,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteFamily(fam); table.delete(d); @@ -2880,7 +2880,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteColumns(fam, qual); table.delete(d); @@ -2905,7 +2905,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteFamilyVersion(fam, 126l); table.delete(d); @@ -2969,7 +2969,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityExpressionWithNotEqualORCondition() throws Exception { setAuths(); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); @@ -2992,7 +2992,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - Table table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, tableName); Delete d = new Delete(row1); d.deleteColumn(fam, qual, 124l); d.setCellVisibility(new CellVisibility(PRIVATE )); 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 4918084c0ea..828c89b2ccd 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 @@ -120,7 +120,7 @@ public class TestVisibilityWithCheckAuths { } }; SUPERUSER.runAs(action); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); colDesc.setMaxVersions(5); @@ -135,7 +135,7 @@ public class TestVisibilityWithCheckAuths { public Void run() throws Exception { Table table = null; try { - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); Put p = new Put(row1); p.setCellVisibility(new CellVisibility(PUBLIC + "&" + TOPSECRET)); p.add(fam, qual, 125l, value); @@ -170,7 +170,7 @@ public class TestVisibilityWithCheckAuths { } }; SUPERUSER.runAs(action); - TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); + final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); Table table = null; try { table = TEST_UTIL.createTable(tableName, fam); @@ -181,7 +181,7 @@ public class TestVisibilityWithCheckAuths { public Void run() throws Exception { Table table = null; try { - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); Put put = new Put(row1); put.add(fam, qual, HConstants.LATEST_TIMESTAMP, val); put.setCellVisibility(new CellVisibility(TOPSECRET)); @@ -198,7 +198,7 @@ public class TestVisibilityWithCheckAuths { public Void run() throws Exception { Table table = null; try { - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); Append append = new Append(row1); append.add(fam, qual, Bytes.toBytes("b")); table.append(append); @@ -214,7 +214,7 @@ public class TestVisibilityWithCheckAuths { public Void run() throws Exception { Table table = null; try { - table = new HTable(conf, TEST_NAME.getMethodName()); + table = new HTable(conf, tableName); Append append = new Append(row1); append.add(fam, qual, Bytes.toBytes("c")); append.setCellVisibility(new CellVisibility(PUBLIC)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java index f47b236484f..c5a3d2ef2ba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue; import java.util.Collection; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; @@ -67,7 +68,7 @@ public class TestHTraceHooks { Table table; try { - table = TEST_UTIL.createTable("table".getBytes(), + table = TEST_UTIL.createTable(TableName.valueOf("table"), FAMILY_BYTES); } finally { tableCreationSpan.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index b7868ab7fb6..70b53c5dec4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -163,8 +163,8 @@ public class TestHBaseFsck { @Test public void testHBaseFsck() throws Exception { assertNoErrors(doFsck(conf, false)); - String table = "tableBadMetaAssign"; - TEST_UTIL.createTable(Bytes.toBytes(table), FAM); + TableName table = TableName.valueOf("tableBadMetaAssign"); + TEST_UTIL.createTable(table, FAM); // We created 1 table, should be fine assertNoErrors(doFsck(conf, false)); @@ -216,7 +216,7 @@ public class TestHBaseFsck { assertNoErrors(doFsck(conf, false)); // comment needed - what is the purpose of this line - Table t = new HTable(conf, Bytes.toBytes(table), executorService); + Table t = new HTable(conf, table, executorService); ResultScanner s = t.getScanner(new Scan()); s.close(); t.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java index 9b208bc6706..27de51ddf33 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java @@ -103,7 +103,7 @@ public class TestHBaseFsckEncryption { @Test public void testFsckWithEncryption() throws Exception { // Populate the table with some data - Table table = new HTable(conf, htd.getName()); + Table table = new HTable(conf, htd.getTableName()); try { byte[] values = { 'A', 'B', 'C', 'D' }; for (int i = 0; i < values.length; i++) { @@ -121,7 +121,7 @@ public class TestHBaseFsckEncryption { TEST_UTIL.getHBaseAdmin().flush(htd.getTableName()); // Verify we have encrypted store files on disk - final List paths = findStorefilePaths(htd.getName()); + final List paths = findStorefilePaths(htd.getTableName()); assertTrue(paths.size() > 0); for (Path path: paths) { assertTrue("Store file " + path + " has incorrect key", @@ -138,7 +138,7 @@ public class TestHBaseFsckEncryption { assertEquals(hfcc.getMissing().size(), 0); } - private List findStorefilePaths(byte[] tableName) throws Exception { + private List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); for (HRegion region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java index 92c4e0890c1..451da3c8b10 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java @@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -92,8 +93,9 @@ public class TestRegionSplitter { expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY); // Do table creation/pre-splitting and verification of region boundaries - preSplitTableAndVerify(expectedBounds, - HexStringSplit.class.getSimpleName(), "NewHexPresplitTable"); + preSplitTableAndVerify(expectedBounds, + HexStringSplit.class.getSimpleName(), + TableName.valueOf("NewHexPresplitTable")); } /** @@ -122,7 +124,7 @@ public class TestRegionSplitter { // Do table creation/pre-splitting and verification of region boundaries preSplitTableAndVerify(expectedBounds, UniformSplit.class.getSimpleName(), - "NewUniformPresplitTable"); + TableName.valueOf("NewUniformPresplitTable")); } /** @@ -273,12 +275,12 @@ public class TestRegionSplitter { * @throws Various junit assertions */ private void preSplitTableAndVerify(List expectedBounds, - String splitClass, String tableName) throws Exception { + String splitClass, TableName tableName) throws Exception { final int numRegions = expectedBounds.size()-1; final Configuration conf = UTIL.getConfiguration(); conf.setInt("split.count", numRegions); SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass); - RegionSplitter.createPresplitTable(tableName, splitAlgo, + RegionSplitter.createPresplitTable(tableName.getNameAsString(), splitAlgo, new String[] {CF_NAME}, conf); verifyBounds(expectedBounds, tableName); } @@ -287,26 +289,28 @@ public class TestRegionSplitter { public void noopRollingSplit() throws Exception { final List expectedBounds = new ArrayList(); expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY); - rollingSplitAndVerify(TestRegionSplitter.class.getSimpleName(), "UniformSplit", expectedBounds); + rollingSplitAndVerify( + TableName.valueOf(TestRegionSplitter.class.getSimpleName()), + "UniformSplit", expectedBounds); } - private void rollingSplitAndVerify(String tableName, String splitClass, + private void rollingSplitAndVerify(TableName tableName, String splitClass, List expectedBounds) throws Exception { final Configuration conf = UTIL.getConfiguration(); // Set this larger than the number of splits so RegionSplitter won't block conf.setInt("split.outstanding", 5); SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass); - RegionSplitter.rollingSplit(tableName, splitAlgo, conf); + RegionSplitter.rollingSplit(tableName.getNameAsString(), splitAlgo, conf); verifyBounds(expectedBounds, tableName); } - private void verifyBounds(List expectedBounds, String tableName) + private void verifyBounds(List expectedBounds, TableName tableName) throws Exception { // Get region boundaries from the cluster and verify their endpoints final Configuration conf = UTIL.getConfiguration(); final int numRegions = expectedBounds.size()-1; - final HTable hTable = new HTable(conf, tableName.getBytes()); + final HTable hTable = new HTable(conf, tableName); final Map regionInfoMap = hTable.getRegionLocations(); assertEquals(numRegions, regionInfoMap.size()); for (Map.Entry entry: regionInfoMap.entrySet()) {