Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A44F810489 for ; Thu, 8 Aug 2013 04:22:00 +0000 (UTC) Received: (qmail 25776 invoked by uid 500); 8 Aug 2013 04:22:00 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 25606 invoked by uid 500); 8 Aug 2013 04:22:00 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 25477 invoked by uid 99); 8 Aug 2013 04:21:59 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 08 Aug 2013 04:21:59 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 08 Aug 2013 04:21:29 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id C33F72388C4E; Thu, 8 Aug 2013 04:20:16 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1511577 [22/23] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/ap... Date: Thu, 08 Aug 2013 04:19:56 -0000 To: commits@hbase.apache.org From: stack@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20130808042016.C33F72388C4E@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java Thu Aug 8 04:19:49 2013 @@ -35,14 +35,14 @@ import org.apache.commons.logging.impl.L import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.client.HBaseAdmin; @@ -52,13 +52,11 @@ import org.apache.hadoop.hbase.master.HM import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; -import org.apache.hadoop.hbase.util.MD5Hash; import org.apache.log4j.Level; import org.junit.After; import org.junit.AfterClass; @@ -83,7 +81,8 @@ public class TestFlushSnapshotFromClient private static final String STRING_TABLE_NAME = "test"; private static final byte[] TEST_FAM = Bytes.toBytes("fam"); private static final byte[] TEST_QUAL = Bytes.toBytes("q"); - private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME); + private static final TableName TABLE_NAME = + TableName.valueOf(STRING_TABLE_NAME); private final int DEFAULT_NUM_ROWS = 100; /** @@ -163,7 +162,7 @@ public class TestFlushSnapshotFromClient LOG.debug("FS state before snapshot:"); FSUtils.logFileSystemState(UTIL.getTestFileSystem(), - FSUtils.getRootDir(UTIL.getConfiguration()), LOG); + FSUtils.getRootDir(UTIL.getConfiguration()), LOG); // take a snapshot of the enabled table String snapshotString = "offlineTableSnapshot"; @@ -180,10 +179,10 @@ public class TestFlushSnapshotFromClient Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir(); LOG.debug("FS state after snapshot:"); FSUtils.logFileSystemState(UTIL.getTestFileSystem(), - FSUtils.getRootDir(UTIL.getConfiguration()), LOG); + FSUtils.getRootDir(UTIL.getConfiguration()), LOG); SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir, - admin, fs, false, new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), snapshotServers); + admin, fs, false, new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), snapshotServers); } @Test (timeout=300000) @@ -219,7 +218,9 @@ public class TestFlushSnapshotFromClient public void testAsyncFlushSnapshot() throws Exception { HBaseAdmin admin = UTIL.getHBaseAdmin(); SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("asyncSnapshot") - .setTable(STRING_TABLE_NAME).setType(SnapshotDescription.Type.FLUSH).build(); + .setTable(TABLE_NAME.getNameAsString()) + .setType(SnapshotDescription.Type.FLUSH) + .build(); // take the snapshot async admin.takeSnapshotAsync(snapshot); @@ -250,7 +251,7 @@ public class TestFlushSnapshotFromClient // Clone the table String cloneBeforeMergeName = "cloneBeforeMerge"; admin.cloneSnapshot(snapshotBeforeMergeName, cloneBeforeMergeName); - SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneBeforeMergeName)); + SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneBeforeMergeName)); // Merge two regions List regions = admin.getTableRegions(TABLE_NAME); @@ -274,11 +275,11 @@ public class TestFlushSnapshotFromClient // Clone the table String cloneAfterMergeName = "cloneAfterMerge"; admin.cloneSnapshot(snapshotBeforeMergeName, cloneAfterMergeName); - SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneAfterMergeName)); + SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneAfterMergeName)); SnapshotTestingUtils.verifyRowCount(UTIL, TABLE_NAME, numRows); - SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneBeforeMergeName), numRows); - SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneAfterMergeName), numRows); + SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneBeforeMergeName), numRows); + SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneAfterMergeName), numRows); // test that we can delete the snapshot UTIL.deleteTable(cloneAfterMergeName); @@ -319,10 +320,10 @@ public class TestFlushSnapshotFromClient // Clone the table String cloneName = "cloneMerge"; admin.cloneSnapshot(snapshotName, cloneName); - SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneName)); + SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneName)); SnapshotTestingUtils.verifyRowCount(UTIL, TABLE_NAME, numRows); - SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneName), numRows); + SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneName), numRows); // test that we can delete the snapshot UTIL.deleteTable(cloneName); @@ -347,7 +348,7 @@ public class TestFlushSnapshotFromClient // make sure we only have 1 matching snapshot List snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, - snapshotName, STRING_TABLE_NAME); + snapshotName, TABLE_NAME); // check the directory structure FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem(); @@ -360,9 +361,9 @@ public class TestFlushSnapshotFromClient // check the table info HTableDescriptor desc = FSTableDescriptors.getTableDescriptorFromFs(fs, - rootDir,STRING_TABLE_NAME); + rootDir, TABLE_NAME); HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptorFromFs(fs, - SnapshotDescriptionUtils.getSnapshotsDir(rootDir), snapshotName); + new Path(SnapshotDescriptionUtils.getSnapshotsDir(rootDir), snapshotName)); assertEquals(desc, snapshotDesc); // check the region snapshot for all the regions @@ -391,7 +392,8 @@ public class TestFlushSnapshotFromClient @Test(timeout=300000) public void testConcurrentSnapshottingAttempts() throws IOException, InterruptedException { final String STRING_TABLE2_NAME = STRING_TABLE_NAME + "2"; - final byte[] TABLE2_NAME = Bytes.toBytes(STRING_TABLE2_NAME); + final TableName TABLE2_NAME = + TableName.valueOf(STRING_TABLE2_NAME); int ssNum = 20; HBaseAdmin admin = UTIL.getHBaseAdmin(); @@ -431,7 +433,7 @@ public class TestFlushSnapshotFromClient SnapshotDescription[] descs = new SnapshotDescription[ssNum]; for (int i = 0; i < ssNum; i++) { SnapshotDescription.Builder builder = SnapshotDescription.newBuilder(); - builder.setTable((i % 2) == 0 ? STRING_TABLE_NAME : STRING_TABLE2_NAME); + builder.setTable(((i % 2) == 0 ? TABLE_NAME : TABLE2_NAME).getNameAsString()); builder.setName("ss"+i); builder.setType(SnapshotDescription.Type.FLUSH); descs[i] = builder.build(); @@ -477,9 +479,9 @@ public class TestFlushSnapshotFromClient int t1SnapshotsCount = 0; int t2SnapshotsCount = 0; for (SnapshotDescription ss : taken) { - if (ss.getTable().equals(STRING_TABLE_NAME)) { + if (TableName.valueOf(ss.getTable()).equals(TABLE_NAME)) { t1SnapshotsCount++; - } else if (ss.getTable().equals(STRING_TABLE2_NAME)) { + } else if (TableName.valueOf(ss.getTable()).equals(TABLE2_NAME)) { t2SnapshotsCount++; } } Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestReferenceRegionHFilesTask.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestReferenceRegionHFilesTask.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestReferenceRegionHFilesTask.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestReferenceRegionHFilesTask.java Thu Aug 8 04:19:49 2013 @@ -29,9 +29,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.Test; import org.junit.experimental.categories.Category; Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java Thu Aug 8 04:19:49 2013 @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HColumnDe import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.LargeTests; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; @@ -71,7 +72,7 @@ public class TestRestoreFlushSnapshotFro private byte[] snapshotName2; private int snapshot0Rows; private int snapshot1Rows; - private byte[] tableName; + private TableName tableName; private HBaseAdmin admin; @BeforeClass @@ -106,7 +107,7 @@ public class TestRestoreFlushSnapshotFro this.admin = UTIL.getHBaseAdmin(); long tid = System.currentTimeMillis(); - tableName = Bytes.toBytes("testtb-" + tid); + tableName = TableName.valueOf("testtb-" + tid); snapshotName0 = Bytes.toBytes("snaptb0-" + tid); snapshotName1 = Bytes.toBytes("snaptb1-" + tid); snapshotName2 = Bytes.toBytes("snaptb2-" + tid); @@ -121,7 +122,7 @@ public class TestRestoreFlushSnapshotFro logFSTree(); // take a snapshot - admin.snapshot(Bytes.toString(snapshotName0), Bytes.toString(tableName), + admin.snapshot(Bytes.toString(snapshotName0), tableName, SnapshotDescription.Type.FLUSH); LOG.info("=== after snapshot with 500 rows"); @@ -134,7 +135,7 @@ public class TestRestoreFlushSnapshotFro logFSTree(); // take a snapshot of the updated table - admin.snapshot(Bytes.toString(snapshotName1), Bytes.toString(tableName), + admin.snapshot(Bytes.toString(snapshotName1), tableName, SnapshotDescription.Type.FLUSH); LOG.info("=== after snapshot with 1000 rows"); logFSTree(); @@ -183,12 +184,12 @@ public class TestRestoreFlushSnapshotFro @Test public void testCloneSnapshot() throws IOException, InterruptedException { - byte[] clonedTableName = Bytes.toBytes("clonedtb-" + System.currentTimeMillis()); + TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis()); testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows); testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows); } - private void testCloneSnapshot(final byte[] tableName, final byte[] snapshotName, + private void testCloneSnapshot(final TableName tableName, final byte[] snapshotName, int snapshotRows) throws IOException, InterruptedException { // create a new table from snapshot admin.cloneSnapshot(snapshotName, tableName); @@ -199,10 +200,10 @@ public class TestRestoreFlushSnapshotFro @Test public void testRestoreSnapshotOfCloned() throws IOException, InterruptedException { - byte[] clonedTableName = Bytes.toBytes("clonedtb-" + System.currentTimeMillis()); + TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis()); admin.cloneSnapshot(snapshotName0, clonedTableName); SnapshotTestingUtils.verifyRowCount(UTIL, clonedTableName, snapshot0Rows); - admin.snapshot(Bytes.toString(snapshotName2), Bytes.toString(clonedTableName), SnapshotDescription.Type.FLUSH); + admin.snapshot(Bytes.toString(snapshotName2), clonedTableName, SnapshotDescription.Type.FLUSH); UTIL.deleteTable(clonedTableName); admin.cloneSnapshot(snapshotName2, clonedTableName); Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java Thu Aug 8 04:19:49 2013 @@ -38,10 +38,12 @@ import org.apache.hadoop.hbase.HConstant import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; @@ -92,23 +94,23 @@ public class TestRestoreSnapshotHelper { // Test clone a snapshot HTableDescriptor htdClone = createTableDescriptor("testtb-clone"); - testRestore(snapshotDir, htd.getNameAsString(), htdClone); + testRestore(snapshotDir, htd.getTableName().getNameAsString(), htdClone); verifyRestore(rootDir, htd, htdClone); // Test clone a clone ("link to link") - Path cloneDir = HTableDescriptor.getTableDir(rootDir, htdClone.getName()); + Path cloneDir = FSUtils.getTableDir(rootDir, htdClone.getTableName()); HTableDescriptor htdClone2 = createTableDescriptor("testtb-clone2"); - testRestore(cloneDir, htdClone.getNameAsString(), htdClone2); + testRestore(cloneDir, htdClone.getTableName().getNameAsString(), htdClone2); verifyRestore(rootDir, htd, htdClone2); } private void verifyRestore(final Path rootDir, final HTableDescriptor sourceHtd, final HTableDescriptor htdClone) throws IOException { - String[] files = getHFiles(HTableDescriptor.getTableDir(rootDir, htdClone.getName())); + String[] files = getHFiles(FSUtils.getTableDir(rootDir, htdClone.getTableName())); assertEquals(2, files.length); assertTrue(files[0] + " should be a HFileLink", HFileLink.isHFileLink(files[0])); assertTrue(files[1] + " should be a Referene", StoreFileInfo.isReference(files[1])); - assertEquals(sourceHtd.getNameAsString(), HFileLink.getReferencedTableName(files[0])); + assertEquals(sourceHtd.getTableName(), HFileLink.getReferencedTableName(files[0])); assertEquals(TEST_HFILE, HFileLink.getReferencedHFileName(files[0])); Path refPath = getReferredToFile(files[1]); assertTrue(refPath.getName() + " should be a HFileLink", HFileLink.isHFileLink(refPath.getName())); @@ -123,14 +125,14 @@ public class TestRestoreSnapshotHelper { */ public void testRestore(final Path snapshotDir, final String sourceTableName, final HTableDescriptor htdClone) throws IOException { - LOG.debug("pre-restore table=" + htdClone.getNameAsString() + " snapshot=" + snapshotDir); + LOG.debug("pre-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir); FSUtils.logFileSystemState(fs, rootDir, LOG); new FSTableDescriptors(conf).createTableDescriptor(htdClone); RestoreSnapshotHelper helper = getRestoreHelper(rootDir, snapshotDir, sourceTableName, htdClone); helper.restoreHdfsRegions(); - LOG.debug("post-restore table=" + htdClone.getNameAsString() + " snapshot=" + snapshotDir); + LOG.debug("post-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir); FSUtils.logFileSystemState(fs, rootDir, LOG); } @@ -145,37 +147,39 @@ public class TestRestoreSnapshotHelper { MonitoredTask status = Mockito.mock(MonitoredTask.class); SnapshotDescription sd = SnapshotDescription.newBuilder() - .setName("snapshot").setTable(sourceTableName).build(); + .setName("snapshot") + .setTable(sourceTableName) + .build(); return new RestoreSnapshotHelper(conf, fs, sd, snapshotDir, - htdClone, HTableDescriptor.getTableDir(rootDir, htdClone.getName()), monitor, status); + htdClone, rootDir, monitor, status); } private void createSnapshot(final Path rootDir, final Path snapshotDir, final HTableDescriptor htd) throws IOException { // First region, simple with one plain hfile. - HRegionInfo hri = new HRegionInfo(htd.getName()); + HRegionInfo hri = new HRegionInfo(htd.getTableName()); HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf, - fs, new Path(archiveDir, hri.getTableNameAsString()), hri); + fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri); Path storeFile = new Path(rootDir, TEST_HFILE); fs.createNewFile(storeFile); r0fs.commitStoreFile(TEST_FAMILY, storeFile); // Second region, used to test the split case. // This region contains a reference to the hfile in the first region. - hri = new HRegionInfo(htd.getName()); + hri = new HRegionInfo(htd.getTableName()); HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf, - fs, new Path(archiveDir, hri.getTableNameAsString()), hri); + fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri); storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName()); fs.createNewFile(storeFile); r1fs.commitStoreFile(TEST_FAMILY, storeFile); - Path tableDir = HTableDescriptor.getTableDir(archiveDir, htd.getName()); + Path tableDir = FSUtils.getTableDir(archiveDir, htd.getTableName()); FileUtil.copy(fs, tableDir, fs, snapshotDir, false, conf); } private HTableDescriptor createTableDescriptor(final String tableName) { - HTableDescriptor htd = new HTableDescriptor(tableName); + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); return htd; } Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java Thu Aug 8 04:19:49 2013 @@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configurat import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -41,7 +42,6 @@ import org.apache.hadoop.hbase.regionser import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; -import org.apache.hadoop.hbase.snapshot.SnapshotLogSplitter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.*; @@ -84,7 +84,7 @@ public class TestSnapshotLogSplitter { @Test public void testSplitLogsOnDifferentTable() throws IOException { - byte[] tableName = getTableName(1); + TableName tableName = getTableName(1); Map regionsMap = new TreeMap(Bytes.BYTES_COMPARATOR); for (int j = 0; j < 10; ++j) { byte[] regionName = getRegionName(tableName, j); @@ -97,9 +97,9 @@ public class TestSnapshotLogSplitter { /* * Split and verify test logs for the specified table */ - private void splitTestLogs(final byte[] tableName, final Map regionsMap) - throws IOException { - Path tableDir = new Path(TEST_UTIL.getDataTestDir(), Bytes.toString(tableName)); + private void splitTestLogs(final TableName tableName, + final Map regionsMap) throws IOException { + Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDataTestDir(), tableName); SnapshotLogSplitter logSplitter = new SnapshotLogSplitter(conf, fs, tableDir, tableName, regionsMap); try { @@ -113,10 +113,10 @@ public class TestSnapshotLogSplitter { /* * Verify that every logs in the table directory has just the specified table and regions. */ - private void verifyRecoverEdits(final Path tableDir, final byte[] tableName, + private void verifyRecoverEdits(final Path tableDir, final TableName tableName, final Map regionsMap) throws IOException { for (FileStatus regionStatus: FSUtils.listStatus(fs, tableDir)) { - assertTrue(regionStatus.getPath().getName().startsWith(Bytes.toString(tableName))); + assertTrue(regionStatus.getPath().getName().startsWith(tableName.getNameAsString())); Path regionEdits = HLogUtil.getRegionDirRecoveredEditsDir(regionStatus.getPath()); byte[] regionName = Bytes.toBytes(regionStatus.getPath().getName()); assertFalse(regionsMap.containsKey(regionName)); @@ -126,7 +126,7 @@ public class TestSnapshotLogSplitter { HLog.Entry entry; while ((entry = reader.next()) != null) { HLogKey key = entry.getKey(); - assertArrayEquals(tableName, key.getTablename()); + assertEquals(tableName, key.getTablename()); assertArrayEquals(regionName, key.getEncodedRegionName()); } } finally { @@ -147,7 +147,7 @@ public class TestSnapshotLogSplitter { HLog.Writer writer = HLogFactory.createWriter(fs, logFile, conf); try { for (int i = 0; i < 7; ++i) { - byte[] tableName = getTableName(i); + TableName tableName = getTableName(i); for (int j = 0; j < 10; ++j) { byte[] regionName = getRegionName(tableName, j); for (int k = 0; k < 50; ++k) { @@ -165,15 +165,15 @@ public class TestSnapshotLogSplitter { } } - private byte[] getTableName(int tableId) { - return Bytes.toBytes("testtb-" + tableId); + private TableName getTableName(int tableId) { + return TableName.valueOf("testtb-" + tableId); } - private byte[] getRegionName(final byte[] tableName, int regionId) { - return Bytes.toBytes(Bytes.toString(tableName) + "-region-" + regionId); + private byte[] getRegionName(final TableName tableName, int regionId) { + return Bytes.toBytes(tableName + "-region-" + regionId); } - private byte[] getNewRegionName(final byte[] tableName, int regionId) { - return Bytes.toBytes(Bytes.toString(tableName) + "-new-region-" + regionId); + private byte[] getNewRegionName(final TableName tableName, int regionId) { + return Bytes.toBytes(tableName + "-new-region-" + regionId); } } Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java Thu Aug 8 04:19:49 2013 @@ -18,15 +18,15 @@ package org.apache.hadoop.hbase.snapshot; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.snapshot.SnapshotTask; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java Thu Aug 8 04:19:49 2013 @@ -32,6 +32,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -51,11 +52,6 @@ import org.apache.hadoop.hbase.thrift.ge import org.apache.hadoop.hbase.thrift.generated.TRowResult; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.metrics.ContextFactory; -import org.apache.hadoop.metrics.MetricsContext; -import org.apache.hadoop.metrics.MetricsUtil; -import org.apache.hadoop.metrics.spi.NoEmitMetricsContext; -import org.apache.hadoop.metrics.spi.OutputRecord; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -580,7 +576,8 @@ public class TestThriftServer { try { handler.mutateRow(tableAname, rowAname, getMutations(), null); byte[] searchRow = HRegionInfo.createRegionName( - tableAname.array(), rowAname.array(), HConstants.NINES, false); + TableName.valueOf(tableAname.array()), rowAname.array(), + HConstants.NINES, false); TRegionInfo regionInfo = handler.getRegionInfo(ByteBuffer.wrap(searchRow)); assertTrue(Bytes.toStringBinary(regionInfo.getName()).startsWith( Bytes.toStringBinary(tableAname))); Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java Thu Aug 8 04:19:49 2013 @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTest import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.Put; @@ -130,7 +131,7 @@ public class TestThriftHBaseServiceHandl public static void beforeClass() throws Exception { UTIL.startMiniCluster(); HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration()); - HTableDescriptor tableDescriptor = new HTableDescriptor(tableAname); + HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(tableAname)); for (HColumnDescriptor family : families) { tableDescriptor.addFamily(family); } Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java Thu Aug 8 04:19:49 2013 @@ -32,11 +32,9 @@ import org.apache.hadoop.conf.Configurat import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.HStore; /** * Test helper for testing archiving of HFiles @@ -212,9 +210,11 @@ public class HFileArchiveTestingUtil { * @param region region that is being archived * @return {@link Path} to the archive directory for the given region */ - public static Path getRegionArchiveDir(Configuration conf, HRegion region) { - return HFileArchiveUtil.getRegionArchiveDir(region.getRegionFileSystem().getTableDir(), - region.getRegionFileSystem().getRegionDir()); + public static Path getRegionArchiveDir(Configuration conf, HRegion region) throws IOException { + return HFileArchiveUtil.getRegionArchiveDir( + FSUtils.getRootDir(conf), + region.getTableDesc().getTableName(), + region.getRegionInfo().getEncodedName()); } /** Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java Thu Aug 8 04:19:49 2013 @@ -27,6 +27,7 @@ import org.apache.commons.cli.CommandLin import org.apache.commons.cli.ParseException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -50,7 +51,7 @@ public class LoadTestTool extends Abstra private static final Log LOG = LogFactory.getLog(LoadTestTool.class); /** Table name for the test */ - protected byte[] tableName; + private TableName tableName; /** Table name to use of not overridden on the command line */ protected static final String DEFAULT_TABLE_NAME = "cluster_test"; @@ -169,11 +170,11 @@ public class LoadTestTool extends Abstra * Apply column family options such as Bloom filters, compression, and data * block encoding. */ - protected void applyColumnFamilyOptions(byte[] tableName, + protected void applyColumnFamilyOptions(TableName tableName, byte[][] columnFamilies) throws IOException { HBaseAdmin admin = new HBaseAdmin(conf); HTableDescriptor tableDesc = admin.getTableDescriptor(tableName); - LOG.info("Disabling table " + Bytes.toString(tableName)); + LOG.info("Disabling table " + tableName); admin.disableTable(tableName); for (byte[] cf : columnFamilies) { HColumnDescriptor columnDesc = tableDesc.getFamily(cf); @@ -200,7 +201,7 @@ public class LoadTestTool extends Abstra admin.modifyColumn(tableName, columnDesc); } } - LOG.info("Enabling table " + Bytes.toString(tableName)); + LOG.info("Enabling table " + tableName); admin.enableTable(tableName); } @@ -244,7 +245,7 @@ public class LoadTestTool extends Abstra protected void processOptions(CommandLine cmd) { this.cmd = cmd; - tableName = Bytes.toBytes(cmd.getOptionValue(OPT_TABLE_NAME, + tableName = TableName.valueOf(cmd.getOptionValue(OPT_TABLE_NAME, DEFAULT_TABLE_NAME)); isWrite = cmd.hasOption(OPT_WRITE); Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java Thu Aug 8 04:19:49 2013 @@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentSk import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.catalog.CatalogTracker; @@ -79,7 +80,7 @@ public class MockRegionServerServices im return this.regions.get(encodedRegionName); } - public List getOnlineRegions(byte[] tableName) throws IOException { + public List getOnlineRegions(TableName tableName) throws IOException { return null; } Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java Thu Aug 8 04:19:49 2013 @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.util; import java.io.IOException; import java.util.Collection; import java.util.Map; -import java.util.NavigableMap; import java.util.Random; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; @@ -28,6 +27,7 @@ import java.util.concurrent.atomic.Atomi import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator; @@ -40,7 +40,7 @@ import org.apache.hadoop.util.StringUtil public abstract class MultiThreadedAction { private static final Log LOG = LogFactory.getLog(MultiThreadedAction.class); - protected final byte[] tableName; + protected final TableName tableName; protected final Configuration conf; protected int numThreads = 1; @@ -129,8 +129,9 @@ public abstract class MultiThreadedActio public static final int REPORTING_INTERVAL_MS = 5000; - public MultiThreadedAction(LoadTestDataGenerator dataGen, Configuration conf, byte[] tableName, - String actionLetter) { + public MultiThreadedAction(LoadTestDataGenerator dataGen, Configuration conf, + TableName tableName, + String actionLetter) { this.conf = conf; this.dataGenerator = dataGen; this.tableName = tableName; Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java Thu Aug 8 04:19:49 2013 @@ -18,8 +18,6 @@ package org.apache.hadoop.hbase.util; import java.io.IOException; import java.util.HashSet; -import java.util.List; -import java.util.Map; import java.util.Random; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; @@ -27,13 +25,12 @@ import java.util.concurrent.atomic.Atomi import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; -import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator; /** Creates multiple threads that read and verify previously written data */ public class MultiThreadedReader extends MultiThreadedAction @@ -75,7 +72,7 @@ public class MultiThreadedReader extends private int keyWindow = DEFAULT_KEY_WINDOW; public MultiThreadedReader(LoadTestDataGenerator dataGen, Configuration conf, - byte[] tableName, double verifyPercent) { + TableName tableName, double verifyPercent) { super(dataGen, conf, tableName, "R"); this.verifyPercent = verifyPercent; } Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java Thu Aug 8 04:19:49 2013 @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; import java.util.HashSet; -import java.util.Map; import java.util.PriorityQueue; import java.util.Queue; import java.util.Set; @@ -35,13 +34,12 @@ import java.util.concurrent.atomic.Atomi import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; -import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator; -import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; /** Creates multiple threads that write key/values into the */ @@ -85,7 +83,7 @@ public class MultiThreadedWriter extends private boolean trackInsertedKeys; public MultiThreadedWriter(LoadTestDataGenerator dataGen, Configuration conf, - byte[] tableName) { + TableName tableName) { super(dataGen, conf, tableName, "W"); } Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java Thu Aug 8 04:19:49 2013 @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HBaseTest import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -171,7 +172,7 @@ public class ProcessBasedLocalHBaseClust int attemptsLeft = 10; while (attemptsLeft-- > 0) { try { - new HTable(conf, HConstants.META_TABLE_NAME); + new HTable(conf, TableName.META_TABLE_NAME); } catch (Exception e) { LOG.info("Waiting for HBase to startup. Retries left: " + attemptsLeft, e); Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java Thu Aug 8 04:19:49 2013 @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.commons.cli.CommandLine; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.HTable; @@ -44,7 +45,8 @@ public class RestartMetaTest extends Abs private static final int DEFAULT_NUM_RS = 2; /** Table name for the test */ - private static byte[] TABLE_NAME = Bytes.toBytes("load_test"); + private static TableName TABLE_NAME = + TableName.valueOf("load_test"); /** The number of seconds to sleep after loading the data */ private static final int SLEEP_SEC_AFTER_DATA_LOAD = 5; @@ -118,7 +120,7 @@ public class RestartMetaTest extends Abs LOG.debug("Trying to scan meta"); - HTable metaTable = new HTable(conf, HConstants.META_TABLE_NAME); + HTable metaTable = new HTable(conf, TableName.META_TABLE_NAME); ResultScanner scanner = metaTable.getScanner(new Scan()); Result result; while ((result = scanner.next()) != null) { Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java Thu Aug 8 04:19:49 2013 @@ -33,6 +33,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -57,7 +58,6 @@ import org.apache.hadoop.hbase.regionser import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -101,7 +101,8 @@ public class TestCoprocessorScanPolicy { @Test public void testBaseCases() throws Exception { - byte[] tableName = Bytes.toBytes("baseCases"); + TableName tableName = + TableName.valueOf("baseCases"); if (TEST_UTIL.getHBaseAdmin().tableExists(tableName)) { TEST_UTIL.deleteTable(tableName); } @@ -109,7 +110,7 @@ public class TestCoprocessorScanPolicy { // set the version override to 2 Put p = new Put(R); p.setAttribute("versions", new byte[]{}); - p.add(F, tableName, Bytes.toBytes(2)); + p.add(F, tableName.getName(), Bytes.toBytes(2)); t.put(p); long now = EnvironmentEdgeManager.currentTimeMillis(); @@ -150,7 +151,8 @@ public class TestCoprocessorScanPolicy { @Test public void testTTL() throws Exception { - byte[] tableName = Bytes.toBytes("testTTL"); + TableName tableName = + TableName.valueOf("testTTL"); if (TEST_UTIL.getHBaseAdmin().tableExists(tableName)) { TEST_UTIL.deleteTable(tableName); } @@ -170,7 +172,7 @@ public class TestCoprocessorScanPolicy { // Set the TTL override to 3s Put p = new Put(R); p.setAttribute("ttl", new byte[]{}); - p.add(F, tableName, Bytes.toBytes(3000L)); + p.add(F, tableName.getName(), Bytes.toBytes(3000L)); t.put(p); p = new Put(R); @@ -209,8 +211,10 @@ public class TestCoprocessorScanPolicy { } public static class ScanObserver extends BaseRegionObserver { - private Map ttls = new HashMap(); - private Map versions = new HashMap(); + private Map ttls = + new HashMap(); + private Map versions = + new HashMap(); // lame way to communicate with the coprocessor, // since it is loaded by a different class loader @@ -220,12 +224,12 @@ public class TestCoprocessorScanPolicy { if (put.getAttribute("ttl") != null) { Cell cell = put.getFamilyMap().values().iterator().next().get(0); KeyValue kv = KeyValueUtil.ensureKeyValue(cell); - ttls.put(Bytes.toString(kv.getQualifier()), Bytes.toLong(kv.getValue())); + ttls.put(TableName.valueOf(kv.getQualifier()), Bytes.toLong(kv.getValue())); c.bypass(); } else if (put.getAttribute("versions") != null) { Cell cell = put.getFamilyMap().values().iterator().next().get(0); KeyValue kv = KeyValueUtil.ensureKeyValue(cell); - versions.put(Bytes.toString(kv.getQualifier()), Bytes.toInt(kv.getValue())); + versions.put(TableName.valueOf(kv.getQualifier()), Bytes.toInt(kv.getValue())); c.bypass(); } } Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java?rev=1511577&r1=1511576&r2=1511577&view=diff ============================================================================== --- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java (original) +++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java Thu Aug 8 04:19:49 2013 @@ -34,6 +34,7 @@ import org.apache.commons.logging.LogFac import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -67,7 +68,7 @@ public class TestFSTableDescriptors { @Test public void testCreateAndUpdate() throws IOException { Path testdir = UTIL.getDataTestDir("testCreateAndUpdate"); - HTableDescriptor htd = new HTableDescriptor("testCreate"); + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCreate")); FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); assertTrue(fstd.createTableDescriptor(htd)); @@ -79,7 +80,7 @@ public class TestFSTableDescriptors { } statuses = fs.listStatus(testdir); assertTrue(statuses.length == 1); - Path tmpTableDir = new Path(FSUtils.getTablePath(testdir, htd.getName()), ".tmp"); + Path tmpTableDir = new Path(FSUtils.getTableDir(testdir, htd.getTableName()), ".tmp"); statuses = fs.listStatus(tmpTableDir); assertTrue(statuses.length == 0); } @@ -87,7 +88,8 @@ public class TestFSTableDescriptors { @Test public void testSequenceIdAdvancesOnTableInfo() throws IOException { Path testdir = UTIL.getDataTestDir("testSequenceidAdvancesOnTableInfo"); - HTableDescriptor htd = new HTableDescriptor("testSequenceidAdvancesOnTableInfo"); + HTableDescriptor htd = new HTableDescriptor( + TableName.valueOf("testSequenceidAdvancesOnTableInfo")); FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); Path p0 = fstd.updateTableDescriptor(htd); @@ -143,21 +145,21 @@ public class TestFSTableDescriptors { // Cleanup old tests if any detrius laying around. Path rootdir = new Path(UTIL.getDataTestDir(), name); TableDescriptors htds = new FSTableDescriptors(fs, rootdir); - HTableDescriptor htd = new HTableDescriptor(name); + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); htds.add(htd); - assertNotNull(htds.remove(htd.getNameAsString())); - assertNull(htds.remove(htd.getNameAsString())); + assertNotNull(htds.remove(htd.getTableName())); + assertNull(htds.remove(htd.getTableName())); } @Test public void testReadingHTDFromFS() throws IOException { final String name = "testReadingHTDFromFS"; FileSystem fs = FileSystem.get(UTIL.getConfiguration()); - HTableDescriptor htd = new HTableDescriptor(name); + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); Path rootdir = UTIL.getDataTestDir(name); FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir); fstd.createTableDescriptor(htd); HTableDescriptor htd2 = - FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getNameAsString()); + FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName()); assertTrue(htd.equals(htd2)); } @@ -169,9 +171,9 @@ public class TestFSTableDescriptors { Path rootdir = new Path(UTIL.getDataTestDir(), name); FSTableDescriptors htds = new FSTableDescriptors(fs, rootdir) { @Override - public HTableDescriptor get(byte[] tablename) + public HTableDescriptor get(TableName tablename) throws TableExistsException, FileNotFoundException, IOException { - LOG.info(Bytes.toString(tablename) + ", cachehits=" + this.cachehits); + LOG.info(tablename + ", cachehits=" + this.cachehits); return super.get(tablename); } }; @@ -183,29 +185,29 @@ public class TestFSTableDescriptors { } for (int i = 0; i < count; i++) { - assertTrue(htds.get(Bytes.toBytes(name + i)) != null); + assertTrue(htds.get(TableName.valueOf(name + i)) != null); } for (int i = 0; i < count; i++) { - assertTrue(htds.get(Bytes.toBytes(name + i)) != null); + assertTrue(htds.get(TableName.valueOf(name + i)) != null); } // Update the table infos for (int i = 0; i < count; i++) { - HTableDescriptor htd = new HTableDescriptor(name + i); + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i)); htd.addFamily(new HColumnDescriptor("" + i)); htds.updateTableDescriptor(htd); } // Wait a while so mod time we write is for sure different. Thread.sleep(100); for (int i = 0; i < count; i++) { - assertTrue(htds.get(Bytes.toBytes(name + i)) != null); + assertTrue(htds.get(TableName.valueOf(name + i)) != null); } for (int i = 0; i < count; i++) { - assertTrue(htds.get(Bytes.toBytes(name + i)) != null); + assertTrue(htds.get(TableName.valueOf(name + i)) != null); } assertEquals(count * 4, htds.invocations); assertTrue("expected=" + (count * 2) + ", actual=" + htds.cachehits, htds.cachehits >= (count * 2)); - assertTrue(htds.get(HConstants.ROOT_TABLE_NAME) != null); + assertTrue(htds.get(TableName.ROOT_TABLE_NAME) != null); assertEquals(htds.invocations, count * 4 + 1); assertTrue("expected=" + ((count * 2) + 1) + ", actual=" + htds.cachehits, htds.cachehits >= ((count * 2) + 1)); @@ -219,7 +221,7 @@ public class TestFSTableDescriptors { Path rootdir = new Path(UTIL.getDataTestDir(), name); TableDescriptors htds = new FSTableDescriptors(fs, rootdir); assertNull("There shouldn't be any HTD for this table", - htds.get("NoSuchTable")); + htds.get(TableName.valueOf("NoSuchTable"))); } @Test @@ -229,7 +231,7 @@ public class TestFSTableDescriptors { // Cleanup old tests if any detrius laying around. Path rootdir = new Path(UTIL.getDataTestDir(), name); TableDescriptors htds = new FSTableDescriptors(fs, rootdir); - HTableDescriptor htd = new HTableDescriptor(name); + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); htds.add(htd); htds.add(htd); htds.add(htd); @@ -266,9 +268,9 @@ public class TestFSTableDescriptors { FileSystem fs = FileSystem.get(UTIL.getConfiguration()); try { new FSTableDescriptors(fs, FSUtils.getRootDir(UTIL.getConfiguration())) - .get(HConstants.HFILE_ARCHIVE_DIRECTORY); + .get(TableName.valueOf(HConstants.HFILE_ARCHIVE_DIRECTORY)); fail("Shouldn't be able to read a table descriptor for the archive directory."); - } catch (IOException e) { + } catch (Exception e) { LOG.debug("Correctly got error when reading a table descriptor from the archive directory: " + e.getMessage()); } @@ -277,15 +279,15 @@ public class TestFSTableDescriptors { @Test public void testCreateTableDescriptorUpdatesIfExistsAlready() throws IOException { Path testdir = UTIL.getDataTestDir("testCreateTableDescriptorUpdatesIfThereExistsAlready"); - HTableDescriptor htd = new HTableDescriptor( - "testCreateTableDescriptorUpdatesIfThereExistsAlready"); + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf( + "testCreateTableDescriptorUpdatesIfThereExistsAlready")); FileSystem fs = FileSystem.get(UTIL.getConfiguration()); FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); assertTrue(fstd.createTableDescriptor(htd)); assertFalse(fstd.createTableDescriptor(htd)); htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue")); assertTrue(fstd.createTableDescriptor(htd)); //this will re-create - Path tableDir = fstd.getTableDirectory(htd.getNameAsString()); + Path tableDir = fstd.getTableDir(htd.getTableName()); Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR); FileStatus[] statuses = fs.listStatus(tmpTableDir); assertTrue(statuses.length == 0);