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 d3dfa990084..f7dbbc60310 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
@@ -882,8 +882,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 8ea8f418b8e..c31fed208fd 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
@@ -125,7 +125,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 cc1648315f0..01a9aa57eed 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
@@ -21,6 +21,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.MediumTests;
+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;
@@ -59,7 +60,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 465b046f09b..190d92bf1f3 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.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 064d09918b9..24dafa500ab 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 aee06b444cf..b6beadc349a 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
@@ -39,6 +39,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;
@@ -396,8 +397,8 @@ public class Import {
*/
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 = new Job(conf, NAME + "_" + tableName);
job.setJarByClass(Importer.class);
@@ -430,7 +431,7 @@ public class Import {
// 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 0c34fa2807c..c01957f7bc8 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 = new Job(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 898d6bb4586..39b573df36d 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
@@ -245,7 +245,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 61caf587952..ca417807cbc 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
@@ -121,7 +121,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 cec27dfa12f..8cd2e1518d5 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
@@ -1073,9 +1073,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});
}
/**
@@ -2147,20 +2147,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 f4ffa2b9bd2..ce6885246f3 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
@@ -37,7 +37,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 893c2d5c056..c00a92c189a 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
@@ -71,7 +71,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 cb01558f10d..d5f83d7c0ff 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
@@ -81,7 +81,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 f6d27461703..b6742ff3375 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
@@ -97,17 +97,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 e619d5e5c28..08835aa2789 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
@@ -142,7 +142,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();
@@ -151,7 +151,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 877b7273c89..f8bf6ee1a01 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
@@ -260,7 +260,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"));
@@ -545,7 +545,7 @@ public class TestZooKeeper {
admin.createTable(htd, SPLIT_KEYS);
ZooKeeperWatcher zooKeeperWatcher = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
- 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 c9a2bb01f60..04bd466d1d8 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
@@ -301,8 +301,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);
@@ -351,8 +351,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()
@@ -412,7 +411,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();
@@ -1224,7 +1223,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();
}
/***
@@ -1261,7 +1260,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
@@ -1307,7 +1306,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);
}
@@ -1345,8 +1344,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()));
}
@@ -1387,7 +1387,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()) {
@@ -1445,7 +1445,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());
@@ -1469,7 +1469,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());
@@ -1493,7 +1493,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) {
@@ -1576,7 +1576,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");
@@ -1656,19 +1656,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);
@@ -1769,10 +1769,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 546b9ea965c..ae8bd503737 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
@@ -83,7 +83,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 a38cb1f560b..22d2e3906d1 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
@@ -181,7 +181,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();
@@ -384,7 +384,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);
@@ -532,7 +532,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);
@@ -1207,7 +1207,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) {}
@@ -1242,7 +1242,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 {
@@ -3596,8 +3596,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());
@@ -3629,8 +3628,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
@@ -3644,8 +3643,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
@@ -3725,7 +3724,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);
@@ -3734,7 +3733,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);
@@ -3757,7 +3756,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);
@@ -3821,7 +3820,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;
@@ -4063,10 +4062,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);
}
@@ -4079,12 +4078,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);
}
}
@@ -4095,7 +4094,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);
@@ -4109,7 +4108,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));
@@ -4124,7 +4123,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);
@@ -4149,8 +4148,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");
@@ -4192,7 +4191,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
@@ -4210,7 +4209,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);
@@ -4227,7 +4226,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");
@@ -4341,7 +4340,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);
@@ -4372,7 +4371,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")
@@ -4404,7 +4403,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");
@@ -4455,7 +4454,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);
@@ -4481,7 +4480,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 {
@@ -4536,7 +4535,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 [][] {
@@ -4576,7 +4575,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 [][] {
@@ -4647,7 +4646,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;
@@ -4655,8 +4654,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);
@@ -4684,7 +4682,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;
@@ -4770,8 +4768,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);
@@ -4812,8 +4809,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);
@@ -4878,8 +4874,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);
@@ -4952,7 +4948,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);
@@ -5056,13 +5052,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);
@@ -5170,8 +5165,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);
@@ -5219,7 +5213,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);
@@ -5270,7 +5264,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();
@@ -5296,7 +5290,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);
@@ -5325,7 +5319,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);
@@ -5469,8 +5463,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
@@ -5545,7 +5539,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
@@ -5581,7 +5575,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);
@@ -5627,7 +5621,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--"),
@@ -5667,7 +5661,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--"),
@@ -5708,7 +5702,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);
@@ -5773,7 +5767,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);
@@ -5785,7 +5779,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);
@@ -5810,13 +5804,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]);
@@ -5997,7 +5990,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)),
@@ -6054,7 +6047,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 5858e75fe2e..fee836fe08c 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
@@ -337,8 +337,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);
@@ -356,8 +356,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);
@@ -409,7 +409,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 5c6769f62fe..98f02ff81c8 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MediumTests;
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;
@@ -66,7 +67,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();
@@ -78,9 +79,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 e22005886e8..f95ac2cac86 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
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
@@ -47,7 +48,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 ee5b4ed37b8..129e7147ae1 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
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HTestConst;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
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;
@@ -103,7 +104,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);
@@ -444,7 +445,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 5f2d2c56c54..c3855eac614 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
@@ -57,7 +57,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 3e382ea7235..1bd11ae070a 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
@@ -96,7 +96,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++) {
@@ -171,7 +171,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"));
@@ -231,7 +231,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 cbeb532c9f5..8c1e5159c34 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
@@ -65,7 +65,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");
@@ -135,7 +135,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);
}
@@ -169,7 +169,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 9f8115189c9..27e77db6b61 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
@@ -161,7 +161,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 c501149140a..f5e3baaef43 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
@@ -158,7 +158,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 d64a5ad432d..66b346e75cc 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
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
+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;
@@ -88,7 +89,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 eee8c9a6e30..ead2c472ef0 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
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LargeTests;
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 d4195106695..a735d47c1d6 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 1cdde7c8728..2fafa31cfcf 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
@@ -89,7 +89,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 218e73bd217..6da7c06f8f8 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
@@ -79,19 +79,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 e527f9772a4..7b96313d3bd 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
@@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
@@ -87,7 +88,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();
}
@@ -104,7 +105,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 8d171a614a1..937feaff9e8 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
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
@@ -79,8 +80,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");
@@ -98,8 +99,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++) {
@@ -117,8 +118,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");
@@ -142,8 +143,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 d02b3787123..e60f9cb9fb0 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
@@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.mapreduce;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
@@ -51,8 +52,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 f89808f203e..ddaac934dcc 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
@@ -160,7 +160,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);
@@ -180,7 +180,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,
@@ -223,7 +223,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
@@ -253,7 +253,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);
@@ -284,7 +284,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);
@@ -316,7 +316,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
@@ -350,7 +350,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);
@@ -377,7 +377,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" };
@@ -543,7 +543,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);
@@ -564,7 +564,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);
@@ -583,7 +583,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 1c134685b7f..10fa2070cab 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -119,7 +120,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);
}
@@ -135,7 +136,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);
}
@@ -176,7 +177,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");
@@ -190,7 +191,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 9494fa489c3..4b7e3944812 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -160,20 +161,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);
@@ -181,7 +182,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;
@@ -228,7 +229,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);
}
@@ -265,7 +266,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);
}
@@ -282,12 +283,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";
@@ -329,7 +331,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;
}
@@ -363,7 +365,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 8e64f77f4b1..e0133045110 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LargeTests;
+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;
@@ -110,7 +111,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);
}
@@ -128,7 +129,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);
}
@@ -145,7 +146,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);
}
@@ -180,7 +181,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);
}
@@ -287,7 +288,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");
@@ -299,7 +300,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 b93fb322126..738d9fce286 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
@@ -120,10 +120,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)));
}
@@ -141,11 +141,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)));
}
@@ -156,9 +156,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;
@@ -167,26 +167,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
}
@@ -198,7 +197,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++;
}
}
@@ -231,10 +230,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);
@@ -261,7 +261,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();
@@ -292,7 +292,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,
@@ -333,7 +333,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);
@@ -358,7 +358,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);
@@ -375,7 +375,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);
@@ -398,7 +398,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);
@@ -411,7 +411,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(
@@ -433,7 +433,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");
@@ -441,9 +441,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);
@@ -478,7 +478,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(
- util.getZooKeeperWatcher(), hConnection, TableName.valueOf(tableName));
+ util.getZooKeeperWatcher(), 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 74af66dfc6c..34cfbddf5df 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
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LargeTests;
+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;
@@ -75,8 +76,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 93dbbebc586..14d9acb5346 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
@@ -57,7 +57,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;
@@ -137,7 +137,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);
@@ -146,11 +146,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) {
@@ -160,7 +160,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 09aa652b816..4027d6d7e6c 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
@@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
+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.mapreduce.RowCounter.RowCounterMapper;
@@ -67,8 +68,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 11a54d4538b..e858e0517bf 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
@@ -121,7 +121,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 a586af5713b..03fa9f21fd8 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
@@ -86,8 +86,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");
@@ -117,8 +117,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 8dffe7d505a..108f76d3ab9 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
@@ -1202,7 +1202,7 @@ public class TestMasterFailover {
assertTrue(master.isInitialized());
// Create a table with a region online
- HTable onlineTable = TEST_UTIL.createTable("onlineTable", "family");
+ HTable 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 44b9573701e..d429f98c667 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
@@ -51,7 +51,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")};
@@ -63,11 +63,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();
}
@@ -490,12 +489,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);
@@ -528,4 +527,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 7fee3239be7..847d19afe31 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
@@ -48,10 +48,10 @@ public class TestRestartCluster {
private static final byte[] TABLENAME = Bytes.toBytes("master_transitions");
private static final byte [][] FAMILIES = {Bytes.toBytes("a")};
- 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");
@@ -95,11 +95,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 =
@@ -121,14 +121,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());
}
}
@@ -145,11 +145,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 b55654f21fb..271f0e960eb 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
@@ -80,7 +80,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;
@@ -123,8 +123,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 71784be06fe..7df33b63505 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
@@ -51,7 +51,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())) {
@@ -102,7 +102,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")));
@@ -120,7 +120,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 efa5a227105..d59d81fa878 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
@@ -132,9 +132,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;
@@ -159,9 +159,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 "
@@ -180,7 +179,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 "
@@ -209,9 +208,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;
@@ -263,10 +262,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)));
}
@@ -282,7 +281,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;
@@ -295,7 +294,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);
@@ -335,7 +334,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 b3bfaaad5a4..9bf360cb0eb 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
@@ -79,7 +79,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 ee5c4db5fe3..a808dd986bd 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
@@ -87,7 +87,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);
}
@@ -99,7 +99,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");
@@ -184,7 +184,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");
@@ -243,7 +243,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");
@@ -279,7 +279,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");
@@ -311,7 +311,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");
@@ -343,7 +343,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 42915907228..14cae3c2380 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MetaTableAccessor;
+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.coordination.BaseCoordinatedStateManager;
@@ -76,10 +77,10 @@ public class TestRegionServerNoMaster {
public static void before() throws Exception {
HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
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 1bf6b761176..38e53bc9e09 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.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -57,7 +58,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);
@@ -67,7 +68,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);
@@ -162,7 +163,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
@@ -196,7 +197,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);
@@ -238,7 +239,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 4f711554e31..9ed22b6dc00 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@@ -133,7 +134,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 eb65b5ef2f2..0708b7911c0 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
@@ -194,12 +194,12 @@ public class TestSplitTransactionOnCluster {
try {
// Create table then get the single region for our new table.
- HTable t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf"));
+ HTable t = createTableAndWait(tableName, Bytes.toBytes("cf"));
final List regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
final HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
- 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.
@@ -281,10 +281,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.
@@ -323,7 +323,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.
@@ -393,8 +393,8 @@ public class TestSplitTransactionOnCluster {
@Test (timeout = 300000) public void testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling()
throws IOException, InterruptedException, NodeExistsException, KeeperException,
DeserializationException, ServiceException {
- final byte [] tableName =
- Bytes.toBytes("testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling");
+ final TableName tableName =
+ TableName.valueOf("testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling");
// Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
@@ -471,8 +471,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);
@@ -541,8 +541,8 @@ public class TestSplitTransactionOnCluster {
*/
@Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit()
throws IOException, InterruptedException, ServiceException {
- 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);
@@ -710,7 +710,7 @@ public class TestSplitTransactionOnCluster {
public void testMasterRestartWhenSplittingIsPartial()
throws IOException, InterruptedException, NodeExistsException,
KeeperException, DeserializationException, ServiceException {
- final byte[] tableName = Bytes.toBytes("testMasterRestartWhenSplittingIsPartial");
+ final TableName tableName = TableName.valueOf("testMasterRestartWhenSplittingIsPartial");
if (!useZKForAssignment) {
// This test doesn't apply if not using ZK for assignment
@@ -805,7 +805,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);
@@ -888,13 +889,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.
@@ -926,7 +927,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"));
@@ -953,7 +954,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);
@@ -1029,19 +1030,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(),
@@ -1051,16 +1052,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) {
@@ -1075,7 +1076,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"));
@@ -1204,7 +1205,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
@@ -1339,7 +1340,7 @@ public class TestSplitTransactionOnCluster {
getServers().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");
@@ -1350,7 +1351,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);
@@ -1360,11 +1361,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 ac71a3d649b..40126296507 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
@@ -112,15 +112,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 06b5f489cb0..23571c2cc85 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
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
+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;
@@ -71,13 +72,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 {
@@ -93,12 +93,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 1a4b08a1045..cf70d4a4e24 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
@@ -178,7 +178,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);
@@ -331,15 +331,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());
@@ -439,14 +438,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();
@@ -598,12 +596,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);
@@ -661,7 +657,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 8a08f0c6f49..99dc68758b3 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
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
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;
@@ -61,10 +62,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");
@@ -232,8 +233,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
@@ -261,7 +262,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 1a617dc22c8..55324f7eee1 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
@@ -110,7 +110,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 {
@@ -119,7 +119,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);
@@ -156,7 +156,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 {
@@ -164,7 +164,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);
@@ -198,7 +198,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);
@@ -208,7 +208,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);
@@ -244,7 +244,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);
@@ -254,7 +254,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);
@@ -290,7 +290,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);
@@ -299,7 +299,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+")"));
@@ -352,7 +352,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);
@@ -361,7 +361,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);
@@ -409,7 +409,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);
@@ -418,7 +418,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);
@@ -470,7 +470,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);
@@ -479,7 +479,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);
@@ -524,7 +524,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();
@@ -547,7 +547,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);
@@ -574,7 +574,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);
@@ -608,7 +608,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();
@@ -631,7 +631,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);
@@ -658,7 +658,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);
@@ -692,7 +692,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();
@@ -715,13 +715,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);
@@ -749,7 +749,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();
@@ -779,13 +779,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);
@@ -816,7 +816,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);
@@ -825,7 +825,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);
@@ -876,7 +876,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);
@@ -885,7 +885,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);
@@ -1055,7 +1055,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);
@@ -1064,7 +1064,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+")"));
@@ -1126,7 +1126,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);
@@ -1135,7 +1135,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);
@@ -1191,7 +1191,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);
@@ -1204,7 +1204,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);
@@ -1284,7 +1284,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);
@@ -1293,7 +1293,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);
@@ -1356,7 +1356,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);
@@ -1365,7 +1365,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);
@@ -1411,7 +1411,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);
@@ -1420,7 +1420,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);
@@ -1474,7 +1474,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();
@@ -1497,7 +1497,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);
@@ -1528,7 +1528,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();
@@ -1551,7 +1551,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);
@@ -1583,7 +1583,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);
@@ -1592,7 +1592,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);
@@ -1628,7 +1628,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);
@@ -1636,7 +1636,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);
@@ -1696,7 +1696,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);
@@ -1705,7 +1705,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 + ")"));
@@ -1757,7 +1757,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);
@@ -1766,7 +1766,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+")"));
@@ -1816,7 +1816,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.
@@ -1825,7 +1825,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+")"));
@@ -1877,7 +1877,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+")"));
@@ -1935,14 +1935,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);
@@ -1992,14 +1992,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 + ")"));
@@ -2045,7 +2045,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);
@@ -2096,14 +2096,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 + ")"));
@@ -2150,7 +2150,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);
@@ -2190,7 +2190,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.
@@ -2199,7 +2199,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);
@@ -2250,7 +2250,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+")"));
@@ -2301,7 +2301,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.
@@ -2310,7 +2310,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 + ")"));
@@ -2362,7 +2362,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);
@@ -2416,7 +2416,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.
@@ -2425,7 +2425,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+")"));
@@ -2482,7 +2482,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+")"));
@@ -2538,7 +2538,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.
@@ -2547,7 +2547,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);
@@ -2598,7 +2598,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+")"));
@@ -2660,7 +2660,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.
@@ -2669,7 +2669,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+")"));
@@ -2721,7 +2721,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+")"));
@@ -2771,7 +2771,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);
@@ -2780,7 +2780,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);
@@ -2804,7 +2804,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);
@@ -2829,7 +2829,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);
@@ -2854,7 +2854,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);
@@ -2879,7 +2879,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);
@@ -2904,7 +2904,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);
@@ -2968,7 +2968,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();
@@ -2991,7 +2991,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 ff0d29f6838..3627e0101e1 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
@@ -119,7 +119,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);
@@ -134,7 +134,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);
@@ -169,7 +169,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);
@@ -180,7 +180,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));
@@ -197,7 +197,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);
@@ -213,7 +213,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 0925fe375f5..0f439e2a785 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
@@ -25,6 +25,7 @@ import java.util.Collection;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
+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;
@@ -66,7 +67,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 7002013e2f7..70afecc24d8 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
@@ -157,8 +157,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));
@@ -210,7 +210,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 6dfe90cda50..901f0eff6c2 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
@@ -102,7 +102,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++) {
@@ -120,7 +120,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",
@@ -137,7 +137,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 878e1093f8c..7a35ea626b4 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
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.util.RegionSplitter.HexStringSplit;
import org.apache.hadoop.hbase.util.RegionSplitter.SplitAlgorithm;
@@ -91,8 +92,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"));
}
/**
@@ -121,7 +123,7 @@ public class TestRegionSplitter {
// Do table creation/pre-splitting and verification of region boundaries
preSplitTableAndVerify(expectedBounds, UniformSplit.class.getSimpleName(),
- "NewUniformPresplitTable");
+ TableName.valueOf("NewUniformPresplitTable"));
}
/**
@@ -272,12 +274,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);
}
@@ -286,26 +288,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()) {
| |