HBASE-12042 Replace internal uses of HTable(Configuration, String) with HTable(Configuration, TableName) (Solomon Duskis)

This commit is contained in:
Enis Soztutar 2014-09-29 18:39:44 -07:00
parent 683f3b3d50
commit 6189b52fb0
75 changed files with 521 additions and 503 deletions

View File

@ -872,8 +872,21 @@ public class MetaTableAccessor {
* @return Count or regions in table <code>tableName</code> * @return Count or regions in table <code>tableName</code>
* @throws IOException * @throws IOException
*/ */
@Deprecated
public static int getRegionCount(final Configuration c, final String tableName) public static int getRegionCount(final Configuration c, final String tableName)
throws IOException { throws IOException {
return getRegionCount(c, TableName.valueOf(tableName));
}
/**
* Count regions in <code>hbase:meta</code> for passed table.
* @param c Configuration object
* @param tableName table name to count regions for
* @return Count or regions in table <code>tableName</code>
* @throws IOException
*/
public static int getRegionCount(final Configuration c, final TableName tableName)
throws IOException {
HTable t = new HTable(c, tableName); HTable t = new HTable(c, tableName);
try { try {
return t.getRegionLocations().size(); return t.getRegionLocations().size();

View File

@ -128,7 +128,7 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Method for getting the tableDescriptor * Method for getting the tableDescriptor
* *
* @param tableName as a byte [] * @param tableName as a {@link TableName}
* @return the tableDescriptor * @return the tableDescriptor
* @throws org.apache.hadoop.hbase.TableNotFoundException * @throws org.apache.hadoop.hbase.TableNotFoundException
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.coprocessor.example;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; 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.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -60,7 +61,7 @@ public class TestRowCountEndpoint {
RowCountEndpoint.class.getName()); RowCountEndpoint.class.getName());
TEST_UTIL.startMiniCluster(); TEST_UTIL.startMiniCluster();
TEST_UTIL.createTable(TEST_TABLE, TEST_FAMILY); TEST_UTIL.createTable(TableName.valueOf(TEST_TABLE), new byte[][]{TEST_FAMILY});
} }
// @Ignore @AfterClass // @Ignore @AfterClass

View File

@ -24,6 +24,7 @@ import java.util.Collection;
import org.apache.commons.lang.math.RandomUtils; import org.apache.commons.lang.math.RandomUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
/** /**
@ -43,7 +44,7 @@ public class RestartRsHoldingTableAction extends RestartActionBaseAction {
HTable table = null; HTable table = null;
try { try {
Configuration conf = context.getHBaseIntegrationTestingUtility().getConfiguration(); Configuration conf = context.getHBaseIntegrationTestingUtility().getConfiguration();
table = new HTable(conf, tableName); table = new HTable(conf, TableName.valueOf(tableName));
} catch (IOException e) { } catch (IOException e) {
LOG.debug("Error creating HTable used to get list of region locations.", e); LOG.debug("Error creating HTable used to get list of region locations.", e);
return; return;

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.Type; 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
@ -129,10 +130,10 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
* Verify the data described by <code>simple_tsv</code> matches * Verify the data described by <code>simple_tsv</code> matches
* <code>simple_expected</code>. * <code>simple_expected</code>.
*/ */
protected void doLoadIncrementalHFiles(Path hfiles, String tableName) protected void doLoadIncrementalHFiles(Path hfiles, TableName tableName)
throws Exception { throws Exception {
String[] args = { hfiles.toString(), tableName }; String[] args = { hfiles.toString(), tableName.getNameAsString() };
LOG.info(format("Running LoadIncrememntalHFiles with args: %s", Arrays.asList(args))); LOG.info(format("Running LoadIncrememntalHFiles with args: %s", Arrays.asList(args)));
assertEquals("Loading HFiles failed.", assertEquals("Loading HFiles failed.",
0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args)); 0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args));
@ -181,9 +182,10 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
@Test @Test
public void testGenerateAndLoad() throws Exception { public void testGenerateAndLoad() throws Exception {
LOG.info("Running test testGenerateAndLoad."); LOG.info("Running test testGenerateAndLoad.");
String table = NAME + "-" + UUID.randomUUID(); TableName table = TableName.valueOf(NAME + "-" + UUID.randomUUID());
String cf = "d"; String cf = "d";
Path hfiles = new Path(util.getDataTestDirOnTestFS(table), "hfiles"); Path hfiles = new Path(
util.getDataTestDirOnTestFS(table.getNameAsString()), "hfiles");
String[] args = { String[] args = {
format("-D%s=%s", ImportTsv.BULK_OUTPUT_CONF_KEY, hfiles), 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 // configure the test harness to NOT delete the HFiles after they're
// generated. We need those for doLoadIncrementalHFiles // generated. We need those for doLoadIncrementalHFiles
format("-D%s=false", TestImportTsv.DELETE_AFTER_LOAD_CONF), format("-D%s=false", TestImportTsv.DELETE_AFTER_LOAD_CONF),
table table.getNameAsString()
}; };
// run the job, complete the load. // 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); Tool t = TestImportTsv.doMROnTableTest(util, cf, simple_tsv, args);
doLoadIncrementalHFiles(hfiles, table); doLoadIncrementalHFiles(hfiles, table);
@ -205,7 +207,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
// clean up after ourselves. // clean up after ourselves.
util.deleteTable(table); util.deleteTable(table);
util.cleanupDataTestDirOnTestFS(table); util.cleanupDataTestDirOnTestFS(table.getNameAsString());
LOG.info("testGenerateAndLoad completed successfully."); LOG.info("testGenerateAndLoad completed successfully.");
} }

View File

@ -179,7 +179,7 @@ public void cleanUpCluster() throws Exception {
recordsToWrite = conf.getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT); recordsToWrite = conf.getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT);
String tableName = conf.get(TABLE_NAME_KEY, TABLE_NAME_DEFAULT); String tableName = conf.get(TABLE_NAME_KEY, TABLE_NAME_DEFAULT);
numBackReferencesPerRow = conf.getInt(NUM_BACKREFS_KEY, NUM_BACKREFS_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.setWriteBufferSize(4*1024*1024);
table.setAutoFlush(false, true); table.setAutoFlush(false, true);

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; 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.HTable;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@ -122,8 +123,9 @@ implements Configurable {
public void setConf(Configuration configuration) { public void setConf(Configuration configuration) {
this.conf = HBaseConfiguration.create(configuration); this.conf = HBaseConfiguration.create(configuration);
try { try {
this.table = new HTable(this.conf, TableName tableName = TableName.valueOf(configuration
configuration.get(TableOutputFormat.OUTPUT_TABLE)); .get(TableOutputFormat.OUTPUT_TABLE));
this.table = new HTable(this.conf, tableName);
} catch (IOException e) { } catch (IOException e) {
LOG.error(e); LOG.error(e);
} }

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
@ -401,8 +402,8 @@ public class Import extends Configured implements Tool {
*/ */
public static Job createSubmittableJob(Configuration conf, String[] args) public static Job createSubmittableJob(Configuration conf, String[] args)
throws IOException { throws IOException {
String tableName = args[0]; TableName tableName = TableName.valueOf(args[0]);
conf.set(TABLE_NAME, tableName); conf.set(TABLE_NAME, tableName.getNameAsString());
Path inputDir = new Path(args[1]); Path inputDir = new Path(args[1]);
Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName)); Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
job.setJarByClass(Importer.class); job.setJarByClass(Importer.class);
@ -435,7 +436,7 @@ public class Import extends Configured implements Tool {
// No reducers. Just write straight to table. Call initTableReducerJob // No reducers. Just write straight to table. Call initTableReducerJob
// because it sets up the TableOutputFormat. // because it sets up the TableOutputFormat.
job.setMapperClass(Importer.class); job.setMapperClass(Importer.class);
TableMapReduceUtil.initTableReducerJob(tableName, null, job); TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job);
job.setNumReduceTasks(0); job.setNumReduceTasks(0);
} }
return job; return job;

View File

@ -414,9 +414,9 @@ public class ImportTsv extends Configured implements Tool {
Class mapperClass = mapperClassName != null ? Class mapperClass = mapperClassName != null ?
Class.forName(mapperClassName) : DEFAULT_MAPPER; Class.forName(mapperClassName) : DEFAULT_MAPPER;
String tableName = args[0]; TableName tableName = TableName.valueOf(args[0]);
Path inputDir = new Path(args[1]); Path inputDir = new Path(args[1]);
String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName); String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName.getNameAsString());
Job job = Job.getInstance(conf, jobName); Job job = Job.getInstance(conf, jobName);
job.setJarByClass(mapperClass); job.setJarByClass(mapperClass);
FileInputFormat.setInputPaths(job, inputDir); FileInputFormat.setInputPaths(job, inputDir);
@ -460,7 +460,8 @@ public class ImportTsv extends Configured implements Tool {
} }
// No reducers. Just write straight to table. Call initTableReducerJob // No reducers. Just write straight to table. Call initTableReducerJob
// to set up the TableOutputFormat. // to set up the TableOutputFormat.
TableMapReduceUtil.initTableReducerJob(tableName, null, job); TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null,
job);
job.setNumReduceTasks(0); job.setNumReduceTasks(0);
} }
@ -470,9 +471,9 @@ public class ImportTsv extends Configured implements Tool {
return job; return job;
} }
private static void createTable(Admin admin, String tableName, String[] columns) private static void createTable(Admin admin, TableName tableName, String[] columns)
throws IOException { throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); HTableDescriptor htd = new HTableDescriptor(tableName);
Set<String> cfSet = new HashSet<String>(); Set<String> cfSet = new HashSet<String>();
for (String aColumn : columns) { for (String aColumn : columns) {
if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn) if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.KeyValue; 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.HTable;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -96,7 +97,7 @@ implements Configurable {
@Override @Override
public void setConf(Configuration configuration) { public void setConf(Configuration configuration) {
this.conf = configuration; this.conf = configuration;
String tableName = conf.get(INPUT_TABLE); TableName tableName = TableName.valueOf(conf.get(INPUT_TABLE));
try { try {
setHTable(new HTable(new Configuration(conf), tableName)); setHTable(new HTable(new Configuration(conf), tableName));
} catch (Exception e) { } catch (Exception e) {

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; 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.Delete;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
@ -204,7 +205,7 @@ implements Configurable {
if (zkClientPort != 0) { if (zkClientPort != 0) {
this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort); 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); this.table.setAutoFlush(false, true);
LOG.info("Created table instance for " + tableName); LOG.info("Created table instance for " + tableName);
} catch(IOException e) { } catch(IOException e) {

View File

@ -247,7 +247,7 @@ public class WALPlayer extends Configured implements Tool {
if (tables.length != 1) { if (tables.length != 1) {
throw new IOException("Exactly one table must be specified for the bulk export option"); 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.setMapperClass(HLogKeyValueMapper.class);
job.setReducerClass(KeyValueSortReducer.class); job.setReducerClass(KeyValueSortReducer.class);
Path outputDir = new Path(hfileOutPath); Path outputDir = new Path(hfileOutPath);

View File

@ -123,7 +123,8 @@ public class VerifyReplication extends Configured implements Tool {
Configuration peerConf = HBaseConfiguration.create(conf); Configuration peerConf = HBaseConfiguration.create(conf);
ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey); 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()); scan.setStartRow(value.getRow());
replicatedScanner = replicatedTable.getScanner(scan); replicatedScanner = replicatedTable.getScanner(scan);
return null; return null;

View File

@ -606,7 +606,7 @@ public final class Canary implements Tool {
private void monitorRegionServers(Map<String, List<HRegionInfo>> rsAndRMap) { private void monitorRegionServers(Map<String, List<HRegionInfo>> rsAndRMap) {
String serverName = null; String serverName = null;
String tableName = null; TableName tableName = null;
HRegionInfo region = null; HRegionInfo region = null;
Table table = null; Table table = null;
Get get = null; Get get = null;
@ -620,7 +620,7 @@ public final class Canary implements Tool {
// always get the first region // always get the first region
region = entry.getValue().get(0); region = entry.getValue().get(0);
try { try {
tableName = region.getTable().getNameAsString(); tableName = region.getTable();
table = new HTable(this.admin.getConfiguration(), tableName); table = new HTable(this.admin.getConfiguration(), tableName);
startKey = region.getStartKey(); startKey = region.getStartKey();
// Can't do a get on empty start row so do a Scan of first element if any instead. // 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(); s.close();
stopWatch.stop(); stopWatch.stop();
} }
this.getSink().publishReadTiming(tableName, serverName, stopWatch.getTime()); this.getSink().publishReadTiming(tableName.getNameAsString(),
serverName, stopWatch.getTime());
} catch (TableNotFoundException tnfe) { } catch (TableNotFoundException tnfe) {
// This is ignored because it doesn't imply that the regionserver is dead // This is ignored because it doesn't imply that the regionserver is dead
} catch (TableNotEnabledException tnee) { } catch (TableNotEnabledException tnee) {
// This is considered a success since we got a response. // This is considered a success since we got a response.
LOG.debug("The targeted table was disabled. Assuming success."); LOG.debug("The targeted table was disabled. Assuming success.");
} catch (DoNotRetryIOException dnrioe) { } catch (DoNotRetryIOException dnrioe) {
this.getSink().publishReadFailure(tableName, serverName); this.getSink().publishReadFailure(tableName.getNameAsString(), serverName);
LOG.error(dnrioe); LOG.error(dnrioe);
} catch (IOException e) { } catch (IOException e) {
this.getSink().publishReadFailure(tableName, serverName); this.getSink().publishReadFailure(tableName.getNameAsString(), serverName);
LOG.error(e); LOG.error(e);
this.errorCode = ERROR_EXIT_CODE; this.errorCode = ERROR_EXIT_CODE;
} finally { } finally {

View File

@ -1071,9 +1071,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table. * @return An HTable instance for the created table.
* @throws IOException * @throws IOException
*/ */
public HTable createTable(String tableName, String family) public HTable createTable(TableName tableName, String family)
throws IOException{ throws IOException{
return createTable(TableName.valueOf(tableName), new String[]{family}); return createTable(tableName, new String[]{family});
} }
/** /**
@ -2226,20 +2226,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws IOException * @throws IOException
* @throws InterruptedException * @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) public HRegionServer getRSForFirstRegionInTable(TableName tableName)
throws IOException, InterruptedException { throws IOException, InterruptedException {
List<byte[]> metaRows = getMetaTableRows(tableName); List<byte[]> metaRows = getMetaTableRows(tableName);

View File

@ -39,7 +39,7 @@ public class TestFullLogReconstruction {
private final static HBaseTestingUtility private final static HBaseTestingUtility
TEST_UTIL = new 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"); private final static byte[] FAMILY = Bytes.toBytes("family");
/** /**

View File

@ -73,7 +73,7 @@ public class TestGlobalMemStoreSize {
byte [] table = Bytes.toBytes("TestGlobalMemStoreSize"); byte [] table = Bytes.toBytes("TestGlobalMemStoreSize");
byte [] family = Bytes.toBytes("family"); byte [] family = Bytes.toBytes("family");
LOG.info("Creating table with " + regionNum + " regions"); 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, int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family,
regionNum); regionNum);
assertEquals(regionNum,numRegions); assertEquals(regionNum,numRegions);

View File

@ -83,7 +83,7 @@ public class TestHBaseTestingUtility {
htu2.startMiniCluster(); htu2.startMiniCluster();
htu3.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[] FAM_NAME = Bytes.toBytes("fam");
final byte[] ROW = Bytes.toBytes("row"); final byte[] ROW = Bytes.toBytes("row");
final byte[] QUAL_NAME = Bytes.toBytes("qual"); final byte[] QUAL_NAME = Bytes.toBytes("qual");

View File

@ -99,17 +99,16 @@ public class TestInfoServers {
@Test @Test
public void testMasterServerReadOnly() throws Exception { public void testMasterServerReadOnly() throws Exception {
String sTableName = "testMasterServerReadOnly"; TableName tableName = TableName.valueOf("testMasterServerReadOnly");
byte[] tableName = Bytes.toBytes(sTableName);
byte[] cf = Bytes.toBytes("d"); byte[] cf = Bytes.toBytes("d");
UTIL.createTable(tableName, cf); UTIL.createTable(tableName, cf);
new HTable(UTIL.getConfiguration(), tableName).close(); new HTable(UTIL.getConfiguration(), tableName).close();
int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
assertDoesNotContainContent( 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"); "Table action request accepted");
assertDoesNotContainContent( assertDoesNotContainContent(
new URL("http://localhost:" + port + "/table.jsp?name=" + sTableName), new URL("http://localhost:" + port + "/table.jsp?name=" + tableName),
"Actions:"); "Actions:");
} }

View File

@ -144,7 +144,7 @@ public class TestMultiVersions {
this.admin.createTable(desc); this.admin.createTable(desc);
Put put = new Put(row, timestamp1); Put put = new Put(row, timestamp1);
put.add(contents, contents, value1); put.add(contents, contents, value1);
Table table = new HTable(UTIL.getConfiguration(), tableName); Table table = new HTable(UTIL.getConfiguration(), desc.getTableName());
table.put(put); table.put(put);
// Shut down and restart the HBase cluster // Shut down and restart the HBase cluster
table.close(); table.close();
@ -153,7 +153,7 @@ public class TestMultiVersions {
UTIL.startMiniHBaseCluster(1, NUM_SLAVES); UTIL.startMiniHBaseCluster(1, NUM_SLAVES);
// Make a new connection. Use new Configuration instance because old one // Make a new connection. Use new Configuration instance because old one
// is tied to an HConnection that has since gone stale. // 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 // Overwrite previous value
put = new Put(row, timestamp2); put = new Put(row, timestamp2);
put.add(contents, contents, value2); put.add(contents, contents, value2);

View File

@ -261,7 +261,7 @@ public class TestZooKeeper {
} }
Table table = 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 put = new Put(Bytes.toBytes("testrow"));
put.add(Bytes.toBytes("fam"), put.add(Bytes.toBytes("fam"),
Bytes.toBytes("col"), Bytes.toBytes("testdata")); Bytes.toBytes("col"), Bytes.toBytes("testdata"));
@ -543,7 +543,7 @@ public class TestZooKeeper {
htd.addFamily(hcd); htd.addFamily(hcd);
admin.createTable(htd, SPLIT_KEYS); admin.createTable(htd, SPLIT_KEYS);
TEST_UTIL.waitUntilNoRegionsInTransition(60000); TEST_UTIL.waitUntilNoRegionsInTransition(60000);
table = new HTable(TEST_UTIL.getConfiguration(), tableName); table = new HTable(TEST_UTIL.getConfiguration(), htd.getTableName());
Put p; Put p;
int numberOfPuts; int numberOfPuts;
for (numberOfPuts = 0; numberOfPuts < 6; numberOfPuts++) { for (numberOfPuts = 0; numberOfPuts < 6; numberOfPuts++) {

View File

@ -300,8 +300,8 @@ public class TestAdmin {
final byte [] row = Bytes.toBytes("row"); final byte [] row = Bytes.toBytes("row");
final byte [] qualifier = Bytes.toBytes("qualifier"); final byte [] qualifier = Bytes.toBytes("qualifier");
final byte [] value = Bytes.toBytes("value"); final byte [] value = Bytes.toBytes("value");
final byte [] table1 = Bytes.toBytes("testDisableAndEnableTable1"); final TableName table1 = TableName.valueOf("testDisableAndEnableTable1");
final byte [] table2 = Bytes.toBytes("testDisableAndEnableTable2"); final TableName table2 = TableName.valueOf("testDisableAndEnableTable2");
Table ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY); Table ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY);
Table ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY); Table ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY);
Put put = new Put(row); Put put = new Put(row);
@ -350,8 +350,7 @@ public class TestAdmin {
public void testCreateTable() throws IOException { public void testCreateTable() throws IOException {
HTableDescriptor [] tables = admin.listTables(); HTableDescriptor [] tables = admin.listTables();
int numTables = tables.length; int numTables = tables.length;
TEST_UTIL.createTable(Bytes.toBytes("testCreateTable"), TEST_UTIL.createTable(TableName.valueOf("testCreateTable"), HConstants.CATALOG_FAMILY).close();
HConstants.CATALOG_FAMILY).close();
tables = this.admin.listTables(); tables = this.admin.listTables();
assertEquals(numTables + 1, tables.length); assertEquals(numTables + 1, tables.length);
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster() assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
@ -411,7 +410,7 @@ public class TestAdmin {
htd.addFamily(fam2); htd.addFamily(fam2);
htd.addFamily(fam3); htd.addFamily(fam3);
this.admin.createTable(htd); 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(); HTableDescriptor confirmedHtd = table.getTableDescriptor();
assertEquals(htd.compareTo(confirmedHtd), 0); assertEquals(htd.compareTo(confirmedHtd), 0);
table.close(); table.close();
@ -1227,7 +1226,7 @@ public class TestAdmin {
admin.createTable(htd1); admin.createTable(htd1);
admin.createTable(htd2); admin.createTable(htd2);
// Before fix, below would fail throwing a NoServerForRegionException. // Before fix, below would fail throwing a NoServerForRegionException.
new HTable(TEST_UTIL.getConfiguration(), name).close(); new HTable(TEST_UTIL.getConfiguration(), htd2.getTableName()).close();
} }
/*** /***
@ -1264,7 +1263,7 @@ public class TestAdmin {
*/ */
@Test (timeout=300000) @Test (timeout=300000)
public void testReadOnlyTable() throws Exception { public void testReadOnlyTable() throws Exception {
byte [] name = Bytes.toBytes("testReadOnlyTable"); TableName name = TableName.valueOf("testReadOnlyTable");
Table table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY); Table table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
byte[] value = Bytes.toBytes("somedata"); byte[] value = Bytes.toBytes("somedata");
// This used to use an empty row... That must have been a bug // This used to use an empty row... That must have been a bug
@ -1310,7 +1309,7 @@ public class TestAdmin {
*/ */
@Test (expected=TableExistsException.class, timeout=300000) @Test (expected=TableExistsException.class, timeout=300000)
public void testTableExistsExceptionWithATable() throws IOException { 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).close();
TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY); TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
} }
@ -1348,8 +1347,9 @@ public class TestAdmin {
*/ */
@Test (expected=TableNotFoundException.class, timeout=300000) @Test (expected=TableNotFoundException.class, timeout=300000)
public void testTableNotFoundExceptionWithoutAnyTables() throws IOException { public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
Table ht = TableName tableName = TableName
new HTable(TEST_UTIL.getConfiguration(),"testTableNotFoundExceptionWithoutAnyTables"); .valueOf("testTableNotFoundExceptionWithoutAnyTables");
Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
ht.get(new Get("e".getBytes())); ht.get(new Get("e".getBytes()));
} }
@ -1390,7 +1390,7 @@ public class TestAdmin {
createTableWithDefaultConf(TABLENAME); createTableWithDefaultConf(TABLENAME);
HRegionInfo info = null; HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME); HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME));
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()); List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (HRegionInfo regionInfo : onlineRegions) { for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) { if (!regionInfo.isMetaTable()) {
@ -1448,7 +1448,7 @@ public class TestAdmin {
byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion3"); byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion3");
createTableWithDefaultConf(TABLENAME); createTableWithDefaultConf(TABLENAME);
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME); HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME));
try { try {
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()); List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
@ -1472,7 +1472,7 @@ public class TestAdmin {
byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegionWhenServerNameIsEmpty"); byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegionWhenServerNameIsEmpty");
createTableWithDefaultConf(TABLENAME); createTableWithDefaultConf(TABLENAME);
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME); HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME));
try { try {
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()); List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
@ -1496,7 +1496,7 @@ public class TestAdmin {
createTableWithDefaultConf(TABLENAME); createTableWithDefaultConf(TABLENAME);
HRegionInfo info = null; HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME); HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME));
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()); List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (HRegionInfo regionInfo : onlineRegions) { for (HRegionInfo regionInfo : onlineRegions) {
@ -1579,7 +1579,7 @@ public class TestAdmin {
v.append(className); v.append(className);
} }
byte[] value = Bytes.toBytes(v.toString()); byte[] value = Bytes.toBytes(v.toString());
HRegionServer regionServer = startAndWriteData("TestLogRolling", value); HRegionServer regionServer = startAndWriteData(TableName.valueOf("TestLogRolling"), value);
LOG.info("after writing there are " LOG.info("after writing there are "
+ HLogUtilsForTests.getNumRolledLogFiles(regionServer.getWAL()) + " log files"); + HLogUtilsForTests.getNumRolledLogFiles(regionServer.getWAL()) + " log files");
@ -1659,19 +1659,19 @@ public class TestAdmin {
"hbase.regionserver.hlog.lowreplication.rolllimit", 3); "hbase.regionserver.hlog.lowreplication.rolllimit", 3);
} }
private HRegionServer startAndWriteData(String tableName, byte[] value) private HRegionServer startAndWriteData(TableName tableName, byte[] value)
throws IOException, InterruptedException { throws IOException, InterruptedException {
// When the hbase:meta table can be opened, the region servers are running // When the hbase:meta table can be opened, the region servers are running
new HTable( new HTable(
TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME).close(); TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME).close();
// Create the test table and open it // 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)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); admin.createTable(desc);
Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); 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 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 put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
put.add(HConstants.CATALOG_FAMILY, null, value); put.add(HConstants.CATALOG_FAMILY, null, value);
@ -1772,10 +1772,9 @@ public class TestAdmin {
// here because makes use of an internal HBA method (TODO: Fix.). // here because makes use of an internal HBA method (TODO: Fix.).
HBaseAdmin rawAdmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HBaseAdmin rawAdmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
final String name = "testGetRegion"; final TableName tableName = TableName.valueOf("testGetRegion");
LOG.info("Started " + name); LOG.info("Started " + tableName);
final byte [] nameBytes = Bytes.toBytes(name); HTable t = TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
HTable t = TEST_UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
TEST_UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY); TEST_UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
HRegionLocation regionLocation = t.getRegionLocation("mmm"); HRegionLocation regionLocation = t.getRegionLocation("mmm");

View File

@ -84,7 +84,7 @@ public class TestClientScannerRPCTimeout {
@Test @Test
public void testScannerNextRPCTimesout() throws Exception { 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); Table ht = TEST_UTIL.createTable(TABLE_NAME, FAMILY);
byte[] r1 = Bytes.toBytes("row-1"); byte[] r1 = Bytes.toBytes("row-1");
byte[] r2 = Bytes.toBytes("row-2"); byte[] r2 = Bytes.toBytes("row-2");

View File

@ -182,7 +182,7 @@ public class TestFromClientSide {
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY) HColumnDescriptor hcd = new HColumnDescriptor(FAMILY)
.setKeepDeletedCells(true).setMaxVersions(3); .setKeepDeletedCells(true).setMaxVersions(3);
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TABLENAME)); HTableDescriptor desc = new HTableDescriptor(TABLENAME);
desc.addFamily(hcd); desc.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(desc); TEST_UTIL.getHBaseAdmin().createTable(desc);
Configuration c = TEST_UTIL.getConfiguration(); Configuration c = TEST_UTIL.getConfiguration();
@ -385,7 +385,7 @@ public class TestFromClientSide {
*/ */
@Test @Test
public void testGetConfiguration() throws Exception { public void testGetConfiguration() throws Exception {
byte[] TABLE = Bytes.toBytes("testGetConfiguration"); TableName TABLE = TableName.valueOf("testGetConfiguration");
byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") }; byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
Configuration conf = TEST_UTIL.getConfiguration(); Configuration conf = TEST_UTIL.getConfiguration();
Table table = TEST_UTIL.createTable(TABLE, FAMILIES, conf); Table table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
@ -533,7 +533,7 @@ public class TestFromClientSide {
@Test @Test
public void testFilterAcrossMultipleRegions() public void testFilterAcrossMultipleRegions()
throws IOException, InterruptedException { throws IOException, InterruptedException {
byte [] name = Bytes.toBytes("testFilterAcrossMutlipleRegions"); TableName name = TableName.valueOf("testFilterAcrossMutlipleRegions");
HTable t = TEST_UTIL.createTable(name, FAMILY); HTable t = TEST_UTIL.createTable(name, FAMILY);
int rowCount = TEST_UTIL.loadTable(t, FAMILY, false); int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
assertRowCount(t, rowCount); assertRowCount(t, rowCount);
@ -1208,7 +1208,7 @@ public class TestFromClientSide {
// Null family (should NOT work) // Null family (should NOT work)
try { 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"); fail("Creating a table with a null family passed, should fail");
} catch(Exception e) {} } catch(Exception e) {}
@ -1243,7 +1243,7 @@ public class TestFromClientSide {
// Use a new table // Use a new table
byte [] TABLE2 = Bytes.toBytes("testNull2"); 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) // Empty qualifier, byte[0] instead of null (should work)
try { try {
@ -3597,8 +3597,7 @@ public class TestFromClientSide {
@Test @Test
public void testUpdatesWithMajorCompaction() throws Exception { public void testUpdatesWithMajorCompaction() throws Exception {
String tableName = "testUpdatesWithMajorCompaction"; TableName TABLE = TableName.valueOf("testUpdatesWithMajorCompaction");
byte [] TABLE = Bytes.toBytes(tableName);
Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
@ -3630,8 +3629,8 @@ public class TestFromClientSide {
assertEquals("BBB", Bytes.toString(navigableMap.get(2L))); assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
// Trigger a major compaction // Trigger a major compaction
admin.flush(tableName); admin.flush(TABLE);
admin.majorCompact(tableName); admin.majorCompact(TABLE);
Thread.sleep(6000); Thread.sleep(6000);
// Update the value at timestamp 1 // Update the value at timestamp 1
@ -3645,8 +3644,8 @@ public class TestFromClientSide {
hTable.put(put); hTable.put(put);
// Trigger a major compaction // Trigger a major compaction
admin.flush(tableName); admin.flush(TABLE);
admin.majorCompact(tableName); admin.majorCompact(TABLE);
Thread.sleep(6000); Thread.sleep(6000);
// Check that the values at timestamp 2 and 1 got updated // Check that the values at timestamp 2 and 1 got updated
@ -3726,7 +3725,7 @@ public class TestFromClientSide {
@Test @Test
public void testGet_EmptyTable() throws IOException { 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 get = new Get(ROW);
get.addFamily(FAMILY); get.addFamily(FAMILY);
Result r = table.get(get); Result r = table.get(get);
@ -3735,7 +3734,7 @@ public class TestFromClientSide {
@Test @Test
public void testGet_NullQualifier() throws IOException { 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 put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE); put.add(FAMILY, QUALIFIER, VALUE);
table.put(put); table.put(put);
@ -3758,7 +3757,7 @@ public class TestFromClientSide {
@Test @Test
public void testGet_NonExistentRow() throws IOException { 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 put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE); put.add(FAMILY, QUALIFIER, VALUE);
table.put(put); table.put(put);
@ -3822,7 +3821,7 @@ public class TestFromClientSide {
public void testPutNoCF() throws IOException { public void testPutNoCF() throws IOException {
final byte[] BAD_FAM = Bytes.toBytes("BAD_CF"); final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
final byte[] VAL = Bytes.toBytes(100); 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; boolean caughtNSCFE = false;
@ -4064,10 +4063,10 @@ public class TestFromClientSide {
@Test @Test
public void testListTables() throws IOException, InterruptedException { public void testListTables() throws IOException, InterruptedException {
byte [] t1 = Bytes.toBytes("testListTables1"); TableName t1 = TableName.valueOf("testListTables1");
byte [] t2 = Bytes.toBytes("testListTables2"); TableName t2 = TableName.valueOf("testListTables2");
byte [] t3 = Bytes.toBytes("testListTables3"); TableName t3 = TableName.valueOf("testListTables3");
byte [][] tables = new byte[][] { t1, t2, t3 }; TableName [] tables = new TableName[] { t1, t2, t3 };
for (int i = 0; i < tables.length; i++) { for (int i = 0; i < tables.length; i++) {
TEST_UTIL.createTable(tables[i], FAMILY); TEST_UTIL.createTable(tables[i], FAMILY);
} }
@ -4080,12 +4079,12 @@ public class TestFromClientSide {
for (int i = 0; i < tables.length && i < size; i++) { for (int i = 0; i < tables.length && i < size; i++) {
boolean found = false; boolean found = false;
for (int j = 0; j < ts.length; j++) { 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; found = true;
break; break;
} }
} }
assertTrue("Not found: " + Bytes.toString(tables[i]), found); assertTrue("Not found: " + tables[i], found);
} }
} }
@ -4096,7 +4095,7 @@ public class TestFromClientSide {
* @return the created HTable object * @return the created HTable object
* @throws IOException * @throws IOException
*/ */
HTable createUnmangedHConnectionHTable(final byte [] tableName) throws IOException { HTable createUnmangedHConnectionHTable(final TableName tableName) throws IOException {
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
return (HTable)conn.getTable(tableName); return (HTable)conn.getTable(tableName);
@ -4110,7 +4109,7 @@ public class TestFromClientSide {
*/ */
@Test @Test
public void testUnmanagedHConnection() throws IOException { public void testUnmanagedHConnection() throws IOException {
final byte[] tableName = Bytes.toBytes("testUnmanagedHConnection"); final TableName tableName = TableName.valueOf("testUnmanagedHConnection");
HTable t = createUnmangedHConnectionHTable(tableName); HTable t = createUnmangedHConnectionHTable(tableName);
HBaseAdmin ha = new HBaseAdmin(t.getConnection()); HBaseAdmin ha = new HBaseAdmin(t.getConnection());
assertTrue(ha.tableExists(tableName)); assertTrue(ha.tableExists(tableName));
@ -4125,7 +4124,7 @@ public class TestFromClientSide {
*/ */
@Test @Test
public void testUnmanagedHConnectionReconnect() throws Exception { public void testUnmanagedHConnectionReconnect() throws Exception {
final byte[] tableName = Bytes.toBytes("testUnmanagedHConnectionReconnect"); final TableName tableName = TableName.valueOf("testUnmanagedHConnectionReconnect");
HTable t = createUnmangedHConnectionHTable(tableName); HTable t = createUnmangedHConnectionHTable(tableName);
Connection conn = t.getConnection(); Connection conn = t.getConnection();
HBaseAdmin ha = new HBaseAdmin(conn); HBaseAdmin ha = new HBaseAdmin(conn);
@ -4150,8 +4149,8 @@ public class TestFromClientSide {
@Test @Test
public void testMiscHTableStuff() throws IOException { public void testMiscHTableStuff() throws IOException {
final byte[] tableAname = Bytes.toBytes("testMiscHTableStuffA"); final TableName tableAname = TableName.valueOf("testMiscHTableStuffA");
final byte[] tableBname = Bytes.toBytes("testMiscHTableStuffB"); final TableName tableBname = TableName.valueOf("testMiscHTableStuffB");
final byte[] attrName = Bytes.toBytes("TESTATTR"); final byte[] attrName = Bytes.toBytes("TESTATTR");
final byte[] attrValue = Bytes.toBytes("somevalue"); final byte[] attrValue = Bytes.toBytes("somevalue");
byte[] value = Bytes.toBytes("value"); byte[] value = Bytes.toBytes("value");
@ -4193,7 +4192,7 @@ public class TestFromClientSide {
// to be reloaded. // to be reloaded.
// Test user metadata // Test user metadata
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
// make a modifiable descriptor // make a modifiable descriptor
HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor()); HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor());
// offline the table // offline the table
@ -4211,7 +4210,7 @@ public class TestFromClientSide {
// Test that attribute changes were applied // Test that attribute changes were applied
desc = a.getTableDescriptor(); desc = a.getTableDescriptor();
assertTrue("wrong table descriptor returned", assertTrue("wrong table descriptor returned",
Bytes.compareTo(desc.getTableName().getName(), tableAname) == 0); desc.getTableName().equals(tableAname));
// check HTD attribute // check HTD attribute
value = desc.getValue(attrName); value = desc.getValue(attrName);
assertFalse("missing HTD attribute value", value == null); assertFalse("missing HTD attribute value", value == null);
@ -4228,7 +4227,7 @@ public class TestFromClientSide {
@Test @Test
public void testGetClosestRowBefore() throws IOException, InterruptedException { 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[] firstRow = Bytes.toBytes("row111");
final byte[] secondRow = Bytes.toBytes("row222"); final byte[] secondRow = Bytes.toBytes("row222");
final byte[] thirdRow = Bytes.toBytes("row333"); final byte[] thirdRow = Bytes.toBytes("row333");
@ -4342,7 +4341,7 @@ public class TestFromClientSide {
@Test @Test
public void testMultiRowMutation() throws Exception { public void testMultiRowMutation() throws Exception {
LOG.info("Starting testMultiRowMutation"); LOG.info("Starting testMultiRowMutation");
final byte [] TABLENAME = Bytes.toBytes("testMultiRowMutation"); final TableName TABLENAME = TableName.valueOf("testMultiRowMutation");
final byte [] ROW1 = Bytes.toBytes("testRow1"); final byte [] ROW1 = Bytes.toBytes("testRow1");
Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
@ -4373,7 +4372,7 @@ public class TestFromClientSide {
@Test @Test
public void testRowMutation() throws Exception { public void testRowMutation() throws Exception {
LOG.info("Starting testRowMutation"); LOG.info("Starting testRowMutation");
final byte [] TABLENAME = Bytes.toBytes("testRowMutation"); final TableName TABLENAME = TableName.valueOf("testRowMutation");
Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
byte [][] QUALIFIERS = new byte [][] { byte [][] QUALIFIERS = new byte [][] {
Bytes.toBytes("a"), Bytes.toBytes("b") Bytes.toBytes("a"), Bytes.toBytes("b")
@ -4405,7 +4404,7 @@ public class TestFromClientSide {
@Test @Test
public void testAppend() throws Exception { public void testAppend() throws Exception {
LOG.info("Starting testAppend"); LOG.info("Starting testAppend");
final byte [] TABLENAME = Bytes.toBytes("testAppend"); final TableName TABLENAME = TableName.valueOf("testAppend");
Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
byte[] v1 = Bytes.toBytes("42"); byte[] v1 = Bytes.toBytes("42");
byte[] v2 = Bytes.toBytes("23"); byte[] v2 = Bytes.toBytes("23");
@ -4456,7 +4455,7 @@ public class TestFromClientSide {
@Test @Test
public void testIncrementingInvalidValue() throws Exception { public void testIncrementingInvalidValue() throws Exception {
LOG.info("Starting testIncrementingInvalidValue"); LOG.info("Starting testIncrementingInvalidValue");
final byte [] TABLENAME = Bytes.toBytes("testIncrementingInvalidValue"); final TableName TABLENAME = TableName.valueOf("testIncrementingInvalidValue");
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
final byte[] COLUMN = Bytes.toBytes("column"); final byte[] COLUMN = Bytes.toBytes("column");
Put p = new Put(ROW); Put p = new Put(ROW);
@ -4482,7 +4481,7 @@ public class TestFromClientSide {
@Test @Test
public void testIncrementInvalidArguments() throws Exception { public void testIncrementInvalidArguments() throws Exception {
LOG.info("Starting testIncrementInvalidArguments"); LOG.info("Starting testIncrementInvalidArguments");
final byte[] TABLENAME = Bytes.toBytes("testIncrementInvalidArguments"); final TableName TABLENAME = TableName.valueOf("testIncrementInvalidArguments");
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
final byte[] COLUMN = Bytes.toBytes("column"); final byte[] COLUMN = Bytes.toBytes("column");
try { try {
@ -4537,7 +4536,7 @@ public class TestFromClientSide {
@Test @Test
public void testIncrementOutOfOrder() throws Exception { public void testIncrementOutOfOrder() throws Exception {
LOG.info("Starting testIncrementOutOfOrder"); LOG.info("Starting testIncrementOutOfOrder");
final byte [] TABLENAME = Bytes.toBytes("testIncrementOutOfOrder"); final TableName TABLENAME = TableName.valueOf("testIncrementOutOfOrder");
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
byte [][] QUALIFIERS = new byte [][] { byte [][] QUALIFIERS = new byte [][] {
@ -4577,7 +4576,7 @@ public class TestFromClientSide {
@Test @Test
public void testIncrement() throws Exception { public void testIncrement() throws Exception {
LOG.info("Starting testIncrement"); LOG.info("Starting testIncrement");
final byte [] TABLENAME = Bytes.toBytes("testIncrement"); final TableName TABLENAME = TableName.valueOf("testIncrement");
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
byte [][] ROWS = new byte [][] { byte [][] ROWS = new byte [][] {
@ -4648,7 +4647,7 @@ public class TestFromClientSide {
@Test @Test
public void testClientPoolRoundRobin() throws IOException { public void testClientPoolRoundRobin() throws IOException {
final byte[] tableName = Bytes.toBytes("testClientPoolRoundRobin"); final TableName tableName = TableName.valueOf("testClientPoolRoundRobin");
int poolSize = 3; int poolSize = 3;
int numVersions = poolSize * 2; int numVersions = poolSize * 2;
@ -4656,8 +4655,7 @@ public class TestFromClientSide {
conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin"); conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin");
conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, conf, Integer.MAX_VALUE);
conf, Integer.MAX_VALUE);
final long ts = EnvironmentEdgeManager.currentTime(); final long ts = EnvironmentEdgeManager.currentTime();
Get get = new Get(ROW); Get get = new Get(ROW);
@ -4685,7 +4683,7 @@ public class TestFromClientSide {
@Ignore ("Flakey: HBASE-8989") @Test @Ignore ("Flakey: HBASE-8989") @Test
public void testClientPoolThreadLocal() throws IOException { public void testClientPoolThreadLocal() throws IOException {
final byte[] tableName = Bytes.toBytes("testClientPoolThreadLocal"); final TableName tableName = TableName.valueOf("testClientPoolThreadLocal");
int poolSize = Integer.MAX_VALUE; int poolSize = Integer.MAX_VALUE;
int numVersions = 3; int numVersions = 3;
@ -4771,8 +4769,7 @@ public class TestFromClientSide {
final byte [] anotherrow = Bytes.toBytes("anotherrow"); final byte [] anotherrow = Bytes.toBytes("anotherrow");
final byte [] value2 = Bytes.toBytes("abcd"); final byte [] value2 = Bytes.toBytes("abcd");
Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"), Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndPut"), FAMILY);
new byte [][] {FAMILY});
Put put1 = new Put(ROW); Put put1 = new Put(ROW);
put1.add(FAMILY, QUALIFIER, VALUE); put1.add(FAMILY, QUALIFIER, VALUE);
@ -4813,8 +4810,7 @@ public class TestFromClientSide {
final byte [] value3 = Bytes.toBytes("cccc"); final byte [] value3 = Bytes.toBytes("cccc");
final byte [] value4 = Bytes.toBytes("dddd"); final byte [] value4 = Bytes.toBytes("dddd");
Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPutWithCompareOp"), Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndPutWithCompareOp"), FAMILY);
new byte [][] {FAMILY});
Put put2 = new Put(ROW); Put put2 = new Put(ROW);
put2.add(FAMILY, QUALIFIER, value2); put2.add(FAMILY, QUALIFIER, value2);
@ -4879,8 +4875,8 @@ public class TestFromClientSide {
final byte [] value3 = Bytes.toBytes("cccc"); final byte [] value3 = Bytes.toBytes("cccc");
final byte [] value4 = Bytes.toBytes("dddd"); final byte [] value4 = Bytes.toBytes("dddd");
Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndDeleteWithCompareOp"), Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndDeleteWithCompareOp"),
new byte [][] {FAMILY}); FAMILY);
Put put2 = new Put(ROW); Put put2 = new Put(ROW);
put2.add(FAMILY, QUALIFIER, value2); put2.add(FAMILY, QUALIFIER, value2);
@ -4953,7 +4949,7 @@ public class TestFromClientSide {
@Test @Test
@SuppressWarnings ("unused") @SuppressWarnings ("unused")
public void testScanMetrics() throws Exception { public void testScanMetrics() throws Exception {
byte [] TABLENAME = Bytes.toBytes("testScanMetrics"); TableName TABLENAME = TableName.valueOf("testScanMetrics");
Configuration conf = TEST_UTIL.getConfiguration(); Configuration conf = TEST_UTIL.getConfiguration();
TEST_UTIL.createTable(TABLENAME, FAMILY); TEST_UTIL.createTable(TABLENAME, FAMILY);
@ -5057,13 +5053,12 @@ public class TestFromClientSide {
*/ */
@Test @Test
public void testCacheOnWriteEvictOnClose() throws Exception { public void testCacheOnWriteEvictOnClose() throws Exception {
byte [] tableName = Bytes.toBytes("testCOWEOCfromClient"); TableName tableName = TableName.valueOf("testCOWEOCfromClient");
byte [] data = Bytes.toBytes("data"); 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 // get the block cache and region
String regionName = table.getRegionLocations().firstKey().getEncodedName(); String regionName = table.getRegionLocations().firstKey().getEncodedName();
HRegion region = TEST_UTIL.getRSForFirstRegionInTable( HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
tableName).getFromOnlineRegions(regionName);
Store store = region.getStores().values().iterator().next(); Store store = region.getStores().values().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
@ -5171,8 +5166,7 @@ public class TestFromClientSide {
*/ */
public void testNonCachedGetRegionLocation() throws Exception { public void testNonCachedGetRegionLocation() throws Exception {
// Test Initialization. // Test Initialization.
String tableName = "testNonCachedGetRegionLocation"; TableName TABLE = TableName.valueOf("testNonCachedGetRegionLocation");
byte [] TABLE = Bytes.toBytes(tableName);
byte [] family1 = Bytes.toBytes("f1"); byte [] family1 = Bytes.toBytes("f1");
byte [] family2 = Bytes.toBytes("f2"); byte [] family2 = Bytes.toBytes("f2");
HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10); HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10);
@ -5220,7 +5214,7 @@ public class TestFromClientSide {
// Test Initialization. // Test Initialization.
byte [] startKey = Bytes.toBytes("ddc"); byte [] startKey = Bytes.toBytes("ddc");
byte [] endKey = Bytes.toBytes("mmm"); 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); HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
int numOfRegions = TEST_UTIL.createMultiRegions(table, FAMILY); int numOfRegions = TEST_UTIL.createMultiRegions(table, FAMILY);
assertEquals(25, numOfRegions); assertEquals(25, numOfRegions);
@ -5271,7 +5265,7 @@ public class TestFromClientSide {
@Test @Test
public void testJira6912() throws Exception { 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); Table foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
@ -5297,7 +5291,7 @@ public class TestFromClientSide {
@Test @Test
public void testScan_NullQualifier() throws IOException { 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 put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE); put.add(FAMILY, QUALIFIER, VALUE);
table.put(put); table.put(put);
@ -5326,7 +5320,7 @@ public class TestFromClientSide {
@Test @Test
public void testNegativeTimestamp() throws IOException { public void testNegativeTimestamp() throws IOException {
Table table = TEST_UTIL.createTable(Bytes.toBytes("testNegativeTimestamp"), FAMILY); Table table = TEST_UTIL.createTable(TableName.valueOf("testNegativeTimestamp"), FAMILY);
try { try {
Put put = new Put(ROW, -1); Put put = new Put(ROW, -1);
@ -5470,8 +5464,8 @@ public class TestFromClientSide {
@Test @Test
public void testRawScanRespectsVersions() throws Exception { public void testRawScanRespectsVersions() throws Exception {
byte[] TABLE = Bytes.toBytes("testRawScan"); TableName TABLE = TableName.valueOf("testRawScan");
Table table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }); Table table = TEST_UTIL.createTable(TABLE, FAMILY);
byte[] row = Bytes.toBytes("row"); byte[] row = Bytes.toBytes("row");
// put the same row 4 times, with different values // put the same row 4 times, with different values
@ -5546,7 +5540,7 @@ public class TestFromClientSide {
@Test @Test
public void testSmallScan() throws Exception { public void testSmallScan() throws Exception {
// Test Initialization. // Test Initialization.
byte[] TABLE = Bytes.toBytes("testSmallScan"); TableName TABLE = TableName.valueOf("testSmallScan");
Table table = TEST_UTIL.createTable(TABLE, FAMILY); Table table = TEST_UTIL.createTable(TABLE, FAMILY);
// Insert one row each region // Insert one row each region
@ -5582,7 +5576,7 @@ public class TestFromClientSide {
@Test @Test
public void testSuperSimpleWithReverseScan() throws Exception { public void testSuperSimpleWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testSuperSimpleWithReverseScan"); TableName TABLE = TableName.valueOf("testSuperSimpleWithReverseScan");
Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000")); Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000"));
put.add(FAMILY, QUALIFIER, VALUE); put.add(FAMILY, QUALIFIER, VALUE);
@ -5628,7 +5622,7 @@ public class TestFromClientSide {
@Test @Test
public void testFiltersWithReverseScan() throws Exception { public void testFiltersWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testFiltersWithReverseScan"); TableName TABLE = TableName.valueOf("testFiltersWithReverseScan");
Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte[][] ROWS = makeN(ROW, 10); byte[][] ROWS = makeN(ROW, 10);
byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"), byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
@ -5668,7 +5662,7 @@ public class TestFromClientSide {
@Test @Test
public void testKeyOnlyFilterWithReverseScan() throws Exception { public void testKeyOnlyFilterWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testKeyOnlyFilterWithReverseScan"); TableName TABLE = TableName.valueOf("testKeyOnlyFilterWithReverseScan");
Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte[][] ROWS = makeN(ROW, 10); byte[][] ROWS = makeN(ROW, 10);
byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"), byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
@ -5709,7 +5703,7 @@ public class TestFromClientSide {
*/ */
@Test @Test
public void testSimpleMissingWithReverseScan() throws Exception { public void testSimpleMissingWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testSimpleMissingWithReverseScan"); TableName TABLE = TableName.valueOf("testSimpleMissingWithReverseScan");
Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte[][] ROWS = makeN(ROW, 4); byte[][] ROWS = makeN(ROW, 4);
@ -5774,7 +5768,7 @@ public class TestFromClientSide {
@Test @Test
public void testNullWithReverseScan() throws Exception { public void testNullWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testNullWithReverseScan"); TableName TABLE = TableName.valueOf("testNullWithReverseScan");
Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
// Null qualifier (should work) // Null qualifier (should work)
Put put = new Put(ROW); Put put = new Put(ROW);
@ -5786,7 +5780,7 @@ public class TestFromClientSide {
ht.delete(delete); ht.delete(delete);
// Use a new table // Use a new table
byte[] TABLE2 = Bytes.toBytes("testNull2WithReverseScan"); 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) // Empty qualifier, byte[0] instead of null (should work)
put = new Put(ROW); put = new Put(ROW);
put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE); put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
@ -5811,13 +5805,12 @@ public class TestFromClientSide {
@Test @Test
public void testDeletesWithReverseScan() throws Exception { public void testDeletesWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testDeletesWithReverseScan"); TableName TABLE = TableName.valueOf("testDeletesWithReverseScan");
byte[][] ROWS = makeNAscii(ROW, 6); byte[][] ROWS = makeNAscii(ROW, 6);
byte[][] FAMILIES = makeNAscii(FAMILY, 3); byte[][] FAMILIES = makeNAscii(FAMILY, 3);
byte[][] VALUES = makeN(VALUE, 5); byte[][] VALUES = makeN(VALUE, 5);
long[] ts = { 1000, 2000, 3000, 4000, 5000 }; long[] ts = { 1000, 2000, 3000, 4000, 5000 };
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, TEST_UTIL.getConfiguration(), 3);
TEST_UTIL.getConfiguration(), 3);
Put put = new Put(ROW); Put put = new Put(ROW);
put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
@ -5998,7 +5991,7 @@ public class TestFromClientSide {
@Test @Test
public void testReversedScanUnderMultiRegions() throws Exception { public void testReversedScanUnderMultiRegions() throws Exception {
// Test Initialization. // Test Initialization.
byte[] TABLE = Bytes.toBytes("testReversedScanUnderMultiRegions"); TableName TABLE = TableName.valueOf("testReversedScanUnderMultiRegions");
byte[] maxByteArray = ReversedClientScanner.MAX_BYTE_ARRAY; byte[] maxByteArray = ReversedClientScanner.MAX_BYTE_ARRAY;
byte[][] splitRows = new byte[][] { Bytes.toBytes("005"), byte[][] splitRows = new byte[][] { Bytes.toBytes("005"),
Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)), Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)),
@ -6055,7 +6048,7 @@ public class TestFromClientSide {
@Test @Test
public void testSmallReversedScanUnderMultiRegions() throws Exception { public void testSmallReversedScanUnderMultiRegions() throws Exception {
// Test Initialization. // Test Initialization.
byte[] TABLE = Bytes.toBytes("testSmallReversedScanUnderMultiRegions"); TableName TABLE = TableName.valueOf("testSmallReversedScanUnderMultiRegions");
byte[][] splitRows = new byte[][]{ byte[][] splitRows = new byte[][]{
Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"), Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"),
Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")}; Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")};

View File

@ -338,8 +338,8 @@ public class TestFromClientSide3 {
public void testHTableExistsMethodMultipleRegionsSingleGet() throws Exception { public void testHTableExistsMethodMultipleRegionsSingleGet() throws Exception {
Table table = TEST_UTIL.createTable( Table table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY }, 1, TableName.valueOf("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY },
new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255); 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
Put put = new Put(ROW); Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE); put.add(FAMILY, QUALIFIER, VALUE);
@ -357,8 +357,8 @@ public class TestFromClientSide3 {
@Test @Test
public void testHTableExistsMethodMultipleRegionsMultipleGets() throws Exception { public void testHTableExistsMethodMultipleRegionsMultipleGets() throws Exception {
HTable table = TEST_UTIL.createTable( HTable table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableExistsMethodMultipleRegionsMultipleGets"), new byte[][] { FAMILY }, TableName.valueOf("testHTableExistsMethodMultipleRegionsMultipleGets"),
1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255); new byte[][] { FAMILY }, 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
Put put = new Put(ROW); Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE); put.add(FAMILY, QUALIFIER, VALUE);
table.put (put); table.put (put);
@ -410,7 +410,7 @@ public class TestFromClientSide3 {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test"))); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test")));
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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 put = new Put(ROW_BYTES);
put.add(FAMILY, COL_QUAL, VAL_BYTES); put.add(FAMILY, COL_QUAL, VAL_BYTES);

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.exceptions.OperationConflictException; import org.apache.hadoop.hbase.exceptions.OperationConflictException;
import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcClient;
@ -67,7 +68,7 @@ public class TestMultiParallel {
private static final byte[] VALUE = Bytes.toBytes("value"); private static final byte[] VALUE = Bytes.toBytes("value");
private static final byte[] QUALIFIER = Bytes.toBytes("qual"); private static final byte[] QUALIFIER = Bytes.toBytes("qual");
private static final String FAMILY = "family"; 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[] BYTES_FAMILY = Bytes.toBytes(FAMILY);
private static final byte[] ONE_ROW = Bytes.toBytes("xxx"); private static final byte[] ONE_ROW = Bytes.toBytes("xxx");
private static final byte [][] KEYS = makeKeys(); private static final byte [][] KEYS = makeKeys();
@ -79,9 +80,9 @@ public class TestMultiParallel {
((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
UTIL.startMiniCluster(slaves); 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.createMultiRegions(t, Bytes.toBytes(FAMILY));
UTIL.waitTableEnabled(Bytes.toBytes(TEST_TABLE)); UTIL.waitTableEnabled(TEST_TABLE.getName());
t.close(); t.close();
} }

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -48,7 +49,7 @@ public class TestScannerTimeout {
final Log LOG = LogFactory.getLog(getClass()); final Log LOG = LogFactory.getLog(getClass());
private final static byte[] SOME_BYTES = Bytes.toBytes("f"); 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; private final static int NB_ROWS = 10;
// Be careful w/ what you set this timer to... it can get in the way of // 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. // the mini cluster coming up -- the verification in particular.

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTestConst; import org.apache.hadoop.hbase.HTestConst;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster; 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.ColumnPrefixFilter;
import org.apache.hadoop.hbase.filter.ColumnRangeFilter; import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
@ -101,7 +102,7 @@ public class TestScannersFromClientSide {
*/ */
@Test @Test
public void testScanBatch() throws Exception { public void testScanBatch() throws Exception {
byte [] TABLE = Bytes.toBytes("testScanBatch"); TableName TABLE = TableName.valueOf("testScanBatch");
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 8); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 8);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
@ -442,7 +443,7 @@ public class TestScannersFromClientSide {
*/ */
@Test @Test
public void testScanOnReopenedRegion() throws Exception { public void testScanOnReopenedRegion() throws Exception {
byte [] TABLE = Bytes.toBytes("testScanOnReopenedRegion"); TableName TABLE = TableName.valueOf("testScanOnReopenedRegion");
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 2); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 2);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);

View File

@ -58,7 +58,7 @@ public class TestSnapshotCloneIndependence {
private static final String STRING_TABLE_NAME = "test"; private static final String STRING_TABLE_NAME = "test";
private static final String TEST_FAM_STR = "fam"; private static final String TEST_FAM_STR = "fam";
private static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR); 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 * Setup the config for the cluster and start it

View File

@ -98,7 +98,7 @@ public class TestTimestampsFilter {
Cell kvs[]; Cell kvs[];
// create table; set versions to max... // 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 rowIdx = 0; rowIdx < 5; rowIdx++) {
for (int colIdx = 0; colIdx < 5; colIdx++) { for (int colIdx = 0; colIdx < 5; colIdx++) {
@ -173,7 +173,7 @@ public class TestTimestampsFilter {
byte [][] FAMILIES = new byte[][] { FAMILY }; byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max... // 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")); Put p = new Put(Bytes.toBytes("row"));
p.add(FAMILY, Bytes.toBytes("column0"), 3, Bytes.toBytes("value0-3")); p.add(FAMILY, Bytes.toBytes("column0"), 3, Bytes.toBytes("value0-3"));
@ -233,7 +233,7 @@ public class TestTimestampsFilter {
byte [][] FAMILIES = new byte[][] { FAMILY }; byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max... // 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. // For row:0, col:0: insert versions 1 through 5.
putNVersions(ht, FAMILY, 0, 0, 1, 5); putNVersions(ht, FAMILY, 0, 0, 1, 5);

View File

@ -66,7 +66,7 @@ public class TestHTableWrapper {
private static final HBaseTestingUtility util = new HBaseTestingUtility(); 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[] TEST_FAMILY = Bytes.toBytes("f1");
private static final byte[] ROW_A = Bytes.toBytes("aaa"); private static final byte[] ROW_A = Bytes.toBytes("aaa");
@ -136,7 +136,7 @@ public class TestHTableWrapper {
CoprocessorEnvironment env = cpHost.findCoprocessorEnvironment(implClazz.getName()); CoprocessorEnvironment env = cpHost.findCoprocessorEnvironment(implClazz.getName());
assertEquals(Coprocessor.VERSION, env.getVersion()); assertEquals(Coprocessor.VERSION, env.getVersion());
assertEquals(VersionInfo.getVersion(), env.getHBaseVersion()); assertEquals(VersionInfo.getVersion(), env.getHBaseVersion());
hTableInterface = env.getTable(TableName.valueOf(TEST_TABLE)); hTableInterface = env.getTable(TEST_TABLE);
checkHTableInterfaceMethods(); checkHTableInterfaceMethods();
cpHost.shutdown(env); cpHost.shutdown(env);
} }
@ -170,7 +170,7 @@ public class TestHTableWrapper {
} }
private void checkNameAndDescriptor() throws IOException { private void checkNameAndDescriptor() throws IOException {
assertArrayEquals(TEST_TABLE, hTableInterface.getTableName()); assertEquals(TEST_TABLE, hTableInterface.getName());
assertEquals(table.getTableDescriptor(), hTableInterface.getTableDescriptor()); assertEquals(table.getTableDescriptor(), hTableInterface.getTableDescriptor());
} }

View File

@ -162,7 +162,7 @@ public class TestOpenTableInCoprocessor {
admin.createTable(primary); admin.createTable(primary);
admin.createTable(other); 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' }); Put p = new Put(new byte[] { 'a' });
p.add(family, null, new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' });
table.put(p); table.put(p);

View File

@ -160,7 +160,7 @@ public class TestColumnRangeFilter {
public void TestColumnRangeFilterClient() throws Exception { public void TestColumnRangeFilterClient() throws Exception {
String family = "Family"; String family = "Family";
String table = "TestColumnRangeFilterClient"; 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); Bytes.toBytes(family), Integer.MAX_VALUE);
List<String> rows = generateRandomWords(10, 8); List<String> rows = generateRandomWords(10, 8);

View File

@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -89,7 +90,7 @@ public class TestFuzzyRowAndColumnRangeFilter {
public void Test() throws Exception { public void Test() throws Exception {
String cf = "f"; String cf = "f";
String table = "TestFuzzyAndColumnRangeFilterClient"; 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); Bytes.toBytes(cf), Integer.MAX_VALUE);
// 10 byte row key - (2 bytes 4 bytes 4 bytes) // 10 byte row key - (2 bytes 4 bytes 4 bytes)

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster; 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.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer; 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. // We use the regionserver file system & conf as we expect it to have the hook.
conf = targetRs.getConfiguration(); conf = targetRs.getConfiguration();
HFileSystem rfs = (HFileSystem) targetRs.getFileSystem(); 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 // 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 // with the same node will be used. We can't really stop an existing datanode, this would

View File

@ -74,7 +74,7 @@ public class TestChangingEncoding {
private HBaseAdmin admin; private HBaseAdmin admin;
private HColumnDescriptor hcd; private HColumnDescriptor hcd;
private String tableName; private TableName tableName;
private static final List<DataBlockEncoding> ENCODINGS_TO_ITERATE = private static final List<DataBlockEncoding> ENCODINGS_TO_ITERATE =
createEncodingsToIterate(); createEncodingsToIterate();
@ -89,8 +89,8 @@ public class TestChangingEncoding {
private int numBatchesWritten; private int numBatchesWritten;
private void prepareTest(String testId) throws IOException { private void prepareTest(String testId) throws IOException {
tableName = "test_table_" + testId; tableName = TableName.valueOf("test_table_" + testId);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); HTableDescriptor htd = new HTableDescriptor(tableName);
hcd = new HColumnDescriptor(CF); hcd = new HColumnDescriptor(CF);
htd.addFamily(hcd); htd.addFamily(hcd);
admin.createTable(htd); admin.createTable(htd);
@ -135,7 +135,7 @@ public class TestChangingEncoding {
+ "_col" + j); + "_col" + j);
} }
static void writeTestDataBatch(Configuration conf, String tableName, static void writeTestDataBatch(Configuration conf, TableName tableName,
int batchId) throws Exception { int batchId) throws Exception {
LOG.debug("Writing test data batch " + batchId); LOG.debug("Writing test data batch " + batchId);
Table table = new HTable(conf, tableName); Table table = new HTable(conf, tableName);
@ -153,7 +153,7 @@ public class TestChangingEncoding {
table.close(); table.close();
} }
static void verifyTestDataBatch(Configuration conf, String tableName, static void verifyTestDataBatch(Configuration conf, TableName tableName,
int batchId) throws Exception { int batchId) throws Exception {
LOG.debug("Verifying test data batch " + batchId); LOG.debug("Verifying test data batch " + batchId);
Table table = new HTable(conf, tableName); Table table = new HTable(conf, tableName);

View File

@ -91,7 +91,7 @@ public class TestTableInputFormat {
* @throws IOException * @throws IOException
*/ */
public static Table createTable(byte[] tableName) 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()); Put p = new Put("aaa".getBytes());
p.add(FAMILY, null, "value aaa".getBytes()); p.add(FAMILY, null, "value aaa".getBytes());
table.put(p); table.put(p);

View File

@ -80,19 +80,19 @@ public class TestTableMapReduce extends TestTableMapReduceBase {
jobConf = new JobConf(UTIL.getConfiguration(), TestTableMapReduce.class); jobConf = new JobConf(UTIL.getConfiguration(), TestTableMapReduce.class);
jobConf.setJobName("process column contents"); jobConf.setJobName("process column contents");
jobConf.setNumReduceTasks(1); jobConf.setNumReduceTasks(1);
TableMapReduceUtil.initTableMapJob(Bytes.toString(table.getTableName()), TableMapReduceUtil.initTableMapJob(table.getName().getNameAsString(),
Bytes.toString(INPUT_FAMILY), ProcessContentsMapper.class, Bytes.toString(INPUT_FAMILY), ProcessContentsMapper.class,
ImmutableBytesWritable.class, Put.class, jobConf); ImmutableBytesWritable.class, Put.class, jobConf);
TableMapReduceUtil.initTableReduceJob(Bytes.toString(table.getTableName()), TableMapReduceUtil.initTableReduceJob(table.getName().getNameAsString(),
IdentityTableReduce.class, jobConf); IdentityTableReduce.class, jobConf);
LOG.info("Started " + Bytes.toString(table.getTableName())); LOG.info("Started " + table.getName());
RunningJob job = JobClient.runJob(jobConf); RunningJob job = JobClient.runJob(jobConf);
assertTrue(job.isSuccessful()); assertTrue(job.isSuccessful());
LOG.info("After map/reduce completion"); LOG.info("After map/reduce completion");
// verify map-reduce results // verify map-reduce results
verify(Bytes.toString(table.getTableName())); verify(table.getName());
} finally { } finally {
if (jobConf != null) { if (jobConf != null) {
FileUtil.fullyDelete(new File(jobConf.get("hadoop.tmp.dir"))); FileUtil.fullyDelete(new File(jobConf.get("hadoop.tmp.dir")));

View File

@ -33,6 +33,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -88,7 +89,7 @@ public class TestTableMapReduceUtil {
@BeforeClass @BeforeClass
public static void beforeClass() throws Exception { public static void beforeClass() throws Exception {
UTIL.startMiniCluster(); UTIL.startMiniCluster();
presidentsTable = createAndFillTable(Bytes.toBytes(TABLE_NAME)); presidentsTable = createAndFillTable(TableName.valueOf(TABLE_NAME));
UTIL.startMiniMapReduceCluster(); UTIL.startMiniMapReduceCluster();
} }
@ -105,7 +106,7 @@ public class TestTableMapReduceUtil {
LOG.info("before done"); 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); Table table = UTIL.createTable(tableName, COLUMN_FAMILY);
createPutCommand(table); createPutCommand(table);
return table; return table;

View File

@ -30,6 +30,7 @@ import java.io.PrintStream;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
@ -80,8 +81,8 @@ public class TestCopyTable {
*/ */
@Test @Test
public void testCopyTable() throws Exception { public void testCopyTable() throws Exception {
final byte[] TABLENAME1 = Bytes.toBytes("testCopyTable1"); final TableName TABLENAME1 = TableName.valueOf("testCopyTable1");
final byte[] TABLENAME2 = Bytes.toBytes("testCopyTable2"); final TableName TABLENAME2 = TableName.valueOf("testCopyTable2");
final byte[] FAMILY = Bytes.toBytes("family"); final byte[] FAMILY = Bytes.toBytes("family");
final byte[] COLUMN1 = Bytes.toBytes("c1"); final byte[] COLUMN1 = Bytes.toBytes("c1");
@ -99,8 +100,8 @@ public class TestCopyTable {
assertEquals( assertEquals(
0, 0,
copy.run(new String[] { "--new.name=" + Bytes.toString(TABLENAME2), copy.run(new String[] { "--new.name=" + TABLENAME2.getNameAsString(),
Bytes.toString(TABLENAME1) })); TABLENAME1.getNameAsString() }));
// verify the data was copied into table 2 // verify the data was copied into table 2
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
@ -118,8 +119,8 @@ public class TestCopyTable {
@Test @Test
public void testStartStopRow() throws Exception { public void testStartStopRow() throws Exception {
final byte[] TABLENAME1 = Bytes.toBytes("testStartStopRow1"); final TableName TABLENAME1 = TableName.valueOf("testStartStopRow1");
final byte[] TABLENAME2 = Bytes.toBytes("testStartStopRow2"); final TableName TABLENAME2 = TableName.valueOf("testStartStopRow2");
final byte[] FAMILY = Bytes.toBytes("family"); final byte[] FAMILY = Bytes.toBytes("family");
final byte[] COLUMN1 = Bytes.toBytes("c1"); final byte[] COLUMN1 = Bytes.toBytes("c1");
final byte[] ROW0 = Bytes.toBytes("row0"); final byte[] ROW0 = Bytes.toBytes("row0");
@ -143,8 +144,8 @@ public class TestCopyTable {
CopyTable copy = new CopyTable(TEST_UTIL.getConfiguration()); CopyTable copy = new CopyTable(TEST_UTIL.getConfiguration());
assertEquals( assertEquals(
0, 0,
copy.run(new String[] { "--new.name=" + Bytes.toString(TABLENAME2), "--startrow=row1", copy.run(new String[] { "--new.name=" + TABLENAME2, "--startrow=row1",
"--stoprow=row2", Bytes.toString(TABLENAME1) })); "--stoprow=row2", TABLENAME1.getNameAsString() }));
// verify the data was copied into table 2 // verify the data was copied into table 2
// row1 exist, row0, row2 do not exist // row1 exist, row0, row2 do not exist

View File

@ -17,6 +17,7 @@ package org.apache.hadoop.hbase.mapreduce;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@ -52,8 +53,8 @@ public class TestHRegionPartitioner {
byte[][] families = { Bytes.toBytes("familyA"), Bytes.toBytes("familyB") }; byte[][] families = { Bytes.toBytes("familyA"), Bytes.toBytes("familyB") };
UTIL.createTable(Bytes.toBytes("out_table"), families, 1, Bytes.toBytes("aa"), UTIL.createTable(TableName.valueOf("out_table"), families, 1,
Bytes.toBytes("cc"), 3); Bytes.toBytes("aa"), Bytes.toBytes("cc"), 3);
HRegionPartitioner<Long, Long> partitioner = new HRegionPartitioner<Long, Long>(); HRegionPartitioner<Long, Long> partitioner = new HRegionPartitioner<Long, Long>();
Configuration configuration = UTIL.getConfiguration(); Configuration configuration = UTIL.getConfiguration();

View File

@ -161,7 +161,7 @@ public class TestImportExport {
@Test @Test
public void testSimpleCase() throws Exception { public void testSimpleCase() throws Exception {
String EXPORT_TABLE = "exportSimpleCase"; 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); Put p = new Put(ROW1);
p.add(FAMILYA, QUAL, now, QUAL); p.add(FAMILYA, QUAL, now, QUAL);
p.add(FAMILYA, QUAL, now+1, QUAL); p.add(FAMILYA, QUAL, now+1, QUAL);
@ -181,7 +181,7 @@ public class TestImportExport {
assertTrue(runExport(args)); assertTrue(runExport(args));
String IMPORT_TABLE = "importTableSimpleCase"; 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[] { args = new String[] {
"-D" + Import.CF_RENAME_PROP + "="+FAMILYA_STRING+":"+FAMILYB_STRING, "-D" + Import.CF_RENAME_PROP + "="+FAMILYA_STRING+":"+FAMILYB_STRING,
IMPORT_TABLE, IMPORT_TABLE,
@ -224,7 +224,7 @@ public class TestImportExport {
fs.copyFromLocalFile(importPath, new Path(FQ_OUTPUT_DIR + Path.SEPARATOR fs.copyFromLocalFile(importPath, new Path(FQ_OUTPUT_DIR + Path.SEPARATOR
+ "exportedTableIn94Format")); + "exportedTableIn94Format"));
String IMPORT_TABLE = "importTableExportedFrom94"; 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[] { String[] args = new String[] {
"-Dhbase.import.version=0.94" , "-Dhbase.import.version=0.94" ,
IMPORT_TABLE, FQ_OUTPUT_DIR IMPORT_TABLE, FQ_OUTPUT_DIR
@ -254,7 +254,7 @@ public class TestImportExport {
.setMaxVersions(1) .setMaxVersions(1)
); );
UTIL.getHBaseAdmin().createTable(desc); 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); Put p = new Put(ROW1);
p.add(FAMILYA, QUAL, now, QUAL); p.add(FAMILYA, QUAL, now, QUAL);
@ -285,7 +285,7 @@ public class TestImportExport {
.setKeepDeletedCells(true) .setKeepDeletedCells(true)
); );
UTIL.getHBaseAdmin().createTable(desc); 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); Put p = new Put(ROW1);
p.add(FAMILYA, QUAL, now, QUAL); p.add(FAMILYA, QUAL, now, QUAL);
@ -317,7 +317,7 @@ public class TestImportExport {
); );
UTIL.getHBaseAdmin().createTable(desc); UTIL.getHBaseAdmin().createTable(desc);
t.close(); t.close();
t = new HTable(UTIL.getConfiguration(), IMPORT_TABLE); t = new HTable(UTIL.getConfiguration(), desc.getTableName());
args = new String[] { args = new String[] {
IMPORT_TABLE, IMPORT_TABLE,
FQ_OUTPUT_DIR FQ_OUTPUT_DIR
@ -351,7 +351,7 @@ public class TestImportExport {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE)); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE));
desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5));
UTIL.getHBaseAdmin().createTable(desc); 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); Put p = new Put(ROW1);
p.add(FAMILYA, QUAL, now, QUAL); p.add(FAMILYA, QUAL, now, QUAL);
@ -378,7 +378,7 @@ public class TestImportExport {
desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5));
UTIL.getHBaseAdmin().createTable(desc); 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(), 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, "-D" + Import.FILTER_ARGS_CONF_KEY + "=" + Bytes.toString(ROW1), IMPORT_TABLE, FQ_OUTPUT_DIR,
"1000" }; "1000" };
@ -544,7 +544,7 @@ public class TestImportExport {
public void testDurability() throws IOException, InterruptedException, ClassNotFoundException { public void testDurability() throws IOException, InterruptedException, ClassNotFoundException {
// Create an export table. // Create an export table.
String exportTableName = "exporttestDurability"; String exportTableName = "exporttestDurability";
Table exportTable = UTIL.createTable(Bytes.toBytes(exportTableName), FAMILYA, 3); Table exportTable = UTIL.createTable(TableName.valueOf(exportTableName), FAMILYA, 3);
// Insert some data // Insert some data
Put put = new Put(ROW1); Put put = new Put(ROW1);
@ -565,7 +565,7 @@ public class TestImportExport {
// Create the table for import // Create the table for import
String importTableName = "importTestDurability1"; 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 // Register the hlog listener for the import table
TableWALActionListener walListener = new TableWALActionListener(importTableName); TableWALActionListener walListener = new TableWALActionListener(importTableName);
@ -584,7 +584,7 @@ public class TestImportExport {
// Run the import with the default durability option // Run the import with the default durability option
importTableName = "importTestDurability2"; importTableName = "importTestDurability2";
importTable = UTIL.createTable(Bytes.toBytes(importTableName), FAMILYA, 3); importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3);
hLog.unregisterWALActionsListener(walListener); hLog.unregisterWALActionsListener(walListener);
walListener = new TableWALActionListener(importTableName); walListener = new TableWALActionListener(importTableName);
hLog.registerWALActionsListener(walListener); hLog.registerWALActionsListener(walListener);

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
@ -120,7 +121,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY", "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001btest=>myvalue\n"; 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); doMROnTableTest(util, FAMILY, data, args, 1, true);
util.deleteTable(tableName); util.deleteTable(tableName);
} }
@ -136,7 +137,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY", "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001btest1=>myvalue\n"; 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); doMROnTableTest(util, FAMILY, data, args, 1, false);
util.deleteTable(tableName); util.deleteTable(tableName);
} }
@ -177,7 +178,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
LOG.debug("Running ImportTsv with arguments: " + argv); LOG.debug("Running ImportTsv with arguments: " + argv);
assertEquals(0, ToolRunner.run(conf, tool, argv.toArray(args))); 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)) { if (conf.getBoolean(DELETE_AFTER_LOAD_CONF, true)) {
LOG.debug("Deleting test subdirectory"); LOG.debug("Deleting test subdirectory");
@ -191,7 +192,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
* *
* @param dataAvailable * @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 { int valueMultiplier, boolean dataAvailable) throws IOException {
LOG.debug("Validating table."); LOG.debug("Validating table.");

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
@ -161,20 +162,20 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY", "-D" + ImportTsv.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 };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n"; String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n";
util.createTable(tableName, FAMILY); util.createTable(TableName.valueOf(tableName), FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1); doMROnTableTest(util, FAMILY, data, args, 1);
util.deleteTable(tableName); util.deleteTable(tableName);
} }
@Test @Test
public void testMROnTableWithDeletes() throws Exception { public void testMROnTableWithDeletes() throws Exception {
String tableName = "test-" + UUID.randomUUID(); TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
// Prepare the arguments required for the test. // Prepare the arguments required for the test.
String[] args = new String[] { String[] args = new String[] {
"-D" + ImportTsv.MAPPER_CONF_KEY + "=org.apache.hadoop.hbase.mapreduce.TsvImporterMapper", "-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.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"; String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n";
util.createTable(tableName, FAMILY); util.createTable(tableName, FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1); doMROnTableTest(util, FAMILY, data, args, 1);
@ -182,7 +183,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
util.deleteTable(tableName); util.deleteTable(tableName);
} }
private void issueDeleteAndVerifyData(String tableName) throws IOException { private void issueDeleteAndVerifyData(TableName tableName) throws IOException {
LOG.debug("Validating table after delete."); LOG.debug("Validating table after delete.");
Table table = new HTable(conf, tableName); Table table = new HTable(conf, tableName);
boolean verified = false; boolean verified = false;
@ -229,7 +230,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
+ "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY", + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName }; "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n"; String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n";
util.createTable(tableName, FAMILY); util.createTable(TableName.valueOf(tableName), FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1); doMROnTableTest(util, FAMILY, data, args, 1);
util.deleteTable(tableName); util.deleteTable(tableName);
} }
@ -266,7 +267,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY", "-D" + ImportTsv.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 };
String data = "KEY\u001bVALUE4\u001bVALUE8\u001bsecret&private\n"; String data = "KEY\u001bVALUE4\u001bVALUE8\u001bsecret&private\n";
util.createTable(tableName, FAMILY); util.createTable(TableName.valueOf(tableName), FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1); doMROnTableTest(util, FAMILY, data, args, 1);
util.deleteTable(tableName); util.deleteTable(tableName);
} }
@ -283,12 +284,13 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
*/ */
protected static Tool doMROnTableTest(HBaseTestingUtility util, String family, String data, protected static Tool doMROnTableTest(HBaseTestingUtility util, String family, String data,
String[] args, int valueMultiplier) throws Exception { 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()); Configuration conf = new Configuration(util.getConfiguration());
// populate input file // populate input file
FileSystem fs = FileSystem.get(conf); 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); FSDataOutputStream op = fs.create(inputPath, true);
if (data == null) { if (data == null) {
data = "KEY\u001bVALUE1\u001bVALUE2\n"; data = "KEY\u001bVALUE1\u001bVALUE2\n";
@ -330,7 +332,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
if (conf.getBoolean(DELETE_AFTER_LOAD_CONF, true)) { if (conf.getBoolean(DELETE_AFTER_LOAD_CONF, true)) {
LOG.debug("Deleting test subdirectory"); LOG.debug("Deleting test subdirectory");
util.cleanupDataTestDirOnTestFS(table); util.cleanupDataTestDirOnTestFS(table.getNameAsString());
} }
return tool; return tool;
} }
@ -364,7 +366,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
/** /**
* Confirm ImportTsv via data in online table. * 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 { int valueMultiplier) throws IOException {
LOG.debug("Validating table."); LOG.debug("Validating table.");

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -111,7 +112,7 @@ public class TestImportTsv implements Configurable {
table table
}; };
util.createTable(table, FAMILY); util.createTable(TableName.valueOf(table), FAMILY);
doMROnTableTest(util, FAMILY, null, args, 1); doMROnTableTest(util, FAMILY, null, args, 1);
util.deleteTable(table); util.deleteTable(table);
} }
@ -129,7 +130,7 @@ public class TestImportTsv implements Configurable {
}; };
String data = "KEY,1234,VALUE1,VALUE2\n"; String data = "KEY,1234,VALUE1,VALUE2\n";
util.createTable(table, FAMILY); util.createTable(TableName.valueOf(table), FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1); doMROnTableTest(util, FAMILY, data, args, 1);
util.deleteTable(table); util.deleteTable(table);
} }
@ -146,7 +147,7 @@ public class TestImportTsv implements Configurable {
table table
}; };
util.createTable(table, FAMILY); util.createTable(TableName.valueOf(table), FAMILY);
doMROnTableTest(util, FAMILY, null, args, 3); doMROnTableTest(util, FAMILY, null, args, 3);
util.deleteTable(table); util.deleteTable(table);
} }
@ -181,7 +182,7 @@ public class TestImportTsv implements Configurable {
table table
}; };
util.createTable(table, FAMILY); util.createTable(TableName.valueOf(table), FAMILY);
doMROnTableTest(util, FAMILY, null, args, 3); doMROnTableTest(util, FAMILY, null, args, 3);
util.deleteTable(table); util.deleteTable(table);
} }
@ -288,7 +289,7 @@ public class TestImportTsv implements Configurable {
if (createdHFiles) if (createdHFiles)
validateHFiles(fs, outputPath, family); validateHFiles(fs, outputPath, family);
else else
validateTable(conf, table, family, valueMultiplier); validateTable(conf, TableName.valueOf(table), family, valueMultiplier);
if (conf.getBoolean(DELETE_AFTER_LOAD_CONF, true)) { if (conf.getBoolean(DELETE_AFTER_LOAD_CONF, true)) {
LOG.debug("Deleting test subdirectory"); LOG.debug("Deleting test subdirectory");
@ -300,7 +301,7 @@ public class TestImportTsv implements Configurable {
/** /**
* Confirm ImportTsv via data in online table. * 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 { String family, int valueMultiplier) throws IOException {
LOG.debug("Validating table."); LOG.debug("Validating table.");

View File

@ -121,10 +121,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
* Creates a table with given table name and specified number of column * Creates a table with given table name and specified number of column
* families if the table does not already exist. * 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 { try {
LOG.info("Creating table " + table); LOG.info("Creating table " + table);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); HTableDescriptor htd = new HTableDescriptor(table);
for (int i = 0; i < cfs; i++) { for (int i = 0; i < cfs; i++) {
htd.addFamily(new HColumnDescriptor(family(i))); htd.addFamily(new HColumnDescriptor(family(i)));
} }
@ -142,11 +142,11 @@ public class TestLoadIncrementalHFilesSplitRecovery {
* @param cfs * @param cfs
* @param SPLIT_KEYS * @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 { throws IOException {
try { try {
LOG.info("Creating table " + table); LOG.info("Creating table " + table);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); HTableDescriptor htd = new HTableDescriptor(table);
for (int i = 0; i < cfs; i++) { for (int i = 0; i < cfs; i++) {
htd.addFamily(new HColumnDescriptor(family(i))); htd.addFamily(new HColumnDescriptor(family(i)));
} }
@ -157,9 +157,9 @@ public class TestLoadIncrementalHFilesSplitRecovery {
} }
} }
private Path buildBulkFiles(String table, int value) throws Exception { private Path buildBulkFiles(TableName table, int value) throws Exception {
Path dir = util.getDataTestDirOnTestFS(table); Path dir = util.getDataTestDirOnTestFS(table.getNameAsString());
Path bulk1 = new Path(dir, table+value); Path bulk1 = new Path(dir, table.getNameAsString() + value);
FileSystem fs = util.getTestFileSystem(); FileSystem fs = util.getTestFileSystem();
buildHFiles(fs, bulk1, value); buildHFiles(fs, bulk1, value);
return bulk1; return bulk1;
@ -168,26 +168,25 @@ public class TestLoadIncrementalHFilesSplitRecovery {
/** /**
* Populate table with known values. * 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 // create HFiles for different column families
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()); LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
Path bulk1 = buildBulkFiles(table, value); 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); lih.doBulkLoad(bulk1, t);
} }
/** /**
* Split the known table in half. (this is hard coded for this test suite) * 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 { try {
// need to call regions server to by synchronous but isn't visible. // need to call regions server to by synchronous but isn't visible.
HRegionServer hrs = util.getRSForFirstRegionInTable(Bytes HRegionServer hrs = util.getRSForFirstRegionInTable(table);
.toBytes(table));
for (HRegionInfo hri : for (HRegionInfo hri :
ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) { 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 // splitRegion doesn't work if startkey/endkey are null
ProtobufUtil.split(hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2)); // hard code split ProtobufUtil.split(hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2)); // hard code split
} }
@ -199,7 +198,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
regions = 0; regions = 0;
for (HRegionInfo hri : for (HRegionInfo hri :
ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) { ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) { if (hri.getTable().equals(table)) {
regions++; regions++;
} }
} }
@ -232,10 +231,11 @@ public class TestLoadIncrementalHFilesSplitRecovery {
* expected number of rows. * expected number of rows.
* @throws IOException * @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; Table t = null;
try { try {
assertEquals(util.getHBaseAdmin().listTables(table).length, 1); assertEquals(
util.getHBaseAdmin().listTables(table.getNameAsString()).length, 1);
t = new HTable(util.getConfiguration(), table); t = new HTable(util.getConfiguration(), table);
Scan s = new Scan(); Scan s = new Scan();
ResultScanner sr = t.getScanner(s); ResultScanner sr = t.getScanner(s);
@ -262,7 +262,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
*/ */
@Test(expected=IOException.class) @Test(expected=IOException.class)
public void testBulkLoadPhaseFailure() throws Exception { public void testBulkLoadPhaseFailure() throws Exception {
String table = "bulkLoadPhaseFailure"; TableName table = TableName.valueOf("bulkLoadPhaseFailure");
setupTable(table, 10); setupTable(table, 10);
final AtomicInteger attmptedCalls = new AtomicInteger(); final AtomicInteger attmptedCalls = new AtomicInteger();
@ -293,7 +293,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
// create HFiles for different column families // create HFiles for different column families
Path dir = buildBulkFiles(table, 1); 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); lih.doBulkLoad(dir, t);
} finally { } finally {
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
@ -334,7 +334,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
*/ */
@Test @Test
public void testSplitWhileBulkLoadPhase() throws Exception { public void testSplitWhileBulkLoadPhase() throws Exception {
final String table = "splitWhileBulkloadPhase"; final TableName table = TableName.valueOf("splitWhileBulkloadPhase");
setupTable(table, 10); setupTable(table, 10);
populateTable(table,1); populateTable(table,1);
assertExpectedTable(table, ROWCOUNT, 1); assertExpectedTable(table, ROWCOUNT, 1);
@ -359,7 +359,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
}; };
// create HFiles for different column families // 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); Path bulk = buildBulkFiles(table, 2);
lih2.doBulkLoad(bulk, t); lih2.doBulkLoad(bulk, t);
@ -376,7 +376,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
*/ */
@Test @Test
public void testGroupOrSplitPresplit() throws Exception { public void testGroupOrSplitPresplit() throws Exception {
final String table = "groupOrSplitPresplit"; final TableName table = TableName.valueOf("groupOrSplitPresplit");
setupTable(table, 10); setupTable(table, 10);
populateTable(table, 1); populateTable(table, 1);
assertExpectedTable(table, ROWCOUNT, 1); assertExpectedTable(table, ROWCOUNT, 1);
@ -399,7 +399,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
// create HFiles for different column families // create HFiles for different column families
Path bulk = buildBulkFiles(table, 2); 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); lih.doBulkLoad(bulk, ht);
assertExpectedTable(table, ROWCOUNT, 2); assertExpectedTable(table, ROWCOUNT, 2);
@ -412,7 +412,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
*/ */
@Test(expected = IOException.class) @Test(expected = IOException.class)
public void testGroupOrSplitFailure() throws Exception { public void testGroupOrSplitFailure() throws Exception {
String table = "groupOrSplitFailure"; TableName table = TableName.valueOf("groupOrSplitFailure");
setupTable(table, 10); setupTable(table, 10);
LoadIncrementalHFiles lih = new LoadIncrementalHFiles( LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
@ -434,7 +434,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
// create HFiles for different column families // create HFiles for different column families
Path dir = buildBulkFiles(table,1); 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); lih.doBulkLoad(dir, t);
fail("doBulkLoad should have thrown an exception"); fail("doBulkLoad should have thrown an exception");
@ -442,9 +442,9 @@ public class TestLoadIncrementalHFilesSplitRecovery {
@Test @Test
public void testGroupOrSplitWhenRegionHoleExistsInMeta() throws Exception { public void testGroupOrSplitWhenRegionHoleExistsInMeta() throws Exception {
String tableName = "testGroupOrSplitWhenRegionHoleExistsInMeta"; TableName tableName = TableName.valueOf("testGroupOrSplitWhenRegionHoleExistsInMeta");
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000100") }; 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); setupTableWithSplitkeys(tableName, 10, SPLIT_KEYS);
Path dir = buildBulkFiles(tableName, 2); Path dir = buildBulkFiles(tableName, 2);
@ -479,7 +479,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
// Mess it up by leaving a hole in the hbase:meta // Mess it up by leaving a hole in the hbase:meta
HConnection hConnection = HConnectionManager.getConnection(util.getConfiguration()); HConnection hConnection = HConnectionManager.getConnection(util.getConfiguration());
List<HRegionInfo> regionInfos = MetaTableAccessor.getTableRegions( List<HRegionInfo> regionInfos = MetaTableAccessor.getTableRegions(
hConnection, TableName.valueOf(tableName)); hConnection, tableName);
for (HRegionInfo regionInfo : regionInfos) { for (HRegionInfo regionInfo : regionInfos) {
if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
MetaTableAccessor.deleteRegion(hConnection, regionInfo); MetaTableAccessor.deleteRegion(hConnection, regionInfo);

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility; 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
@ -76,8 +77,7 @@ public class TestMultiTableInputFormat {
// create and fill table // create and fill table
for (int i = 0; i < 3; i++) { for (int i = 0; i < 3; i++) {
HTable table = HTable table =
TEST_UTIL.createTable(Bytes.toBytes(TABLE_NAME + String.valueOf(i)), TEST_UTIL.createTable(TableName.valueOf(TABLE_NAME + String.valueOf(i)), INPUT_FAMILY);
INPUT_FAMILY);
TEST_UTIL.createMultiRegions(TEST_UTIL.getConfiguration(), table, INPUT_FAMILY, 4); TEST_UTIL.createMultiRegions(TEST_UTIL.getConfiguration(), table, INPUT_FAMILY, 4);
TEST_UTIL.loadTable(table, INPUT_FAMILY, false); TEST_UTIL.loadTable(table, INPUT_FAMILY, false);
} }

View File

@ -59,7 +59,7 @@ public class TestMultithreadedTableMapper {
private static final Log LOG = LogFactory.getLog(TestMultithreadedTableMapper.class); private static final Log LOG = LogFactory.getLog(TestMultithreadedTableMapper.class);
private static final HBaseTestingUtility UTIL = private static final HBaseTestingUtility UTIL =
new HBaseTestingUtility(); 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[] INPUT_FAMILY = Bytes.toBytes("contents");
static final byte[] OUTPUT_FAMILY = Bytes.toBytes("text"); static final byte[] OUTPUT_FAMILY = Bytes.toBytes("text");
static final int NUMBER_OF_THREADS = 10; static final int NUMBER_OF_THREADS = 10;
@ -139,7 +139,7 @@ public class TestMultithreadedTableMapper {
Scan scan = new Scan(); Scan scan = new Scan();
scan.addFamily(INPUT_FAMILY); scan.addFamily(INPUT_FAMILY);
TableMapReduceUtil.initTableMapperJob( TableMapReduceUtil.initTableMapperJob(
Bytes.toString(table.getTableName()), scan, table.getTableName(), scan,
MultithreadedTableMapper.class, ImmutableBytesWritable.class, MultithreadedTableMapper.class, ImmutableBytesWritable.class,
Put.class, job); Put.class, job);
MultithreadedTableMapper.setMapperClass(job, ProcessContentsMapper.class); MultithreadedTableMapper.setMapperClass(job, ProcessContentsMapper.class);
@ -148,11 +148,11 @@ public class TestMultithreadedTableMapper {
Bytes.toString(table.getTableName()), Bytes.toString(table.getTableName()),
IdentityTableReducer.class, job); IdentityTableReducer.class, job);
FileOutputFormat.setOutputPath(job, new Path("test")); FileOutputFormat.setOutputPath(job, new Path("test"));
LOG.info("Started " + Bytes.toString(table.getTableName())); LOG.info("Started " + table.getTableName());
assertTrue(job.waitForCompletion(true)); assertTrue(job.waitForCompletion(true));
LOG.info("After map/reduce completion"); LOG.info("After map/reduce completion");
// verify map-reduce results // verify map-reduce results
verify(Bytes.toString(table.getTableName())); verify(table.getName());
} finally { } finally {
table.close(); table.close();
if (job != null) { if (job != null) {
@ -162,7 +162,7 @@ public class TestMultithreadedTableMapper {
} }
} }
private void verify(String tableName) throws IOException { private void verify(TableName tableName) throws IOException {
Table table = new HTable(new Configuration(UTIL.getConfiguration()), tableName); Table table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
boolean verified = false; boolean verified = false;
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);

View File

@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -68,8 +69,7 @@ public class TestRowCounter {
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(); TEST_UTIL.startMiniCluster();
TEST_UTIL.startMiniMapReduceCluster(); TEST_UTIL.startMiniMapReduceCluster();
Table table = TEST_UTIL.createTable(Bytes.toBytes(TABLE_NAME), Table table = TEST_UTIL.createTable(TableName.valueOf(TABLE_NAME), Bytes.toBytes(COL_FAM));
Bytes.toBytes(COL_FAM));
writeRows(table); writeRows(table);
table.close(); table.close();
} }

View File

@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility; 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
@ -77,7 +78,7 @@ public abstract class TestTableInputFormatScanBase {
// start mini hbase cluster // start mini hbase cluster
TEST_UTIL.startMiniCluster(3); TEST_UTIL.startMiniCluster(3);
// create and fill table // 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.createMultiRegions(table, INPUT_FAMILY);
TEST_UTIL.loadTable(table, INPUT_FAMILY, false); TEST_UTIL.loadTable(table, INPUT_FAMILY, false);
// start MR cluster // start MR cluster

View File

@ -111,7 +111,7 @@ public class TestTableMapReduce extends TestTableMapReduceBase {
LOG.info("After map/reduce completion"); LOG.info("After map/reduce completion");
// verify map-reduce results // verify map-reduce results
verify(Bytes.toString(table.getTableName())); verify(table.getName());
} catch (InterruptedException e) { } catch (InterruptedException e) {
throw new IOException(e); throw new IOException(e);
} catch (ClassNotFoundException e) { } catch (ClassNotFoundException e) {

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; 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.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -132,7 +133,7 @@ public abstract class TestTableMapReduceBase {
return outval; return outval;
} }
protected void verify(String tableName) throws IOException { protected void verify(TableName tableName) throws IOException {
Table table = new HTable(UTIL.getConfiguration(), tableName); Table table = new HTable(UTIL.getConfiguration(), tableName);
boolean verified = false; boolean verified = false;
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);

View File

@ -87,8 +87,8 @@ public class TestWALPlayer {
*/ */
@Test @Test
public void testWALPlayer() throws Exception { public void testWALPlayer() throws Exception {
final byte[] TABLENAME1 = Bytes.toBytes("testWALPlayer1"); final TableName TABLENAME1 = TableName.valueOf("testWALPlayer1");
final byte[] TABLENAME2 = Bytes.toBytes("testWALPlayer2"); final TableName TABLENAME2 = TableName.valueOf("testWALPlayer2");
final byte[] FAMILY = Bytes.toBytes("family"); final byte[] FAMILY = Bytes.toBytes("family");
final byte[] COLUMN1 = Bytes.toBytes("c1"); final byte[] COLUMN1 = Bytes.toBytes("c1");
final byte[] COLUMN2 = Bytes.toBytes("c2"); final byte[] COLUMN2 = Bytes.toBytes("c2");
@ -118,8 +118,8 @@ public class TestWALPlayer {
configuration.set(optionName, "1000"); configuration.set(optionName, "1000");
player.setupTime(configuration, optionName); player.setupTime(configuration, optionName);
assertEquals(1000,configuration.getLong(optionName,0)); assertEquals(1000,configuration.getLong(optionName,0));
assertEquals(0, player.run(new String[] { walInputDir, Bytes.toString(TABLENAME1), assertEquals(0, player.run(new String[] {walInputDir, TABLENAME1.getNameAsString(),
Bytes.toString(TABLENAME2) })); TABLENAME2.getNameAsString() }));
// verify the WAL was player into table 2 // verify the WAL was player into table 2

View File

@ -218,7 +218,7 @@ public class TestMasterFailover {
assertTrue(master.isInitialized()); assertTrue(master.isInitialized());
// Create a table with a region online // Create a table with a region online
RegionLocator onlineTable = TEST_UTIL.createTable("onlineTable", "family"); RegionLocator onlineTable = TEST_UTIL.createTable(TableName.valueOf("onlineTable"), "family");
// Create a table in META, so it has a region offline // Create a table in META, so it has a region offline
HTableDescriptor offlineTable = new HTableDescriptor( HTableDescriptor offlineTable = new HTableDescriptor(

View File

@ -52,7 +52,7 @@ import org.junit.experimental.categories.Category;
public class TestMasterTransitions { public class TestMasterTransitions {
private static final Log LOG = LogFactory.getLog(TestMasterTransitions.class); private static final Log LOG = LogFactory.getLog(TestMasterTransitions.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); 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"), private static final byte [][] FAMILIES = new byte [][] {Bytes.toBytes("a"),
Bytes.toBytes("b"), Bytes.toBytes("c")}; Bytes.toBytes("b"), Bytes.toBytes("c")};
@ -64,11 +64,10 @@ public class TestMasterTransitions {
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true); TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
TEST_UTIL.startMiniCluster(2); TEST_UTIL.startMiniCluster(2);
// Create a table of three families. This will assign a region. // 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); HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
int countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily()); int countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily());
TEST_UTIL.waitUntilAllRegionsAssigned(tableName); TEST_UTIL.waitUntilAllRegionsAssigned(TABLENAME);
addToEachStartKey(countOfRegions); addToEachStartKey(countOfRegions);
t.close(); t.close();
} }
@ -491,12 +490,12 @@ public class TestMasterTransitions {
for (Result r = null; (r = s.next()) != null;) { for (Result r = null; (r = s.next()) != null;) {
HRegionInfo hri = HRegionInfo.getHRegionInfo(r); HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) break; if (hri == null) break;
if (!hri.getTable().getNameAsString().equals(TABLENAME)) { if (!hri.getTable().equals(TABLENAME)) {
continue; continue;
} }
// If start key, add 'aaa'. // If start key, add 'aaa'.
if(!hri.getTable().getNameAsString().equals(TABLENAME)) { if(!hri.getTable().equals(TABLENAME)) {
continue; continue;
} }
byte [] row = getStartKey(hri); byte [] row = getStartKey(hri);
@ -529,4 +528,4 @@ public class TestMasterTransitions {
private static byte [] getTestQualifier() { private static byte [] getTestQualifier() {
return getTestFamily(); return getTestFamily();
} }
} }

View File

@ -50,10 +50,10 @@ public class TestRestartCluster {
private static final Log LOG = LogFactory.getLog(TestRestartCluster.class); private static final Log LOG = LogFactory.getLog(TestRestartCluster.class);
private HBaseTestingUtility UTIL = new HBaseTestingUtility(); private HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final byte [][] TABLES = { private static final TableName[] TABLES = {
Bytes.toBytes("restartTableOne"), TableName.valueOf("restartTableOne"),
Bytes.toBytes("restartTableTwo"), TableName.valueOf("restartTableTwo"),
Bytes.toBytes("restartTableThree") TableName.valueOf("restartTableThree")
}; };
private static final byte [] FAMILY = Bytes.toBytes("family"); private static final byte [] FAMILY = Bytes.toBytes("family");
@ -68,11 +68,11 @@ public class TestRestartCluster {
Threads.sleep(1); Threads.sleep(1);
} }
LOG.info("\n\nCreating tables"); LOG.info("\n\nCreating tables");
for(byte [] TABLE : TABLES) { for(TableName TABLE : TABLES) {
UTIL.createTable(TABLE, FAMILY); UTIL.createTable(TABLE, FAMILY);
} }
for(byte [] TABLE : TABLES) { for(TableName TABLE : TABLES) {
UTIL.waitTableEnabled(TABLE); UTIL.waitTableEnabled(TABLE.getName());
} }
List<HRegionInfo> allRegions = List<HRegionInfo> allRegions =
@ -94,14 +94,14 @@ public class TestRestartCluster {
allRegions = MetaScanner.listAllRegions(new Configuration(UTIL.getConfiguration()), true); allRegions = MetaScanner.listAllRegions(new Configuration(UTIL.getConfiguration()), true);
assertEquals(4, allRegions.size()); assertEquals(4, allRegions.size());
LOG.info("\n\nWaiting for tables to be available"); LOG.info("\n\nWaiting for tables to be available");
for(byte [] TABLE: TABLES) { for(TableName TABLE: TABLES) {
try { try {
UTIL.createTable(TABLE, FAMILY); UTIL.createTable(TABLE, FAMILY);
assertTrue("Able to create table that should already exist", false); assertTrue("Able to create table that should already exist", false);
} catch(TableExistsException tee) { } catch(TableExistsException tee) {
LOG.info("Table already exists as expected"); LOG.info("Table already exists as expected");
} }
UTIL.waitTableAvailable(TABLE); UTIL.waitTableAvailable(TABLE.getName());
} }
} }
@ -118,11 +118,11 @@ public class TestRestartCluster {
UTIL.getMiniHBaseCluster().getMaster(). UTIL.getMiniHBaseCluster().getMaster().
getMasterRpcServices().synchronousBalanceSwitch(false); getMasterRpcServices().synchronousBalanceSwitch(false);
LOG.info("\n\nCreating tables"); LOG.info("\n\nCreating tables");
for(byte [] TABLE : TABLES) { for(TableName TABLE : TABLES) {
UTIL.createTable(TABLE, FAMILY); UTIL.createTable(TABLE, FAMILY);
} }
for(byte [] TABLE : TABLES) { for(TableName TABLE : TABLES) {
UTIL.waitTableEnabled(TABLE); UTIL.waitTableEnabled(TABLE.getName());
} }
HMaster master = UTIL.getMiniHBaseCluster().getMaster(); HMaster master = UTIL.getMiniHBaseCluster().getMaster();

View File

@ -81,7 +81,7 @@ public class TestCompactionState {
@Test @Test
public void testInvalidColumnFamily() throws IOException, InterruptedException { public void testInvalidColumnFamily() throws IOException, InterruptedException {
byte [] table = Bytes.toBytes("testInvalidColumnFamily"); TableName table = TableName.valueOf("testInvalidColumnFamily");
byte [] family = Bytes.toBytes("family"); byte [] family = Bytes.toBytes("family");
byte [] fakecf = Bytes.toBytes("fakecf"); byte [] fakecf = Bytes.toBytes("fakecf");
boolean caughtMinorCompact = false; boolean caughtMinorCompact = false;
@ -124,8 +124,7 @@ public class TestCompactionState {
final CompactionState expectedState, boolean singleFamily) final CompactionState expectedState, boolean singleFamily)
throws IOException, InterruptedException { throws IOException, InterruptedException {
// Create a table with regions // Create a table with regions
TableName table = TableName table = TableName.valueOf(tableName);
TableName.valueOf(tableName);
byte [] family = Bytes.toBytes("family"); byte [] family = Bytes.toBytes("family");
byte [][] families = byte [][] families =
{family, Bytes.add(family, Bytes.toBytes("2")), Bytes.add(family, Bytes.toBytes("3"))}; {family, Bytes.add(family, Bytes.toBytes("2")), Bytes.add(family, Bytes.toBytes("3"))};

View File

@ -52,7 +52,7 @@ public class TestEncryptionRandomKeying {
private static Configuration conf = TEST_UTIL.getConfiguration(); private static Configuration conf = TEST_UTIL.getConfiguration();
private static HTableDescriptor htd; private static HTableDescriptor htd;
private static List<Path> findStorefilePaths(byte[] tableName) throws Exception { private static List<Path> findStorefilePaths(TableName tableName) throws Exception {
List<Path> paths = new ArrayList<Path>(); List<Path> paths = new ArrayList<Path>();
for (HRegion region: for (HRegion region:
TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) {
@ -103,7 +103,7 @@ public class TestEncryptionRandomKeying {
TEST_UTIL.waitTableAvailable(htd.getName(), 5000); TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
// Create a store file // Create a store file
Table table = new HTable(conf, htd.getName()); Table table = new HTable(conf, htd.getTableName());
try { try {
table.put(new Put(Bytes.toBytes("testrow")) table.put(new Put(Bytes.toBytes("testrow"))
.add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value"))); .add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value")));
@ -121,7 +121,7 @@ public class TestEncryptionRandomKeying {
@Test @Test
public void testRandomKeying() throws Exception { public void testRandomKeying() throws Exception {
// Verify we have store file(s) with a random key // Verify we have store file(s) with a random key
final List<Path> initialPaths = findStorefilePaths(htd.getName()); final List<Path> initialPaths = findStorefilePaths(htd.getTableName());
assertTrue(initialPaths.size() > 0); assertTrue(initialPaths.size() > 0);
for (Path path: initialPaths) { for (Path path: initialPaths) {
assertNotNull("Store file " + path + " is not encrypted", extractHFileKey(path)); assertNotNull("Store file " + path + " is not encrypted", extractHFileKey(path));

View File

@ -133,9 +133,9 @@ public class TestHRegionServerBulkLoad {
public static class AtomicHFileLoader extends RepeatingTestThread { public static class AtomicHFileLoader extends RepeatingTestThread {
final AtomicLong numBulkLoads = new AtomicLong(); final AtomicLong numBulkLoads = new AtomicLong();
final AtomicLong numCompactions = 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 { byte targetFamilies[][]) throws IOException {
super(ctx); super(ctx);
this.tableName = tableName; this.tableName = tableName;
@ -160,9 +160,8 @@ public class TestHRegionServerBulkLoad {
// bulk load HFiles // bulk load HFiles
final HConnection conn = UTIL.getHBaseAdmin().getConnection(); final HConnection conn = UTIL.getHBaseAdmin().getConnection();
TableName tbl = TableName.valueOf(tableName);
RegionServerCallable<Void> callable = RegionServerCallable<Void> callable =
new RegionServerCallable<Void>(conn, tbl, Bytes.toBytes("aaa")) { new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
@Override @Override
public Void call(int callTimeout) throws Exception { public Void call(int callTimeout) throws Exception {
LOG.debug("Going to connect to server " + getLocation() + " for row " LOG.debug("Going to connect to server " + getLocation() + " for row "
@ -181,7 +180,7 @@ public class TestHRegionServerBulkLoad {
// Periodically do compaction to reduce the number of open file handles. // Periodically do compaction to reduce the number of open file handles.
if (numBulkLoads.get() % 10 == 0) { if (numBulkLoads.get() % 10 == 0) {
// 10 * 50 = 500 open file handles! // 10 * 50 = 500 open file handles!
callable = new RegionServerCallable<Void>(conn, tbl, Bytes.toBytes("aaa")) { callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
@Override @Override
public Void call(int callTimeout) throws Exception { public Void call(int callTimeout) throws Exception {
LOG.debug("compacting " + getLocation() + " for row " LOG.debug("compacting " + getLocation() + " for row "
@ -210,9 +209,9 @@ public class TestHRegionServerBulkLoad {
HTable table; HTable table;
AtomicLong numScans = new AtomicLong(); AtomicLong numScans = new AtomicLong();
AtomicLong numRowsScanned = 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 { byte targetFamilies[][]) throws IOException {
super(ctx); super(ctx);
this.TABLE_NAME = TABLE_NAME; this.TABLE_NAME = TABLE_NAME;
@ -264,10 +263,10 @@ public class TestHRegionServerBulkLoad {
* Creates a table with given table name and specified number of column * Creates a table with given table name and specified number of column
* families if the table does not already exist. * 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 { try {
LOG.info("Creating table " + table); LOG.info("Creating table " + table);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); HTableDescriptor htd = new HTableDescriptor(table);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
htd.addFamily(new HColumnDescriptor(family(i))); htd.addFamily(new HColumnDescriptor(family(i)));
} }
@ -283,7 +282,7 @@ public class TestHRegionServerBulkLoad {
*/ */
@Test @Test
public void testAtomicBulkLoad() throws Exception { public void testAtomicBulkLoad() throws Exception {
String TABLE_NAME = "atomicBulkLoad"; TableName TABLE_NAME = TableName.valueOf("atomicBulkLoad");
int millisToRun = 30000; int millisToRun = 30000;
int numScanners = 50; int numScanners = 50;
@ -296,7 +295,7 @@ public class TestHRegionServerBulkLoad {
} }
} }
void runAtomicBulkloadTest(String tableName, int millisToRun, int numScanners) void runAtomicBulkloadTest(TableName tableName, int millisToRun, int numScanners)
throws Exception { throws Exception {
setupTable(tableName, 10); setupTable(tableName, 10);
@ -336,7 +335,7 @@ public class TestHRegionServerBulkLoad {
Configuration c = HBaseConfiguration.create(); Configuration c = HBaseConfiguration.create();
TestHRegionServerBulkLoad test = new TestHRegionServerBulkLoad(); TestHRegionServerBulkLoad test = new TestHRegionServerBulkLoad();
test.setConf(c); test.setConf(c);
test.runAtomicBulkloadTest("atomicTableTest", 5 * 60 * 1000, 50); test.runAtomicBulkloadTest(TableName.valueOf("atomicTableTest"), 5 * 60 * 1000, 50);
} finally { } finally {
System.exit(0); // something hangs (believe it is lru threadpool) System.exit(0); // something hangs (believe it is lru threadpool)
} }

View File

@ -77,7 +77,7 @@ public class TestRegionReplicas {
@BeforeClass @BeforeClass
public static void before() throws Exception { public static void before() throws Exception {
HTU.startMiniCluster(NB_SERVERS); 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. // Create table then get the single region for our new table.
table = HTU.createTable(tableName, f); table = HTU.createTable(tableName, f);

View File

@ -89,7 +89,7 @@ public class TestRegionServerMetrics {
String regionMetricsKey = "regionCount"; String regionMetricsKey = "regionCount";
long regions = metricsHelper.getGaugeLong(regionMetricsKey, serverSource); long regions = metricsHelper.getGaugeLong(regionMetricsKey, serverSource);
// Creating a table should add one region // 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); metricsHelper.assertGaugeGt(regionMetricsKey, regions, serverSource);
} }
@ -101,7 +101,7 @@ public class TestRegionServerMetrics {
@Test @Test
public void testRequestCount() throws Exception { public void testRequestCount() throws Exception {
String tableNameString = "testRequestCount"; String tableNameString = "testRequestCount";
byte[] tName = Bytes.toBytes(tableNameString); TableName tName = TableName.valueOf(tableNameString);
byte[] cfName = Bytes.toBytes("d"); byte[] cfName = Bytes.toBytes("d");
byte[] row = Bytes.toBytes("rk"); byte[] row = Bytes.toBytes("rk");
byte[] qualifier = Bytes.toBytes("qual"); byte[] qualifier = Bytes.toBytes("qual");
@ -181,7 +181,7 @@ public class TestRegionServerMetrics {
@Test @Test
public void testMutationsWithoutWal() throws Exception { public void testMutationsWithoutWal() throws Exception {
byte[] tableName = Bytes.toBytes("testMutationsWithoutWal"); TableName tableName = TableName.valueOf("testMutationsWithoutWal");
byte[] cf = Bytes.toBytes("d"); byte[] cf = Bytes.toBytes("d");
byte[] row = Bytes.toBytes("rk"); byte[] row = Bytes.toBytes("rk");
byte[] qualifier = Bytes.toBytes("qual"); byte[] qualifier = Bytes.toBytes("qual");
@ -240,7 +240,7 @@ public class TestRegionServerMetrics {
@Test @Test
public void testCheckAndPutCount() throws Exception { public void testCheckAndPutCount() throws Exception {
String tableNameString = "testCheckAndPutCount"; String tableNameString = "testCheckAndPutCount";
byte[] tableName = Bytes.toBytes(tableNameString); TableName tableName = TableName.valueOf(tableNameString);
byte[] cf = Bytes.toBytes("d"); byte[] cf = Bytes.toBytes("d");
byte[] row = Bytes.toBytes("rk"); byte[] row = Bytes.toBytes("rk");
byte[] qualifier = Bytes.toBytes("qual"); byte[] qualifier = Bytes.toBytes("qual");
@ -276,7 +276,7 @@ public class TestRegionServerMetrics {
@Test @Test
public void testIncrement() throws Exception { public void testIncrement() throws Exception {
String tableNameString = "testIncrement"; String tableNameString = "testIncrement";
byte[] tableName = Bytes.toBytes(tableNameString); TableName tableName = TableName.valueOf(tableNameString);
byte[] cf = Bytes.toBytes("d"); byte[] cf = Bytes.toBytes("d");
byte[] row = Bytes.toBytes("rk"); byte[] row = Bytes.toBytes("rk");
byte[] qualifier = Bytes.toBytes("qual"); byte[] qualifier = Bytes.toBytes("qual");
@ -308,7 +308,7 @@ public class TestRegionServerMetrics {
@Test @Test
public void testAppend() throws Exception { public void testAppend() throws Exception {
String tableNameString = "testAppend"; String tableNameString = "testAppend";
byte[] tableName = Bytes.toBytes(tableNameString); TableName tableName = TableName.valueOf(tableNameString);
byte[] cf = Bytes.toBytes("d"); byte[] cf = Bytes.toBytes("d");
byte[] row = Bytes.toBytes("rk"); byte[] row = Bytes.toBytes("rk");
byte[] qualifier = Bytes.toBytes("qual"); byte[] qualifier = Bytes.toBytes("qual");
@ -340,7 +340,7 @@ public class TestRegionServerMetrics {
@Test @Test
public void testScanNext() throws IOException { public void testScanNext() throws IOException {
String tableNameString = "testScanNext"; String tableNameString = "testScanNext";
byte[] tableName = Bytes.toBytes(tableNameString); TableName tableName = TableName.valueOf(tableNameString);
byte[] cf = Bytes.toBytes("d"); byte[] cf = Bytes.toBytes("d");
byte[] qualifier = Bytes.toBytes("qual"); byte[] qualifier = Bytes.toBytes("qual");
byte[] val = Bytes.toBytes("One"); byte[] val = Bytes.toBytes("One");

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@ -70,10 +70,10 @@ public class TestRegionServerNoMaster {
@BeforeClass @BeforeClass
public static void before() throws Exception { public static void before() throws Exception {
HTU.startMiniCluster(NB_SERVERS); 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. // 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); Put p = new Put(row);
p.add(HConstants.CATALOG_FAMILY, row, row); p.add(HConstants.CATALOG_FAMILY, row, row);
table.put(p); table.put(p);

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
@ -59,7 +60,7 @@ public class TestScannerWithBulkload {
TEST_UTIL.startMiniCluster(1); 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); HTableDescriptor desc = new HTableDescriptor(tableName);
HColumnDescriptor hcd = new HColumnDescriptor("col"); HColumnDescriptor hcd = new HColumnDescriptor("col");
hcd.setMaxVersions(3); hcd.setMaxVersions(3);
@ -69,7 +70,7 @@ public class TestScannerWithBulkload {
@Test @Test
public void testBulkLoad() throws Exception { public void testBulkLoad() throws Exception {
String tableName = "testBulkLoad"; TableName tableName = TableName.valueOf("testBulkLoad");
long l = System.currentTimeMillis(); long l = System.currentTimeMillis();
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
createTable(admin, tableName); createTable(admin, tableName);
@ -164,7 +165,7 @@ public class TestScannerWithBulkload {
return hfilePath; 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); HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
Put put0 = new Put(Bytes.toBytes("row1")); Put put0 = new Put(Bytes.toBytes("row1"));
put0.add(new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("col"), Bytes.toBytes("q"), l, Bytes put0.add(new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("col"), Bytes.toBytes("q"), l, Bytes
@ -198,7 +199,7 @@ public class TestScannerWithBulkload {
@Test @Test
public void testBulkLoadWithParallelScan() throws Exception { public void testBulkLoadWithParallelScan() throws Exception {
String tableName = "testBulkLoadWithParallelScan"; TableName tableName = TableName.valueOf("testBulkLoadWithParallelScan");
final long l = System.currentTimeMillis(); final long l = System.currentTimeMillis();
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
createTable(admin, tableName); createTable(admin, tableName);
@ -240,7 +241,7 @@ public class TestScannerWithBulkload {
@Test @Test
public void testBulkLoadNativeHFile() throws Exception { public void testBulkLoadNativeHFile() throws Exception {
String tableName = "testBulkLoadNativeHFile"; TableName tableName = TableName.valueOf("testBulkLoadNativeHFile");
long l = System.currentTimeMillis(); long l = System.currentTimeMillis();
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
createTable(admin, tableName); createTable(admin, tableName);

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
@ -134,7 +135,7 @@ public class TestServerCustomProtocol {
} }
} }
private static final byte[] TEST_TABLE = Bytes.toBytes("test"); private static final TableName TEST_TABLE = TableName.valueOf("test");
private static final byte[] TEST_FAMILY = Bytes.toBytes("f1"); private static final byte[] TEST_FAMILY = Bytes.toBytes("f1");
private static final byte[] ROW_A = Bytes.toBytes("aaa"); private static final byte[] ROW_A = Bytes.toBytes("aaa");

View File

@ -177,10 +177,10 @@ public class TestSplitTransactionOnCluster {
TableName.valueOf("testRITStateForRollback"); TableName.valueOf("testRITStateForRollback");
try { try {
// Create table then get the single region for our new table. // 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<HRegion> regions = cluster.getRegions(tableName); final List<HRegion> regions = cluster.getRegions(tableName);
final HRegionInfo hri = getAndCheckSingleTableRegion(regions); final HRegionInfo hri = getAndCheckSingleTableRegion(regions);
insertData(tableName.getName(), admin, t); insertData(tableName, admin, t);
t.close(); t.close();
// Turn off balancer so it doesn't cut in and mess up our placements. // Turn off balancer so it doesn't cut in and mess up our placements.
@ -219,7 +219,7 @@ public class TestSplitTransactionOnCluster {
} }
@Test(timeout = 60000) @Test(timeout = 60000)
public void testSplitFailedCompactionAndSplit() throws Exception { public void testSplitFailedCompactionAndSplit() throws Exception {
final byte[] tableName = Bytes.toBytes("testSplitFailedCompactionAndSplit"); final TableName tableName = TableName.valueOf("testSplitFailedCompactionAndSplit");
Configuration conf = TESTING_UTIL.getConfiguration(); Configuration conf = TESTING_UTIL.getConfiguration();
HBaseAdmin admin = new HBaseAdmin(conf); HBaseAdmin admin = new HBaseAdmin(conf);
// Create table then get the single region for our new table. // Create table then get the single region for our new table.
@ -284,8 +284,8 @@ public class TestSplitTransactionOnCluster {
*/ */
@Test (timeout = 300000) public void testRSSplitDaughtersAreOnlinedAfterShutdownHandling() @Test (timeout = 300000) public void testRSSplitDaughtersAreOnlinedAfterShutdownHandling()
throws IOException, InterruptedException, ServiceException { throws IOException, InterruptedException, ServiceException {
final byte [] tableName = final TableName tableName =
Bytes.toBytes("testRSSplitDaughtersAreOnlinedAfterShutdownHandling"); TableName.valueOf("testRSSplitDaughtersAreOnlinedAfterShutdownHandling");
// Create table then get the single region for our new table. // Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY); HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
@ -329,8 +329,8 @@ public class TestSplitTransactionOnCluster {
@Test (timeout = 300000) public void testExistingZnodeBlocksSplitAndWeRollback() @Test (timeout = 300000) public void testExistingZnodeBlocksSplitAndWeRollback()
throws IOException, InterruptedException, NodeExistsException, KeeperException, ServiceException { throws IOException, InterruptedException, NodeExistsException, KeeperException, ServiceException {
final byte [] tableName = final TableName tableName =
Bytes.toBytes("testExistingZnodeBlocksSplitAndWeRollback"); TableName.valueOf("testExistingZnodeBlocksSplitAndWeRollback");
// Create table then get the single region for our new table. // Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY); HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
@ -387,8 +387,8 @@ public class TestSplitTransactionOnCluster {
*/ */
@Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit() @Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit()
throws IOException, InterruptedException { throws IOException, InterruptedException {
final byte [] tableName = final TableName tableName =
Bytes.toBytes("testShutdownFixupWhenDaughterHasSplit"); TableName.valueOf("testShutdownFixupWhenDaughterHasSplit");
// Create table then get the single region for our new table. // Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY); HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
@ -554,7 +554,8 @@ public class TestSplitTransactionOnCluster {
public void testMasterRestartAtRegionSplitPendingCatalogJanitor() public void testMasterRestartAtRegionSplitPendingCatalogJanitor()
throws IOException, InterruptedException, NodeExistsException, throws IOException, InterruptedException, NodeExistsException,
KeeperException, ServiceException { 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. // Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY); HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
@ -622,13 +623,13 @@ public class TestSplitTransactionOnCluster {
final TableName tableName = final TableName tableName =
TableName.valueOf("testTableExistsIfTheSpecifiedTableRegionIsSplitParent"); TableName.valueOf("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
// Create table then get the single region for our new table. // 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<HRegion> regions = null; List<HRegion> regions = null;
try { try {
regions = cluster.getRegions(tableName); regions = cluster.getRegions(tableName);
int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName()); int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); 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. // Turn off balancer so it doesn't cut in and mess up our placements.
admin.setBalancerRunning(false, true); admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us. // Turn off the meta scanner so it don't remove parent on us.
@ -674,7 +675,7 @@ public class TestSplitTransactionOnCluster {
try { try {
int regionServerIndex = cluster.getServerWith(oldRegions.get(0).getRegionName()); int regionServerIndex = cluster.getServerWith(oldRegions.get(0).getRegionName());
HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); 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. // Turn off balancer so it doesn't cut in and mess up our placements.
admin.setBalancerRunning(false, true); admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us. // Turn off the meta scanner so it don't remove parent on us.
@ -731,7 +732,7 @@ public class TestSplitTransactionOnCluster {
} }
} }
private void insertData(final byte[] tableName, HBaseAdmin admin, Table t) throws IOException, private void insertData(final TableName tableName, HBaseAdmin admin, Table t) throws IOException,
InterruptedException { InterruptedException {
Put p = new Put(Bytes.toBytes("row1")); Put p = new Put(Bytes.toBytes("row1"));
p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("1")); p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("1"));
@ -758,7 +759,7 @@ public class TestSplitTransactionOnCluster {
final TableName tableName = final TableName tableName =
TableName.valueOf("testSplitRegionWithNoStoreFiles"); TableName.valueOf("testSplitRegionWithNoStoreFiles");
// Create table then get the single region for our new table. // Create table then get the single region for our new table.
createTableAndWait(tableName.getName(), HConstants.CATALOG_FAMILY); createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName); List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions); HRegionInfo hri = getAndCheckSingleTableRegion(regions);
ensureTableRegionNotOnSameServerAsMeta(admin, hri); ensureTableRegionNotOnSameServerAsMeta(admin, hri);
@ -841,7 +842,7 @@ public class TestSplitTransactionOnCluster {
LOG.info("Starting testSplitAndRestartingMaster"); LOG.info("Starting testSplitAndRestartingMaster");
final TableName tableName = TableName.valueOf("testSplitAndRestartingMaster"); final TableName tableName = TableName.valueOf("testSplitAndRestartingMaster");
// Create table then get the single region for our new table. // Create table then get the single region for our new table.
createTableAndWait(tableName.getName(), HConstants.CATALOG_FAMILY); createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName); List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions); HRegionInfo hri = getAndCheckSingleTableRegion(regions);
ensureTableRegionNotOnSameServerAsMeta(admin, hri); ensureTableRegionNotOnSameServerAsMeta(admin, hri);
@ -880,19 +881,19 @@ public class TestSplitTransactionOnCluster {
@Test(timeout = 180000) @Test(timeout = 180000)
public void testSplitHooksBeforeAndAfterPONR() throws Exception { public void testSplitHooksBeforeAndAfterPONR() throws Exception {
String firstTable = "testSplitHooksBeforeAndAfterPONR_1"; TableName firstTable = TableName.valueOf("testSplitHooksBeforeAndAfterPONR_1");
String secondTable = "testSplitHooksBeforeAndAfterPONR_2"; TableName secondTable = TableName.valueOf("testSplitHooksBeforeAndAfterPONR_2");
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(firstTable)); HTableDescriptor desc = new HTableDescriptor(firstTable);
desc.addCoprocessor(MockedRegionObserver.class.getName()); desc.addCoprocessor(MockedRegionObserver.class.getName());
HColumnDescriptor hcd = new HColumnDescriptor("cf"); HColumnDescriptor hcd = new HColumnDescriptor("cf");
desc.addFamily(hcd); desc.addFamily(hcd);
admin.createTable(desc); admin.createTable(desc);
desc = new HTableDescriptor(TableName.valueOf(secondTable)); desc = new HTableDescriptor(secondTable);
hcd = new HColumnDescriptor("cf"); hcd = new HColumnDescriptor("cf");
desc.addFamily(hcd); desc.addFamily(hcd);
admin.createTable(desc); admin.createTable(desc);
List<HRegion> firstTableregions = cluster.getRegions(TableName.valueOf(firstTable)); List<HRegion> firstTableregions = cluster.getRegions(firstTable);
List<HRegion> secondTableRegions = cluster.getRegions(TableName.valueOf(secondTable)); List<HRegion> secondTableRegions = cluster.getRegions(secondTable);
ServerName serverName = ServerName serverName =
cluster.getServerHoldingRegion(firstTableregions.get(0).getRegionName()); cluster.getServerHoldingRegion(firstTableregions.get(0).getRegionName());
admin.move(secondTableRegions.get(0).getRegionInfo().getEncodedNameAsBytes(), admin.move(secondTableRegions.get(0).getRegionInfo().getEncodedNameAsBytes(),
@ -902,16 +903,16 @@ public class TestSplitTransactionOnCluster {
try { try {
table1 = new HTable(TESTING_UTIL.getConfiguration(), firstTable); table1 = new HTable(TESTING_UTIL.getConfiguration(), firstTable);
table2 = new HTable(TESTING_UTIL.getConfiguration(), firstTable); table2 = new HTable(TESTING_UTIL.getConfiguration(), firstTable);
insertData(Bytes.toBytes(firstTable), admin, table1); insertData(firstTable, admin, table1);
insertData(Bytes.toBytes(secondTable), admin, table2); insertData(secondTable, admin, table2);
admin.split(Bytes.toBytes(firstTable), "row2".getBytes()); admin.split(firstTable, "row2".getBytes());
firstTableregions = cluster.getRegions(Bytes.toBytes(firstTable)); firstTableregions = cluster.getRegions(firstTable);
while (firstTableregions.size() != 2) { while (firstTableregions.size() != 2) {
Thread.sleep(1000); 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()); 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()); assertEquals("Number of regions after split should be 2.", 2, secondTableRegions.size());
} finally { } finally {
if (table1 != null) { if (table1 != null) {
@ -926,7 +927,7 @@ public class TestSplitTransactionOnCluster {
} }
private void testSplitBeforeSettingSplittingInZKInternals() throws Exception { private void testSplitBeforeSettingSplittingInZKInternals() throws Exception {
final byte[] tableName = Bytes.toBytes("testSplitBeforeSettingSplittingInZK"); final TableName tableName = TableName.valueOf("testSplitBeforeSettingSplittingInZK");
try { try {
// Create table then get the single region for our new table. // Create table then get the single region for our new table.
createTableAndWait(tableName, Bytes.toBytes("cf")); createTableAndWait(tableName, Bytes.toBytes("cf"));
@ -1001,7 +1002,7 @@ public class TestSplitTransactionOnCluster {
return(null); return(null);
} }
private List<HRegion> checkAndGetDaughters(byte[] tableName) private List<HRegion> checkAndGetDaughters(TableName tableName)
throws InterruptedException { throws InterruptedException {
List<HRegion> daughters = null; List<HRegion> daughters = null;
// try up to 10s // try up to 10s
@ -1126,7 +1127,7 @@ public class TestSplitTransactionOnCluster {
|| cluster.getLiveRegionServerThreads().size() > NB_SERVERS); || cluster.getLiveRegionServerThreads().size() > NB_SERVERS);
} }
private void awaitDaughters(byte[] tableName, int numDaughters) throws InterruptedException { private void awaitDaughters(TableName tableName, int numDaughters) throws InterruptedException {
// Wait till regions are back on line again. // Wait till regions are back on line again.
for (int i=0; cluster.getRegions(tableName).size() < numDaughters && i<60; i++) { for (int i=0; cluster.getRegions(tableName).size() < numDaughters && i<60; i++) {
LOG.info("Waiting for repair to happen"); LOG.info("Waiting for repair to happen");
@ -1137,7 +1138,7 @@ public class TestSplitTransactionOnCluster {
} }
} }
private List<HRegion> awaitTableRegions(final byte[] tableName) throws InterruptedException { private List<HRegion> awaitTableRegions(final TableName tableName) throws InterruptedException {
List<HRegion> regions = null; List<HRegion> regions = null;
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
regions = cluster.getRegions(tableName); regions = cluster.getRegions(tableName);
@ -1147,11 +1148,11 @@ public class TestSplitTransactionOnCluster {
return regions; return regions;
} }
private HTable createTableAndWait(byte[] tableName, byte[] cf) throws IOException, private HTable createTableAndWait(TableName tableName, byte[] cf) throws IOException,
InterruptedException { InterruptedException {
HTable t = TESTING_UTIL.createTable(tableName, cf); HTable t = TESTING_UTIL.createTable(tableName, cf);
awaitTableRegions(tableName); awaitTableRegions(tableName);
assertTrue("Table not online: " + Bytes.toString(tableName), assertTrue("Table not online: " + tableName,
cluster.getRegions(tableName).size() != 0); cluster.getRegions(tableName).size() != 0);
return t; return t;
} }

View File

@ -126,15 +126,15 @@ public class TestLogRollAbort {
TableName.META_TABLE_NAME).close(); TableName.META_TABLE_NAME).close();
// Create the test table and open it // Create the test table and open it
String tableName = this.getClass().getSimpleName(); TableName tableName = TableName.valueOf(this.getClass().getSimpleName());
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); admin.createTable(desc);
Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
try { try {
HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName);
HLog log = server.getWAL(); HLog log = server.getWAL();
assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas());

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
@ -72,13 +73,12 @@ public class TestLogRollPeriod {
*/ */
@Test @Test
public void testNoEdits() throws Exception { public void testNoEdits() throws Exception {
final String tableName = "TestLogRollPeriodNoEdits"; TableName tableName = TableName.valueOf("TestLogRollPeriodNoEdits");
TEST_UTIL.createTable(tableName, "cf"); TEST_UTIL.createTable(tableName, "cf");
try { try {
Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); Table table = new HTable(TEST_UTIL.getConfiguration(), tableName);
try { try {
HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName);
HLog log = server.getWAL(); HLog log = server.getWAL();
checkMinLogRolls(log, 5); checkMinLogRolls(log, 5);
} finally { } finally {
@ -94,12 +94,12 @@ public class TestLogRollPeriod {
*/ */
@Test(timeout=60000) @Test(timeout=60000)
public void testWithEdits() throws Exception { public void testWithEdits() throws Exception {
final String tableName = "TestLogRollPeriodWithEdits"; final TableName tableName = TableName.valueOf("TestLogRollPeriodWithEdits");
final String family = "cf"; final String family = "cf";
TEST_UTIL.createTable(tableName, family); TEST_UTIL.createTable(tableName, family);
try { try {
HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName);
HLog log = server.getWAL(); HLog log = server.getWAL();
final Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); final Table table = new HTable(TEST_UTIL.getConfiguration(), tableName);

View File

@ -179,7 +179,7 @@ public class TestLogRolling {
Table table = createTestTable(this.tableName); Table table = createTestTable(this.tableName);
server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
this.log = server.getWAL(); this.log = server.getWAL();
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
doPut(table, i); doPut(table, i);
@ -332,15 +332,14 @@ public class TestLogRolling {
this.log = server.getWAL(); this.log = server.getWAL();
// Create the test table and open it // Create the test table and open it
String tableName = getName(); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getName()));
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); admin.createTable(desc);
Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
assertTrue(table.isAutoFlush()); assertTrue(table.isAutoFlush());
server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
this.log = server.getWAL(); this.log = server.getWAL();
assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas());
@ -440,14 +439,13 @@ public class TestLogRolling {
this.log = server.getWAL(); this.log = server.getWAL();
// Create the test table and open it // Create the test table and open it
String tableName = getName(); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getName()));
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); 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(); this.log = server.getWAL();
final List<Path> paths = new ArrayList<Path>(); final List<Path> paths = new ArrayList<Path>();
final List<Integer> preLogRolledCalled = new ArrayList<Integer>(); final List<Integer> preLogRolledCalled = new ArrayList<Integer>();
@ -599,12 +597,10 @@ public class TestLogRolling {
// When the hbase:meta table can be opened, the region servers are running // When the hbase:meta table can be opened, the region servers are running
Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
try { try {
String tableName = getName(); table = createTestTable(getName());
table = createTestTable(tableName); table2 = createTestTable(getName() + "1");
String tableName2 = tableName + "1";
table2 = createTestTable(tableName2);
server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
this.log = server.getWAL(); this.log = server.getWAL();
FSHLog fshLog = (FSHLog)log; FSHLog fshLog = (FSHLog)log;
HRegion region = server.getOnlineRegions(table2.getName()).get(0); HRegion region = server.getOnlineRegions(table2.getName()).get(0);
@ -662,7 +658,7 @@ public class TestLogRolling {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); admin.createTable(desc);
return new HTable(TEST_UTIL.getConfiguration(), tableName); return new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
} }
} }

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Stoppable; 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.Get;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -62,10 +63,10 @@ public class TestReplicationSink {
private static ReplicationSink SINK; private static ReplicationSink SINK;
private static final byte[] TABLE_NAME1 = private static final TableName TABLE_NAME1 =
Bytes.toBytes("table1"); TableName.valueOf("table1");
private static final byte[] TABLE_NAME2 = private static final TableName TABLE_NAME2 =
Bytes.toBytes("table2"); TableName.valueOf("table2");
private static final byte[] FAM_NAME1 = Bytes.toBytes("info1"); private static final byte[] FAM_NAME1 = Bytes.toBytes("info1");
private static final byte[] FAM_NAME2 = Bytes.toBytes("info2"); private static final byte[] FAM_NAME2 = Bytes.toBytes("info2");
@ -233,8 +234,8 @@ public class TestReplicationSink {
assertEquals(0, res.size()); assertEquals(0, res.size());
} }
private WALEntry createEntry(byte [] table, int row, KeyValue.Type type, List<Cell> cells) { private WALEntry createEntry(TableName table, int row, KeyValue.Type type, List<Cell> cells) {
byte[] fam = Bytes.equals(table, TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2; byte[] fam = table.equals(TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2;
byte[] rowBytes = Bytes.toBytes(row); byte[] rowBytes = Bytes.toBytes(row);
// Just make sure we don't get the same ts for two consecutive rows with // Just make sure we don't get the same ts for two consecutive rows with
// same key // same key
@ -262,7 +263,7 @@ public class TestReplicationSink {
uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits()); uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits());
uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits()); uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
keyBuilder.setClusterId(uuidBuilder.build()); keyBuilder.setClusterId(uuidBuilder.build());
keyBuilder.setTableName(ByteStringer.wrap(table)); keyBuilder.setTableName(ByteStringer.wrap(table.getName()));
keyBuilder.setWriteTime(now); keyBuilder.setWriteTime(now);
keyBuilder.setEncodedRegionName(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY)); keyBuilder.setEncodedRegionName(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY));
keyBuilder.setLogSequenceNumber(-1); keyBuilder.setLogSequenceNumber(-1);

View File

@ -111,7 +111,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityLabelsWithDeleteColumns() throws Throwable { public void testVisibilityLabelsWithDeleteColumns() throws Throwable {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
final Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + TOPSECRET, final Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + TOPSECRET,
SECRET); SECRET);
try { try {
@ -120,7 +120,7 @@ public class TestVisibilityLabelsWithDeletes {
public Void run() throws Exception { public Void run() throws Exception {
Table table = null; Table table = null;
try { try {
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(TOPSECRET + "&" + SECRET)); d.setCellVisibility(new CellVisibility(TOPSECRET + "&" + SECRET));
d.deleteColumns(fam, qual); d.deleteColumns(fam, qual);
@ -157,7 +157,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityLabelsWithDeleteFamily() throws Exception { public void testVisibilityLabelsWithDeleteFamily() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
final Table table = createTableAndWriteDataWithLabels(tableName, SECRET, CONFIDENTIAL + "|" final Table table = createTableAndWriteDataWithLabels(tableName, SECRET, CONFIDENTIAL + "|"
+ TOPSECRET); + TOPSECRET);
try { try {
@ -165,7 +165,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row2); Delete d = new Delete(row2);
d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -199,7 +199,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityLabelsWithDeleteFamilyVersion() throws Exception { public void testVisibilityLabelsWithDeleteFamilyVersion() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
long[] ts = new long[] { 123l, 125l }; long[] ts = new long[] { 123l, 125l };
final Table table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|" final Table table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|"
+ TOPSECRET, SECRET); + TOPSECRET, SECRET);
@ -209,7 +209,7 @@ public class TestVisibilityLabelsWithDeletes {
public Void run() throws Exception { public Void run() throws Exception {
Table table = null; Table table = null;
try { try {
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
d.deleteFamilyVersion(fam, 123l); d.deleteFamilyVersion(fam, 123l);
@ -245,7 +245,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityLabelsWithDeleteColumnExactVersion() throws Exception { public void testVisibilityLabelsWithDeleteColumnExactVersion() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
long[] ts = new long[] { 123l, 125l }; long[] ts = new long[] { 123l, 125l };
final Table table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|" final Table table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|"
+ TOPSECRET, SECRET); + TOPSECRET, SECRET);
@ -255,7 +255,7 @@ public class TestVisibilityLabelsWithDeletes {
public Void run() throws Exception { public Void run() throws Exception {
Table table = null; Table table = null;
try { try {
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
d.deleteColumn(fam, qual, 123l); d.deleteColumn(fam, qual, 123l);
@ -291,7 +291,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersions() throws Exception { public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersions() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -300,7 +300,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
SECRET + "&" + TOPSECRET+")")); SECRET + "&" + TOPSECRET+")"));
@ -353,7 +353,7 @@ public class TestVisibilityLabelsWithDeletes {
public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersionsNoTimestamp() public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersionsNoTimestamp()
throws Exception { throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -362,7 +362,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteColumns(fam, qual); d.deleteColumns(fam, qual);
@ -410,7 +410,7 @@ public class TestVisibilityLabelsWithDeletes {
testVisibilityLabelsWithDeleteColumnsWithNoMatchVisExpWithMultipleVersionsNoTimestamp() testVisibilityLabelsWithDeleteColumnsWithNoMatchVisExpWithMultipleVersionsNoTimestamp()
throws Exception { throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -419,7 +419,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteColumns(fam, qual); d.deleteColumns(fam, qual);
@ -471,7 +471,7 @@ public class TestVisibilityLabelsWithDeletes {
public void testVisibilityLabelsWithDeleteFamilyWithMultipleVersionsNoTimestamp() public void testVisibilityLabelsWithDeleteFamilyWithMultipleVersionsNoTimestamp()
throws Exception { throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -480,7 +480,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -525,7 +525,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityLabelsWithDeleteFamilyWithPutsReAppearing() throws Exception { public void testVisibilityLabelsWithDeleteFamilyWithPutsReAppearing() throws Exception {
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
@ -548,7 +548,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -575,7 +575,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET)); d.setCellVisibility(new CellVisibility(SECRET));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -609,7 +609,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityLabelsWithDeleteColumnsWithPutsReAppearing() throws Exception { public void testVisibilityLabelsWithDeleteColumnsWithPutsReAppearing() throws Exception {
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
@ -632,7 +632,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteColumns(fam, qual); d.deleteColumns(fam, qual);
@ -659,7 +659,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET)); d.setCellVisibility(new CellVisibility(SECRET));
d.deleteColumns(fam, qual); d.deleteColumns(fam, qual);
@ -693,7 +693,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityCombinations() throws Exception { public void testVisibilityCombinations() throws Exception {
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
@ -716,13 +716,13 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET)); d.setCellVisibility(new CellVisibility(SECRET));
d.deleteColumns(fam, qual, 126l); d.deleteColumns(fam, qual, 126l);
table.delete(d); table.delete(d);
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
d = new Delete(row1); d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteColumn(fam, qual, 123l); d.deleteColumn(fam, qual, 123l);
@ -750,7 +750,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityLabelsWithDeleteColumnWithSpecificVersionWithPutsReAppearing() public void testVisibilityLabelsWithDeleteColumnWithSpecificVersionWithPutsReAppearing()
throws Exception { throws Exception {
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
@ -780,13 +780,13 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteColumn(fam, qual, 123l); d.deleteColumn(fam, qual, 123l);
table.delete(d); table.delete(d);
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
d = new Delete(row1); d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET)); d.setCellVisibility(new CellVisibility(SECRET));
d.deleteColumn(fam, qual, 123l); d.deleteColumn(fam, qual, 123l);
@ -817,7 +817,7 @@ public class TestVisibilityLabelsWithDeletes {
testVisibilityLabelsWithDeleteFamilyWithNoMatchingVisExpWithMultipleVersionsNoTimestamp() testVisibilityLabelsWithDeleteFamilyWithNoMatchingVisExpWithMultipleVersionsNoTimestamp()
throws Exception { throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -826,7 +826,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -877,7 +877,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteFamilyAndDeleteColumnsWithAndWithoutVisibilityExp() throws Exception { public void testDeleteFamilyAndDeleteColumnsWithAndWithoutVisibilityExp() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -886,7 +886,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteFamily(fam); d.deleteFamily(fam);
table.delete(d); table.delete(d);
@ -1056,7 +1056,7 @@ public class TestVisibilityLabelsWithDeletes {
public void testDeleteColumnWithSpecificTimeStampUsingMultipleVersionsUnMatchingVisExpression() public void testDeleteColumnWithSpecificTimeStampUsingMultipleVersionsUnMatchingVisExpression()
throws Exception { throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -1065,7 +1065,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
SECRET + "&" + TOPSECRET+")")); SECRET + "&" + TOPSECRET+")"));
@ -1127,7 +1127,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteColumnWithLatestTimeStampUsingMultipleVersions() throws Exception { public void testDeleteColumnWithLatestTimeStampUsingMultipleVersions() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -1136,7 +1136,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
d.deleteColumn(fam, qual); d.deleteColumn(fam, qual);
@ -1192,7 +1192,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test (timeout=180000) @Test (timeout=180000)
public void testDeleteColumnWithLatestTimeStampWhenNoVersionMatches() throws Exception { public void testDeleteColumnWithLatestTimeStampWhenNoVersionMatches() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -1205,7 +1205,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET )); d.setCellVisibility(new CellVisibility(SECRET ));
d.deleteColumn(fam, qual); d.deleteColumn(fam, qual);
@ -1285,7 +1285,7 @@ public class TestVisibilityLabelsWithDeletes {
public void testDeleteColumnWithLatestTimeStampUsingMultipleVersionsAfterCompaction() public void testDeleteColumnWithLatestTimeStampUsingMultipleVersionsAfterCompaction()
throws Exception { throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -1294,7 +1294,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
d.deleteColumn(fam, qual); d.deleteColumn(fam, qual);
@ -1357,7 +1357,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteFamilyLatestTimeStampWithMulipleVersions() throws Exception { public void testDeleteFamilyLatestTimeStampWithMulipleVersions() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -1366,7 +1366,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -1412,7 +1412,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteColumnswithMultipleColumnsWithMultipleVersions() throws Exception { public void testDeleteColumnswithMultipleColumnsWithMultipleVersions() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPutsWithDiffCols(tableName); table = doPutsWithDiffCols(tableName);
@ -1421,7 +1421,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
d.deleteColumns(fam, qual, 125l); d.deleteColumns(fam, qual, 125l);
@ -1475,7 +1475,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteColumnsWithDiffColsAndTags() throws Exception { public void testDeleteColumnsWithDiffColsAndTags() throws Exception {
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
@ -1498,7 +1498,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET)); d.setCellVisibility(new CellVisibility(SECRET));
d.deleteColumns(fam, qual, 126l); d.deleteColumns(fam, qual, 126l);
@ -1529,7 +1529,7 @@ public class TestVisibilityLabelsWithDeletes {
} }
@Test @Test
public void testDeleteColumnsWithDiffColsAndTags1() throws Exception { public void testDeleteColumnsWithDiffColsAndTags1() throws Exception {
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
@ -1552,7 +1552,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET)); d.setCellVisibility(new CellVisibility(SECRET));
d.deleteColumns(fam, qual, 126l); d.deleteColumns(fam, qual, 126l);
@ -1584,7 +1584,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteFamilyWithoutCellVisibilityWithMulipleVersions() throws Exception { public void testDeleteFamilyWithoutCellVisibilityWithMulipleVersions() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPutsWithoutVisibility(tableName); table = doPutsWithoutVisibility(tableName);
@ -1593,7 +1593,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteFamily(fam); d.deleteFamily(fam);
table.delete(d); table.delete(d);
@ -1629,7 +1629,7 @@ public class TestVisibilityLabelsWithDeletes {
public void testDeleteFamilyLatestTimeStampWithMulipleVersionsWithoutCellVisibilityInPuts() public void testDeleteFamilyLatestTimeStampWithMulipleVersionsWithoutCellVisibilityInPuts()
throws Exception { throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPutsWithoutVisibility(tableName); table = doPutsWithoutVisibility(tableName);
@ -1637,7 +1637,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -1697,7 +1697,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteFamilySpecificTimeStampWithMulipleVersions() throws Exception { public void testDeleteFamilySpecificTimeStampWithMulipleVersions() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -1706,7 +1706,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
+ SECRET + "&" + TOPSECRET + ")")); + SECRET + "&" + TOPSECRET + ")"));
@ -1758,7 +1758,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testScanAfterCompaction() throws Exception { public void testScanAfterCompaction() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -1767,7 +1767,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
SECRET + "&" + TOPSECRET+")")); SECRET + "&" + TOPSECRET+")"));
@ -1817,7 +1817,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteFamilySpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception { public void testDeleteFamilySpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
// Do not flush here. // Do not flush here.
@ -1826,7 +1826,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
+ TOPSECRET + "&" + SECRET+")")); + TOPSECRET + "&" + SECRET+")"));
@ -1878,7 +1878,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
+ TOPSECRET + "&" + SECRET+")")); + TOPSECRET + "&" + SECRET+")"));
@ -1936,14 +1936,14 @@ public class TestVisibilityLabelsWithDeletes {
} }
}; };
VisibilityLabelsResponse response = SUPERUSER.runAs(action); VisibilityLabelsResponse response = SUPERUSER.runAs(action);
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = doPuts(tableName); Table table = doPuts(tableName);
try { try {
PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() { PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteFamilyVersion(fam, 123l); d.deleteFamilyVersion(fam, 123l);
@ -1993,14 +1993,14 @@ public class TestVisibilityLabelsWithDeletes {
@Test (timeout=180000) @Test (timeout=180000)
public void testSpecificDeletesFollowedByDeleteFamily() throws Exception { public void testSpecificDeletesFollowedByDeleteFamily() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = doPuts(tableName); Table table = doPuts(tableName);
try { try {
PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() { PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
+ TOPSECRET + "&" + SECRET + ")")); + TOPSECRET + "&" + SECRET + ")"));
@ -2046,7 +2046,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -2097,14 +2097,14 @@ public class TestVisibilityLabelsWithDeletes {
} }
}; };
VisibilityLabelsResponse response = SUPERUSER.runAs(action); VisibilityLabelsResponse response = SUPERUSER.runAs(action);
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = doPuts(tableName); Table table = doPuts(tableName);
try { try {
PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() { PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
+ TOPSECRET + "&" + SECRET + ")")); + TOPSECRET + "&" + SECRET + ")"));
@ -2151,7 +2151,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
d.deleteFamily(fam); d.deleteFamily(fam);
@ -2191,7 +2191,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception { public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
// Do not flush here. // Do not flush here.
@ -2200,7 +2200,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
d.deleteColumn(fam, qual, 125l); d.deleteColumn(fam, qual, 125l);
@ -2251,7 +2251,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
+ TOPSECRET + "&" + SECRET+")")); + TOPSECRET + "&" + SECRET+")"));
@ -2302,7 +2302,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice1() throws Exception { public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice1() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
// Do not flush here. // Do not flush here.
@ -2311,7 +2311,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")" + d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")" +
"|(" + TOPSECRET + "&" + SECRET + ")")); "|(" + TOPSECRET + "&" + SECRET + ")"));
@ -2363,7 +2363,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
d.deleteColumn(fam, qual, 127l); d.deleteColumn(fam, qual, 127l);
@ -2417,7 +2417,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice2() throws Exception { public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice2() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
// Do not flush here. // Do not flush here.
@ -2426,7 +2426,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
+ TOPSECRET + "&" + SECRET+")")); + TOPSECRET + "&" + SECRET+")"));
@ -2483,7 +2483,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
+ TOPSECRET + "&" + SECRET+")")); + TOPSECRET + "&" + SECRET+")"));
@ -2539,7 +2539,7 @@ public class TestVisibilityLabelsWithDeletes {
public void testDeleteColumnAndDeleteFamilylSpecificTimeStampWithMulipleVersion() public void testDeleteColumnAndDeleteFamilylSpecificTimeStampWithMulipleVersion()
throws Exception { throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
// Do not flush here. // Do not flush here.
@ -2548,7 +2548,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
d.deleteColumn(fam, qual, 125l); d.deleteColumn(fam, qual, 125l);
@ -2599,7 +2599,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
+ TOPSECRET + "&" + SECRET+")")); + TOPSECRET + "&" + SECRET+")"));
@ -2661,7 +2661,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDiffDeleteTypesForTheSameCellUsingMultipleVersions() throws Exception { public void testDiffDeleteTypesForTheSameCellUsingMultipleVersions() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
// Do not flush here. // Do not flush here.
@ -2670,7 +2670,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
+ TOPSECRET + "&" + SECRET+")")); + TOPSECRET + "&" + SECRET+")"));
@ -2722,7 +2722,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
+ TOPSECRET + "&" + SECRET+")")); + TOPSECRET + "&" + SECRET+")"));
@ -2772,7 +2772,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testDeleteColumnLatestWithNoCellVisibility() throws Exception { public void testDeleteColumnLatestWithNoCellVisibility() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = doPuts(tableName); table = doPuts(tableName);
@ -2781,7 +2781,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteColumn(fam, qual, 125l); d.deleteColumn(fam, qual, 125l);
table.delete(d); table.delete(d);
@ -2805,7 +2805,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteColumns(fam, qual, 125l); d.deleteColumns(fam, qual, 125l);
table.delete(d); table.delete(d);
@ -2830,7 +2830,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteFamily(fam, 125l); d.deleteFamily(fam, 125l);
table.delete(d); table.delete(d);
@ -2855,7 +2855,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteFamily(fam); d.deleteFamily(fam);
table.delete(d); table.delete(d);
@ -2880,7 +2880,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteColumns(fam, qual); d.deleteColumns(fam, qual);
table.delete(d); table.delete(d);
@ -2905,7 +2905,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteFamilyVersion(fam, 126l); d.deleteFamilyVersion(fam, 126l);
table.delete(d); table.delete(d);
@ -2969,7 +2969,7 @@ public class TestVisibilityLabelsWithDeletes {
@Test @Test
public void testVisibilityExpressionWithNotEqualORCondition() throws Exception { public void testVisibilityExpressionWithNotEqualORCondition() throws Exception {
setAuths(); setAuths();
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
@ -2992,7 +2992,7 @@ public class TestVisibilityLabelsWithDeletes {
@Override @Override
public Void run() throws Exception { public Void run() throws Exception {
try { try {
Table table = new HTable(conf, TEST_NAME.getMethodName()); Table table = new HTable(conf, tableName);
Delete d = new Delete(row1); Delete d = new Delete(row1);
d.deleteColumn(fam, qual, 124l); d.deleteColumn(fam, qual, 124l);
d.setCellVisibility(new CellVisibility(PRIVATE )); d.setCellVisibility(new CellVisibility(PRIVATE ));

View File

@ -120,7 +120,7 @@ public class TestVisibilityWithCheckAuths {
} }
}; };
SUPERUSER.runAs(action); SUPERUSER.runAs(action);
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
HColumnDescriptor colDesc = new HColumnDescriptor(fam); HColumnDescriptor colDesc = new HColumnDescriptor(fam);
colDesc.setMaxVersions(5); colDesc.setMaxVersions(5);
@ -135,7 +135,7 @@ public class TestVisibilityWithCheckAuths {
public Void run() throws Exception { public Void run() throws Exception {
Table table = null; Table table = null;
try { try {
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
Put p = new Put(row1); Put p = new Put(row1);
p.setCellVisibility(new CellVisibility(PUBLIC + "&" + TOPSECRET)); p.setCellVisibility(new CellVisibility(PUBLIC + "&" + TOPSECRET));
p.add(fam, qual, 125l, value); p.add(fam, qual, 125l, value);
@ -170,7 +170,7 @@ public class TestVisibilityWithCheckAuths {
} }
}; };
SUPERUSER.runAs(action); SUPERUSER.runAs(action);
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
Table table = null; Table table = null;
try { try {
table = TEST_UTIL.createTable(tableName, fam); table = TEST_UTIL.createTable(tableName, fam);
@ -181,7 +181,7 @@ public class TestVisibilityWithCheckAuths {
public Void run() throws Exception { public Void run() throws Exception {
Table table = null; Table table = null;
try { try {
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
Put put = new Put(row1); Put put = new Put(row1);
put.add(fam, qual, HConstants.LATEST_TIMESTAMP, val); put.add(fam, qual, HConstants.LATEST_TIMESTAMP, val);
put.setCellVisibility(new CellVisibility(TOPSECRET)); put.setCellVisibility(new CellVisibility(TOPSECRET));
@ -198,7 +198,7 @@ public class TestVisibilityWithCheckAuths {
public Void run() throws Exception { public Void run() throws Exception {
Table table = null; Table table = null;
try { try {
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
Append append = new Append(row1); Append append = new Append(row1);
append.add(fam, qual, Bytes.toBytes("b")); append.add(fam, qual, Bytes.toBytes("b"));
table.append(append); table.append(append);
@ -214,7 +214,7 @@ public class TestVisibilityWithCheckAuths {
public Void run() throws Exception { public Void run() throws Exception {
Table table = null; Table table = null;
try { try {
table = new HTable(conf, TEST_NAME.getMethodName()); table = new HTable(conf, tableName);
Append append = new Append(row1); Append append = new Append(row1);
append.add(fam, qual, Bytes.toBytes("c")); append.add(fam, qual, Bytes.toBytes("c"));
append.setCellVisibility(new CellVisibility(PUBLIC)); append.setCellVisibility(new CellVisibility(PUBLIC));

View File

@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
import java.util.Collection; import java.util.Collection;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -67,7 +68,7 @@ public class TestHTraceHooks {
Table table; Table table;
try { try {
table = TEST_UTIL.createTable("table".getBytes(), table = TEST_UTIL.createTable(TableName.valueOf("table"),
FAMILY_BYTES); FAMILY_BYTES);
} finally { } finally {
tableCreationSpan.close(); tableCreationSpan.close();

View File

@ -163,8 +163,8 @@ public class TestHBaseFsck {
@Test @Test
public void testHBaseFsck() throws Exception { public void testHBaseFsck() throws Exception {
assertNoErrors(doFsck(conf, false)); assertNoErrors(doFsck(conf, false));
String table = "tableBadMetaAssign"; TableName table = TableName.valueOf("tableBadMetaAssign");
TEST_UTIL.createTable(Bytes.toBytes(table), FAM); TEST_UTIL.createTable(table, FAM);
// We created 1 table, should be fine // We created 1 table, should be fine
assertNoErrors(doFsck(conf, false)); assertNoErrors(doFsck(conf, false));
@ -216,7 +216,7 @@ public class TestHBaseFsck {
assertNoErrors(doFsck(conf, false)); assertNoErrors(doFsck(conf, false));
// comment needed - what is the purpose of this line // 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()); ResultScanner s = t.getScanner(new Scan());
s.close(); s.close();
t.close(); t.close();

View File

@ -103,7 +103,7 @@ public class TestHBaseFsckEncryption {
@Test @Test
public void testFsckWithEncryption() throws Exception { public void testFsckWithEncryption() throws Exception {
// Populate the table with some data // Populate the table with some data
Table table = new HTable(conf, htd.getName()); Table table = new HTable(conf, htd.getTableName());
try { try {
byte[] values = { 'A', 'B', 'C', 'D' }; byte[] values = { 'A', 'B', 'C', 'D' };
for (int i = 0; i < values.length; i++) { for (int i = 0; i < values.length; i++) {
@ -121,7 +121,7 @@ public class TestHBaseFsckEncryption {
TEST_UTIL.getHBaseAdmin().flush(htd.getTableName()); TEST_UTIL.getHBaseAdmin().flush(htd.getTableName());
// Verify we have encrypted store files on disk // Verify we have encrypted store files on disk
final List<Path> paths = findStorefilePaths(htd.getName()); final List<Path> paths = findStorefilePaths(htd.getTableName());
assertTrue(paths.size() > 0); assertTrue(paths.size() > 0);
for (Path path: paths) { for (Path path: paths) {
assertTrue("Store file " + path + " has incorrect key", assertTrue("Store file " + path + " has incorrect key",
@ -138,7 +138,7 @@ public class TestHBaseFsckEncryption {
assertEquals(hfcc.getMissing().size(), 0); assertEquals(hfcc.getMissing().size(), 0);
} }
private List<Path> findStorefilePaths(byte[] tableName) throws Exception { private List<Path> findStorefilePaths(TableName tableName) throws Exception {
List<Path> paths = new ArrayList<Path>(); List<Path> paths = new ArrayList<Path>();
for (HRegion region: for (HRegion region:
TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) {

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.MiscTests;
@ -92,8 +93,9 @@ public class TestRegionSplitter {
expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY); expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
// Do table creation/pre-splitting and verification of region boundaries // Do table creation/pre-splitting and verification of region boundaries
preSplitTableAndVerify(expectedBounds, preSplitTableAndVerify(expectedBounds,
HexStringSplit.class.getSimpleName(), "NewHexPresplitTable"); HexStringSplit.class.getSimpleName(),
TableName.valueOf("NewHexPresplitTable"));
} }
/** /**
@ -122,7 +124,7 @@ public class TestRegionSplitter {
// Do table creation/pre-splitting and verification of region boundaries // Do table creation/pre-splitting and verification of region boundaries
preSplitTableAndVerify(expectedBounds, UniformSplit.class.getSimpleName(), preSplitTableAndVerify(expectedBounds, UniformSplit.class.getSimpleName(),
"NewUniformPresplitTable"); TableName.valueOf("NewUniformPresplitTable"));
} }
/** /**
@ -273,12 +275,12 @@ public class TestRegionSplitter {
* @throws Various junit assertions * @throws Various junit assertions
*/ */
private void preSplitTableAndVerify(List<byte[]> expectedBounds, private void preSplitTableAndVerify(List<byte[]> expectedBounds,
String splitClass, String tableName) throws Exception { String splitClass, TableName tableName) throws Exception {
final int numRegions = expectedBounds.size()-1; final int numRegions = expectedBounds.size()-1;
final Configuration conf = UTIL.getConfiguration(); final Configuration conf = UTIL.getConfiguration();
conf.setInt("split.count", numRegions); conf.setInt("split.count", numRegions);
SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass); SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass);
RegionSplitter.createPresplitTable(tableName, splitAlgo, RegionSplitter.createPresplitTable(tableName.getNameAsString(), splitAlgo,
new String[] {CF_NAME}, conf); new String[] {CF_NAME}, conf);
verifyBounds(expectedBounds, tableName); verifyBounds(expectedBounds, tableName);
} }
@ -287,26 +289,28 @@ public class TestRegionSplitter {
public void noopRollingSplit() throws Exception { public void noopRollingSplit() throws Exception {
final List<byte[]> expectedBounds = new ArrayList<byte[]>(); final List<byte[]> expectedBounds = new ArrayList<byte[]>();
expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY); 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<byte[]> expectedBounds) throws Exception { List<byte[]> expectedBounds) throws Exception {
final Configuration conf = UTIL.getConfiguration(); final Configuration conf = UTIL.getConfiguration();
// Set this larger than the number of splits so RegionSplitter won't block // Set this larger than the number of splits so RegionSplitter won't block
conf.setInt("split.outstanding", 5); conf.setInt("split.outstanding", 5);
SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass); SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass);
RegionSplitter.rollingSplit(tableName, splitAlgo, conf); RegionSplitter.rollingSplit(tableName.getNameAsString(), splitAlgo, conf);
verifyBounds(expectedBounds, tableName); verifyBounds(expectedBounds, tableName);
} }
private void verifyBounds(List<byte[]> expectedBounds, String tableName) private void verifyBounds(List<byte[]> expectedBounds, TableName tableName)
throws Exception { throws Exception {
// Get region boundaries from the cluster and verify their endpoints // Get region boundaries from the cluster and verify their endpoints
final Configuration conf = UTIL.getConfiguration(); final Configuration conf = UTIL.getConfiguration();
final int numRegions = expectedBounds.size()-1; final int numRegions = expectedBounds.size()-1;
final HTable hTable = new HTable(conf, tableName.getBytes()); final HTable hTable = new HTable(conf, tableName);
final Map<HRegionInfo, ServerName> regionInfoMap = hTable.getRegionLocations(); final Map<HRegionInfo, ServerName> regionInfoMap = hTable.getRegionLocations();
assertEquals(numRegions, regionInfoMap.size()); assertEquals(numRegions, regionInfoMap.size());
for (Map.Entry<HRegionInfo, ServerName> entry: regionInfoMap.entrySet()) { for (Map.Entry<HRegionInfo, ServerName> entry: regionInfoMap.entrySet()) {