Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 107C6200B91 for ; Thu, 29 Sep 2016 18:08:26 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 0F612160AEA; Thu, 29 Sep 2016 16:08:26 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 276D1160AC1 for ; Thu, 29 Sep 2016 18:08:24 +0200 (CEST) Received: (qmail 94676 invoked by uid 500); 29 Sep 2016 16:08:22 -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 94398 invoked by uid 99); 29 Sep 2016 16:08:22 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 29 Sep 2016 16:08:22 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 5B894E69B1; Thu, 29 Sep 2016 16:08:22 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: busbey@apache.org To: commits@hbase.apache.org Date: Thu, 29 Sep 2016 16:08:25 -0000 Message-Id: <179d72804fc94af9889d1e9408a836e7@git.apache.org> In-Reply-To: <6aae6d3fe26044478b4b55e09487ea6d@git.apache.org> References: <6aae6d3fe26044478b4b55e09487ea6d@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [04/18] hbase git commit: HBASE-14439 break everything HRegion archived-at: Thu, 29 Sep 2016 16:08:26 -0000 http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index 11bd280..38e0a61 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.io.HeapSize; @@ -96,7 +97,7 @@ public class TestAtomicOperation { static final byte [] row = Bytes.toBytes("rowA"); static final byte [] row2 = Bytes.toBytes("rowB"); - @Before + @Before public void setup() { tableName = Bytes.toBytes(name.getMethodName()); } @@ -114,7 +115,7 @@ public class TestAtomicOperation { } ////////////////////////////////////////////////////////////////////////////// // New tests that doesn't spin up a mini cluster but rather just test the - // individual code pieces in the HRegion. + // individual code pieces in the HRegion. ////////////////////////////////////////////////////////////////////////////// /** @@ -334,8 +335,8 @@ public class TestAtomicOperation { Get g = new Get(row); Result result = region.get(g); - assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam1, qual2).length); - assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam2, qual3).length); + assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam1, qual2).length); + assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam2, qual3).length); } catch (IOException e) { e.printStackTrace(); failures.incrementAndGet(); @@ -567,7 +568,7 @@ public class TestAtomicOperation { this.failures = failures; } } - + private static CountDownLatch latch = new CountDownLatch(1); private enum TestStep { INIT, // initial put of 10 to set value of the cell @@ -579,11 +580,11 @@ public class TestAtomicOperation { } private static volatile TestStep testStep = TestStep.INIT; private final String family = "f1"; - + /** * Test written as a verifier for HBASE-7051, CheckAndPut should properly read - * MVCC. - * + * MVCC. + * * Moved into TestAtomicOperation from its original location, TestHBase7051 */ @Test @@ -598,7 +599,7 @@ public class TestAtomicOperation { Put put = new Put(Bytes.toBytes("r1")); put.addColumn(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10")); puts[0] = put; - + region.batchMutate(puts, HConstants.NO_NONCE, HConstants.NO_NONCE); MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf); @@ -660,9 +661,9 @@ public class TestAtomicOperation { public static class MockHRegion extends HRegion { - public MockHRegion(Path tableDir, WAL log, FileSystem fs, Configuration conf, - final HRegionInfo regionInfo, final HTableDescriptor htd, RegionServerServices rsServices) { - super(tableDir, log, fs, conf, regionInfo, htd, rsServices); + public MockHRegion(final RegionFileSystem rfs, final HTableDescriptor htd, + final WAL wal, final RegionServerServices rsServices) { + super(rfs, htd, wal, rsServices); } @Override @@ -672,7 +673,7 @@ public class TestAtomicOperation { } return new WrappedRowLock(super.getRowLockInternal(row, readLock)); } - + public class WrappedRowLock implements RowLock { private final RowLock rowLock; http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java index caf09ad..212a635 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java @@ -244,12 +244,8 @@ public class TestBulkLoad { } // TODO We need a way to do this without creating files - return HRegion.createHRegion(hRegionInfo, - new Path(testFolder.newFolder().toURI()), - conf, - hTableDescriptor, - log); - + return HRegion.createHRegion(conf, new Path(testFolder.newFolder().toURI()), + hTableDescriptor, hRegionInfo, log); } private HRegion testRegionWithFamilies(byte[]... families) throws IOException { http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java index 1513cd0..ef332a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java @@ -22,6 +22,12 @@ import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -31,8 +37,161 @@ import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; -@Category(SmallTests.class) -public class TestDefaultCompactSelection extends TestCompactionPolicy { +import com.google.common.collect.Lists; + +@Category({RegionServerTests.class, SmallTests.class}) +public class TestDefaultCompactSelection extends TestCase { + private final static Log LOG = LogFactory.getLog(TestDefaultCompactSelection.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + protected Configuration conf; + protected HStore store; + private static final String DIR= + TEST_UTIL.getDataTestDir(TestDefaultCompactSelection.class.getSimpleName()).toString(); + private static Path TEST_FILE; + + protected static final int minFiles = 3; + protected static final int maxFiles = 5; + + protected static final long minSize = 10; + protected static final long maxSize = 2100; + + private WALFactory wals; + private HRegion region; + + @Override + public void setUp() throws Exception { + // setup config values necessary for store + this.conf = TEST_UTIL.getConfiguration(); + this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0); + this.conf.setInt("hbase.hstore.compaction.min", minFiles); + this.conf.setInt("hbase.hstore.compaction.max", maxFiles); + this.conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, minSize); + this.conf.setLong("hbase.hstore.compaction.max.size", maxSize); + this.conf.setFloat("hbase.hstore.compaction.ratio", 1.0F); + // Test depends on this not being set to pass. Default breaks test. TODO: Revisit. + this.conf.unset("hbase.hstore.compaction.min.size"); + + //Setting up a Store + final String id = TestDefaultCompactSelection.class.getName(); + Path basedir = new Path(DIR); + final Path logdir = new Path(basedir, DefaultWALProvider.getWALDirectoryName(id)); + HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family")); + FileSystem fs = FileSystem.get(conf); + + fs.delete(logdir, true); + + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("table"))); + htd.addFamily(hcd); + HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + + final Configuration walConf = new Configuration(conf); + FSUtils.setRootDir(walConf, basedir); + wals = new WALFactory(walConf, null, id); + region = HBaseTestingUtility.createRegionAndWAL(info, basedir, conf, htd); + HBaseTestingUtility.closeRegionAndWAL(region); + + RegionFileSystem rfs = RegionFileSystem.open(conf, fs, basedir, info, false); + region = new HRegion(rfs, htd, + wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), null); + + store = new HStore(region, hcd, conf); + + TEST_FILE = region.getRegionFileSystem().createTempName(); + fs.createNewFile(TEST_FILE); + } + + @After + public void tearDown() throws IOException { + IOException ex = null; + try { + region.close(); + } catch (IOException e) { + LOG.warn("Caught Exception", e); + ex = e; + } + try { + wals.close(); + } catch (IOException e) { + LOG.warn("Caught Exception", e); + ex = e; + } + if (ex != null) { + throw ex; + } + } + + ArrayList toArrayList(long... numbers) { + ArrayList result = new ArrayList(); + for (long i : numbers) { + result.add(i); + } + return result; + } + + List sfCreate(long... sizes) throws IOException { + ArrayList ageInDisk = new ArrayList(); + for (int i = 0; i < sizes.length; i++) { + ageInDisk.add(0L); + } + return sfCreate(toArrayList(sizes), ageInDisk); + } + + List sfCreate(ArrayList sizes, ArrayList ageInDisk) + throws IOException { + return sfCreate(false, sizes, ageInDisk); + } + + List sfCreate(boolean isReference, long... sizes) throws IOException { + ArrayList ageInDisk = new ArrayList(sizes.length); + for (int i = 0; i < sizes.length; i++) { + ageInDisk.add(0L); + } + return sfCreate(isReference, toArrayList(sizes), ageInDisk); + } + + List sfCreate(boolean isReference, ArrayList sizes, ArrayList ageInDisk) + throws IOException { + List ret = Lists.newArrayList(); + for (int i = 0; i < sizes.size(); i++) { + ret.add(new MockStoreFile(TEST_UTIL, TEST_FILE, + sizes.get(i), ageInDisk.get(i), isReference, i)); + } + return ret; + } + + long[] getSizes(List sfList) { + long[] aNums = new long[sfList.size()]; + for (int i = 0; i < sfList.size(); ++i) { + aNums[i] = sfList.get(i).getReader().length(); + } + return aNums; + } + + void compactEquals(List candidates, long... expected) + throws IOException { + compactEquals(candidates, false, false, expected); + } + + void compactEquals(List candidates, boolean forcemajor, long... expected) + throws IOException { + compactEquals(candidates, forcemajor, false, expected); + } + + void compactEquals(List candidates, boolean forcemajor, boolean isOffPeak, + long ... expected) + throws IOException { + store.forceMajor = forcemajor; + //Test Default compactions + CompactionRequest result = ((RatioBasedCompactionPolicy)store.storeEngine.getCompactionPolicy()) + .selectCompaction(candidates, new ArrayList(), false, isOffPeak, forcemajor); + List actual = new ArrayList(result.getFiles()); + if (isOffPeak && !forcemajor) { + assertTrue(result.isOffPeak()); + } + assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual))); + store.forceMajor = false; + } @Test public void testCompactionRatio() throws IOException { http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index e4eff84..3faf8bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -997,7 +997,7 @@ public class TestDefaultMemStore { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("testShouldFlushMeta")); desc.addFamily(new HColumnDescriptor("foo".getBytes())); HRegion r = - HRegion.createHRegion(hri, testDir, conf, desc, + HRegion.createHRegion(conf, testDir, desc, hri, wFactory.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace())); HRegion.addRegionToMETA(meta, r); edge.setCurrentTimeMillis(1234 + 100); http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java index dfc97e9..a324328 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.crypto.aes.AES; import org.apache.hadoop.hbase.io.hfile.HFile; @@ -157,8 +158,10 @@ public class TestHMobStore { final Configuration walConf = new Configuration(conf); FSUtils.setRootDir(walConf, basedir); final WALFactory wals = new WALFactory(walConf, null, methodName); - region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes(), - info.getTable().getNamespace()), fs, conf, info, htd, null); + + RegionFileSystem rfs = RegionFileSystem.open(conf, fs, basedir, info, false); + region = new HRegion(rfs, htd, + wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), null); store = new HMobStore(region, hcd, conf); if(testStore) { init(conf, hcd); http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 79d305f..df0b119 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; @@ -101,6 +102,7 @@ import org.apache.hadoop.hbase.filter.NullComparator; import org.apache.hadoop.hbase.filter.PrefixFilter; import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter; import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; +import org.apache.hadoop.hbase.fs.legacy.LegacyLayout; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.MonitoredTask; @@ -4471,7 +4473,7 @@ public class TestHRegion { info = new HRegionInfo(htd.getTableName(), HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, false); Path path = new Path(dir + "testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization"); - region = HRegion.newHRegion(path, null, fs, CONF, info, htd, null); + region = HRegion.newHRegion(CONF, fs, path, htd, info, null, null); // region initialization throws IOException and set task state to ABORTED. region.initialize(); fail("Region initialization should fail due to IOException"); @@ -4509,10 +4511,10 @@ public class TestHRegion { FileSystem fs = region.getRegionFileSystem().getFileSystem(); HBaseTestingUtility.closeRegionAndWAL(region); - Path regionInfoFile = new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE); + Path regionInfoFile = LegacyLayout.getRegionInfoFile(regionDir); // Verify that the .regioninfo file is present - assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir", + assertTrue(LegacyLayout.REGION_INFO_FILE + " should be present in the region dir", fs.exists(regionInfoFile)); // Try to open the region @@ -4521,12 +4523,12 @@ public class TestHRegion { HBaseTestingUtility.closeRegionAndWAL(region); // Verify that the .regioninfo file is still there - assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir", + assertTrue(LegacyLayout.REGION_INFO_FILE + " should be present in the region dir", fs.exists(regionInfoFile)); // Remove the .regioninfo file and verify is recreated on region open fs.delete(regionInfoFile, true); - assertFalse(HRegionFileSystem.REGION_INFO_FILE + " should be removed from the region dir", + assertFalse(LegacyLayout.REGION_INFO_FILE + " should be removed from the region dir", fs.exists(regionInfoFile)); region = HRegion.openHRegion(rootDir, hri, htd, null, CONF); @@ -4535,8 +4537,8 @@ public class TestHRegion { HBaseTestingUtility.closeRegionAndWAL(region); // Verify that the .regioninfo file is still there - assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir", - fs.exists(new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE))); + assertTrue(LegacyLayout.REGION_INFO_FILE + " should be present in the region dir", + fs.exists(new Path(regionDir, LegacyLayout.REGION_INFO_FILE))); } /** @@ -6097,10 +6099,9 @@ public class TestHRegion { // Helper for test testOpenRegionWrittenToWALForLogReplay static class HRegionWithSeqId extends HRegion { - public HRegionWithSeqId(final Path tableDir, final WAL wal, final FileSystem fs, - final Configuration confParam, final HRegionInfo regionInfo, - final HTableDescriptor htd, final RegionServerServices rsServices) { - super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices); + public HRegionWithSeqId(final RegionFileSystem rfs, final HTableDescriptor htd, + final WAL wal, final RegionServerServices rsServices) { + super(rfs, htd, wal, rsServices); } @Override protected long getNextSequenceId(WAL wal) throws IOException { http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java index 5f792fa..48609b9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java @@ -39,6 +39,8 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.fs.RegionFileSystem; +import org.apache.hadoop.hbase.fs.FSUtilsWithRetries; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FSUtils; @@ -60,25 +62,22 @@ public class TestHRegionFileSystem { // Create a Region HRegionInfo hri = new HRegionInfo(TableName.valueOf("TestTable")); - HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, - FSUtils.getTableDir(rootDir, hri.getTable()), hri); + RegionFileSystem regionFs = RegionFileSystem.open(conf, fs, rootDir, hri, true); // Verify if the region is on disk Path regionDir = regionFs.getRegionDir(); assertTrue("The region folder should be created", fs.exists(regionDir)); // Verify the .regioninfo - HRegionInfo hriVerify = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); + HRegionInfo hriVerify = RegionFileSystem.loadRegionInfoFileContent(fs, regionDir); assertEquals(hri, hriVerify); // Open the region - regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs, - FSUtils.getTableDir(rootDir, hri.getTable()), hri, false); + regionFs = RegionFileSystem.open(conf, fs, rootDir, hri, false); assertEquals(regionDir, regionFs.getRegionDir()); // Delete the region - HRegionFileSystem.deleteRegionFromFileSystem(conf, fs, - FSUtils.getTableDir(rootDir, hri.getTable()), hri); + RegionFileSystem.destroy(conf, fs, rootDir, hri); assertFalse("The region folder should be removed", fs.exists(regionDir)); fs.delete(rootDir, true); @@ -90,24 +89,15 @@ public class TestHRegionFileSystem { FileSystem fs = TEST_UTIL.getTestFileSystem(); Configuration conf = TEST_UTIL.getConfiguration(); - // Create a Region - HRegionInfo hri = new HRegionInfo(TableName.valueOf("TestTable")); - HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, rootDir, hri); - assertTrue(fs.exists(regionFs.getRegionDir())); - - regionFs = new HRegionFileSystem(conf, new MockFileSystemForCreate(), - null, null); - // HRegionFileSystem.createRegionOnFileSystem(conf, new MockFileSystemForCreate(), rootDir, - // hri); + FSUtilsWithRetries regionFs = new FSUtilsWithRetries(conf, new MockFileSystemForCreate()); boolean result = regionFs.createDir(new Path("/foo/bar")); assertTrue("Couldn't create the directory", result); - - regionFs = new HRegionFileSystem(conf, new MockFileSystem(), null, null); + regionFs = new FSUtilsWithRetries(conf, new MockFileSystem()); result = regionFs.rename(new Path("/foo/bar"), new Path("/foo/bar2")); assertTrue("Couldn't rename the directory", result); - regionFs = new HRegionFileSystem(conf, new MockFileSystem(), null, null); + regionFs = new FSUtilsWithRetries(conf, new MockFileSystem()); result = regionFs.deleteDir(new Path("/foo/bar")); assertTrue("Couldn't delete the directory", result); fs.delete(rootDir, true); @@ -217,7 +207,7 @@ public class TestHRegionFileSystem { // Create a Region String familyName = "cf"; HRegionInfo hri = new HRegionInfo(TableName.valueOf("TestTable")); - HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, rootDir, hri); + RegionFileSystem regionFs = RegionFileSystem.open(conf, fs, rootDir, hri, true); // New region, no store files Collection storeFiles = regionFs.getStoreFiles(familyName); http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java index 5fde726..7e0a6f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java @@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.fs.RegionFileSystem; +import org.apache.hadoop.hbase.fs.legacy.LegacyLayout; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo; @@ -77,7 +79,7 @@ public class TestHRegionInfo { long modtime2 = getModTime(r); assertEquals(modtime, modtime2); // Now load the file. - HRegionInfo deserializedHri = HRegionFileSystem.loadRegionInfoFileContent( + HRegionInfo deserializedHri = RegionFileSystem.loadRegionInfoFileContent( r.getRegionFileSystem().getFileSystem(), r.getRegionFileSystem().getRegionDir()); assertTrue(hri.equals(deserializedHri)); HBaseTestingUtility.closeRegionAndWAL(r); @@ -85,7 +87,7 @@ public class TestHRegionInfo { long getModTime(final HRegion r) throws IOException { FileStatus[] statuses = r.getRegionFileSystem().getFileSystem().listStatus( - new Path(r.getRegionFileSystem().getRegionDir(), HRegionFileSystem.REGION_INFO_FILE)); + LegacyLayout.getRegionInfoFile(r.getRegionFileSystem().getRegionDir())); assertTrue(statuses != null && statuses.length == 1); return statuses[0].getModificationTime(); } @@ -296,12 +298,12 @@ public class TestHRegionInfo { String firstPart = descriptiveNameForDisplay.substring(0, descriptiveNameForDisplay.indexOf(new String(HRegionInfo.HIDDEN_START_KEY))); String secondPart = descriptiveNameForDisplay.substring( - descriptiveNameForDisplay.indexOf(new String(HRegionInfo.HIDDEN_START_KEY)) + + descriptiveNameForDisplay.indexOf(new String(HRegionInfo.HIDDEN_START_KEY)) + HRegionInfo.HIDDEN_START_KEY.length); String firstPartOrig = origDesc.substring(0, origDesc.indexOf(Bytes.toStringBinary(startKey))); String secondPartOrig = origDesc.substring( - origDesc.indexOf(Bytes.toStringBinary(startKey)) + + origDesc.indexOf(Bytes.toStringBinary(startKey)) + Bytes.toStringBinary(startKey).length()); assert(firstPart.equals(firstPartOrig)); assert(secondPart.equals(secondPartOrig)); http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java index a3804dd..542fa7a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java @@ -177,7 +177,8 @@ public class TestHRegionReplayEvents { es.startExecutorService( string+"-"+string, 1); when(rss.getExecutorService()).thenReturn(es); - primaryRegion = HRegion.createHRegion(primaryHri, rootDir, CONF, htd, walPrimary); + + primaryRegion = HRegion.createHRegion(CONF, rootDir, htd, primaryHri, walPrimary); primaryRegion.close(); List regions = new ArrayList(); regions.add(primaryRegion); @@ -1671,4 +1672,4 @@ public class TestHRegionReplayEvents { return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly, durability, wal, families); } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java index fc47d7e..014aba7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -100,13 +101,12 @@ public class TestRecoveredEdits { Path hbaseRootDir = TEST_UTIL.getDataTestDir(); FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); Path tableDir = FSUtils.getTableDir(hbaseRootDir, htd.getTableName()); - HRegionFileSystem hrfs = - new HRegionFileSystem(TEST_UTIL.getConfiguration(), fs, tableDir, hri); + RegionFileSystem hrfs = RegionFileSystem.open(TEST_UTIL.getConfiguration(), fs, hbaseRootDir, hri, false); if (fs.exists(hrfs.getRegionDir())) { LOG.info("Region directory already exists. Deleting."); fs.delete(hrfs.getRegionDir(), true); } - HRegion region = HRegion.createHRegion(hri, hbaseRootDir, conf, htd, null); + HRegion region = HRegion.createHRegion(conf, hbaseRootDir, htd, hri, null); assertEquals(encodedRegionName, region.getRegionInfo().getEncodedName()); List storeFiles = region.getStoreFileList(columnFamilyAsByteArray); // There should be no store files. http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java index f88c7dd..326bbef 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.exceptions.MergeRegionException; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; @@ -247,18 +248,11 @@ public class TestRegionMergeTransactionOnCluster { assertTrue(fs.exists(regionAdir)); assertTrue(fs.exists(regionBdir)); - HColumnDescriptor[] columnFamilies = tableDescriptor.getColumnFamilies(); - HRegionFileSystem hrfs = new HRegionFileSystem( - TEST_UTIL.getConfiguration(), fs, tabledir, mergedRegionInfo); - int count = 0; - for(HColumnDescriptor colFamily : columnFamilies) { - count += hrfs.getStoreFiles(colFamily.getName()).size(); - } - ADMIN.compactRegion(mergedRegionInfo.getRegionName()); - // clean up the merged region store files - // wait until merged region have reference file + admin.compactRegion(mergedRegionInfo.getRegionName()); + // wait until merged region doesn't have reference file long timeout = System.currentTimeMillis() + waitTime; - int newcount = 0; + RegionFileSystem hrfs = RegionFileSystem.open( + TEST_UTIL.getConfiguration(), fs, tabledir, mergedRegionInfo, false); while (System.currentTimeMillis() < timeout) { for(HColumnDescriptor colFamily : columnFamilies) { newcount += hrfs.getStoreFiles(colFamily.getName()).size(); @@ -561,4 +555,4 @@ public class TestRegionMergeTransactionOnCluster { return resp; } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java index b031413..52e8724 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; @@ -444,14 +445,9 @@ public class TestScannerHeartbeatMessages { private static volatile int columnFamilySleepTime = DEFAULT_CF_SLEEP_TIME; private static volatile boolean sleepBetweenColumnFamilies = false; - public HeartbeatHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration confParam, - HRegionInfo regionInfo, HTableDescriptor htd, RegionServerServices rsServices) { - super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices); - } - - public HeartbeatHRegion(HRegionFileSystem fs, WAL wal, Configuration confParam, - HTableDescriptor htd, RegionServerServices rsServices) { - super(fs, wal, confParam, htd, rsServices); + public HeartbeatHRegion(final RegionFileSystem rfs, final HTableDescriptor htd, + final WAL wal, final RegionServerServices rsServices) { + super(rfs, htd, wal, rsServices); } private static void columnFamilySleep() { http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index c318fe4..420d77f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -769,7 +769,7 @@ public class TestSplitTransactionOnCluster { try { // Precondition: we created a table with no data, no store files. printOutRegions(regionServer, "Initial regions: "); - cluster.getMaster().getMasterFileSystem().logFSTree(LOG); + cluster.getMaster().getMasterFileSystem().logFileSystemState(LOG); List storefiles = getStoreFiles(tableName); assertEquals("Expected nothing but found " + storefiles.toString(), storefiles.size(), 0); @@ -793,7 +793,7 @@ public class TestSplitTransactionOnCluster { assertTrue(daughters.size() == 2); // check dirs - cluster.getMaster().getMasterFileSystem().logFSTree(LOG); + cluster.getMaster().getMasterFileSystem().logFileSystemState(LOG); List storefilesAfter = getStoreFiles(tableName); assertEquals("Expected nothing but found " + storefilesAfter.toString(), storefilesAfter.size(), 0); @@ -949,12 +949,8 @@ public class TestSplitTransactionOnCluster { SplitTransactionImpl st = new SplitTransactionImpl(regions.get(0), Bytes.toBytes("r3")); st.prepare(); st.stepsBeforePONR(regionServer, regionServer, false); - Path tableDir = - FSUtils.getTableDir(cluster.getMaster().getMasterFileSystem().getRootDir(), - desc.getTableName()); - List regionDirs = - FSUtils.getRegionDirs(tableDir.getFileSystem(cluster.getConfiguration()), tableDir); - assertEquals(3,regionDirs.size()); + assertEquals(3, cluster.getMaster().getMasterFileSystem().getRegions(desc.getTableName()).size()); + cluster.startRegionServer(); regionServer.kill(); // Before we check deadServerInProgress, we should ensure server is dead at master side. while (!cluster.getMaster().getServerManager(). @@ -969,9 +965,7 @@ public class TestSplitTransactionOnCluster { AssignmentManager am = cluster.getMaster().getAssignmentManager(); assertEquals(am.getRegionStates().getRegionsInTransition().toString(), 0, am .getRegionStates().getRegionsInTransition().size()); - regionDirs = - FSUtils.getRegionDirs(tableDir.getFileSystem(cluster.getConfiguration()), tableDir); - assertEquals(1,regionDirs.size()); + assertEquals(1, cluster.getMaster().getMasterFileSystem().getRegions(desc.getTableName()).size()); } finally { TESTING_UTIL.deleteTable(table); } http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index e3e62fc..7f44324 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -170,11 +171,9 @@ public class TestStore { HColumnDescriptor hcd) throws IOException { //Setting up a Store Path basedir = new Path(DIR+methodName); - Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName()); final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName)); FileSystem fs = FileSystem.get(conf); - fs.delete(logdir, true); if (htd.hasFamily(hcd.getName())) { @@ -182,12 +181,14 @@ public class TestStore { } else { htd.addFamily(hcd); } + HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); final Configuration walConf = new Configuration(conf); FSUtils.setRootDir(walConf, basedir); final WALFactory wals = new WALFactory(walConf, null, methodName); - HRegion region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes(), - info.getTable().getNamespace()), fs, conf, info, htd, null); + RegionFileSystem rfs = RegionFileSystem.open(conf, fs, basedir, info, false); + HRegion region = new HRegion(rfs, htd, + wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), null); store = new HStore(region, hcd, conf); return store; http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index 09532cf..e88a948 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.BlockCache; @@ -105,8 +106,7 @@ public class TestStoreFile extends HBaseTestCase { public void testBasicHalfMapFile() throws Exception { final HRegionInfo hri = new HRegionInfo(TableName.valueOf("testBasicHalfMapFileTb")); - HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem( - conf, fs, new Path(testDir, hri.getTable().getNameAsString()), hri); + RegionFileSystem regionFs = RegionFileSystem.open(conf, fs, testDir, hri, true); HFileContext meta = new HFileContextBuilder().withBlockSize(2*1024).build(); StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) @@ -116,8 +116,7 @@ public class TestStoreFile extends HBaseTestCase { writeStoreFile(writer); Path sfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath()); - StoreFile sf = new StoreFile(this.fs, sfPath, conf, cacheConf, - BloomType.NONE); + StoreFile sf = new StoreFile(this.fs, sfPath, conf, cacheConf, BloomType.NONE); checkHalfHFile(regionFs, sf); } @@ -157,8 +156,7 @@ public class TestStoreFile extends HBaseTestCase { @Test public void testReference() throws IOException { final HRegionInfo hri = new HRegionInfo(TableName.valueOf("testReferenceTb")); - HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem( - conf, fs, new Path(testDir, hri.getTable().getNameAsString()), hri); + RegionFileSystem regionFs = RegionFileSystem.open(conf, fs, testDir, hri, true); HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. @@ -223,8 +221,7 @@ public class TestStoreFile extends HBaseTestCase { // force temp data in hbase/target/test-data instead of /tmp/hbase-xxxx/ Configuration testConf = new Configuration(this.conf); FSUtils.setRootDir(testConf, testDir); - HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem( - testConf, fs, FSUtils.getTableDir(testDir, hri.getTable()), hri); + RegionFileSystem regionFs = RegionFileSystem.open(testConf, fs, testDir, hri, true); HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. @@ -268,8 +265,7 @@ public class TestStoreFile extends HBaseTestCase { // adding legal table name chars to verify regex handles it. HRegionInfo hri = new HRegionInfo(TableName.valueOf("_original-evil-name")); - HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem( - testConf, fs, FSUtils.getTableDir(testDir, hri.getTable()), hri); + RegionFileSystem regionFs = RegionFileSystem.open(testConf, fs, testDir, hri, true); HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. //// @@ -282,9 +278,7 @@ public class TestStoreFile extends HBaseTestCase { // create link to store file. /clone/region//-- HRegionInfo hriClone = new HRegionInfo(TableName.valueOf("clone")); - HRegionFileSystem cloneRegionFs = HRegionFileSystem.createRegionOnFileSystem( - testConf, fs, FSUtils.getTableDir(testDir, hri.getTable()), - hriClone); + RegionFileSystem cloneRegionFs = RegionFileSystem.open(testConf, fs, testDir, hriClone, true); Path dstPath = cloneRegionFs.getStoreDir(TEST_FAMILY); HFileLink.create(testConf, this.fs, dstPath, hri, storeFilePath.getName()); Path linkFilePath = new Path(dstPath, @@ -337,7 +331,7 @@ public class TestStoreFile extends HBaseTestCase { assertEquals((LAST_CHAR - FIRST_CHAR + 1) * (LAST_CHAR - FIRST_CHAR + 1), count); } - private void checkHalfHFile(final HRegionFileSystem regionFs, final StoreFile f) + private void checkHalfHFile(final RegionFileSystem regionFs, final StoreFile f) throws IOException { Cell midkey = f.createReader().midkey(); KeyValue midKV = (KeyValue)midkey; @@ -1001,7 +995,7 @@ public class TestStoreFile extends HBaseTestCase { assertEquals(startEvicted, cs.getEvictedCount()); } - private Path splitStoreFile(final HRegionFileSystem regionFs, final HRegionInfo hri, + private Path splitStoreFile(final RegionFileSystem regionFs, final HRegionInfo hri, final String family, final StoreFile sf, final byte[] splitKey, boolean isTopRef) throws IOException { FileSystem fs = regionFs.getFileSystem(); http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java index 74b3df9..7435567 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java @@ -46,6 +46,8 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.fs.RegionFileSystem; +import org.apache.hadoop.hbase.fs.legacy.LegacyRegionFileSystem; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -79,7 +81,7 @@ public class TestStoreFileRefresherChore { return htd; } - static class FailingHRegionFileSystem extends HRegionFileSystem { + static class FailingHRegionFileSystem extends LegacyRegionFileSystem { boolean fail = false; FailingHRegionFileSystem(Configuration conf, FileSystem fs, Path tableDir, HRegionInfo regionInfo) { super(conf, fs, tableDir, regionInfo); @@ -101,14 +103,14 @@ public class TestStoreFileRefresherChore { HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false, 0, replicaId); - HRegionFileSystem fs = new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, + RegionFileSystem fs = new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info); final Configuration walConf = new Configuration(conf); FSUtils.setRootDir(walConf, tableDir); final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId); - HRegion region = - new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), - conf, htd, null); + + HRegion region = new HRegion(fs, htd, + wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), null); region.initialize(); http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java index d6d4404..3065771 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java @@ -218,7 +218,7 @@ public class TestDurability { assertEquals(3, Bytes.toLong(res.getValue(FAMILY, col3))); verifyWALCount(wals, wal, 2); } - + /* * Test when returnResults set to false in increment it should not return the result instead it * resturn null. @@ -275,7 +275,7 @@ public class TestDurability { throw new IOException("Failed delete of " + path); } } - return HRegion.createHRegion(info, path, CONF, htd, log); + return HRegion.createHRegion(CONF, path, htd, info, log); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java index 7ca5d36..d943ccd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java @@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.fs.MasterFileSystem; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.master.HMaster; @@ -66,7 +67,6 @@ import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; -import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; @@ -357,6 +357,7 @@ public final class SnapshotTestingUtils { public static ArrayList listHFileNames(final FileSystem fs, final Path tableDir) throws IOException { final ArrayList hfiles = new ArrayList(); + /* TODO FSVisitor.visitTableStoreFiles(fs, tableDir, new FSVisitor.StoreFileVisitor() { @Override public void storeFile(final String region, final String family, final String hfileName) @@ -364,6 +365,7 @@ public final class SnapshotTestingUtils { hfiles.add(hfileName); } }); + */ Collections.sort(hfiles); return hfiles; } @@ -699,7 +701,7 @@ public final class SnapshotTestingUtils { // First region, simple with one plain hfile. HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKey, endKey); - HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, hri); + RegionFileSystem rfs = RegionFileSystem.open(conf, fs, tableDir, hri, true); regions[i] = new RegionData(tableDir, hri, 3); for (int j = 0; j < regions[i].files.length; ++j) { Path storeFile = createStoreFile(rfs.createTempName()); @@ -711,7 +713,7 @@ public final class SnapshotTestingUtils { startKey = Bytes.toBytes(2 + i * 2); endKey = Bytes.toBytes(3 + i * 2); hri = new HRegionInfo(htd.getTableName()); - rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, hri); + rfs = RegionFileSystem.open(conf, fs, tableDir, hri, true); regions[i+1] = new RegionData(tableDir, hri, regions[i].files.length); for (int j = 0; j < regions[i].files.length; ++j) { String refName = regions[i].files[j].getName() + '.' + regions[i].hri.getEncodedName(); http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index da51516..f6867f2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -61,12 +61,12 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.fs.legacy.LegacyLayout; import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.mob.MobFileName; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter; import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo; import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo; @@ -200,7 +200,7 @@ public class BaseTestHBaseFsck { FileSystem fs = rootDir.getFileSystem(conf); Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()), hri.getEncodedName()); - Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); + Path hriPath = LegacyLayout.getRegionInfoFile(p); fs.delete(hriPath, true); } @@ -582,7 +582,7 @@ public class BaseTestHBaseFsck { FileSystem fs = rootDir.getFileSystem(conf); Path p = new Path(rootDir + "/" + TableName.META_TABLE_NAME.getNameAsString(), hri.getEncodedName()); - Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); + Path hriPath = LegacyLayout.getRegionInfoFile(p); fs.delete(hriPath, true); } http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java index 97b6ddb..f5c2ac4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.fs.RegionFileSystem; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -206,15 +206,7 @@ public class OfflineMetaRebuildTestCore { HRegionInfo hri = new HRegionInfo(htbl.getName(), startKey, endKey); LOG.info("manually adding regioninfo and hdfs data: " + hri.toString()); - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(FSUtils.getTableDir(rootDir, htbl.getName()), - hri.getEncodedName()); - fs.mkdirs(p); - Path riPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); - FSDataOutputStream out = fs.create(riPath); - out.write(hri.toDelimitedByteArray()); - out.close(); + RegionFileSystem rfs = RegionFileSystem.open(conf, hri, true); // add to meta. MetaTableAccessor.addRegionToMeta(meta, hri); http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java index c7ea5f2..2b1bc8f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java @@ -349,7 +349,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { long putTime = runBenchmark(benchmarks, numThreads); logBenchmarkResult("Summary: threads=" + numThreads + ", iterations=" + numIterations + ", syncInterval=" + syncInterval, numIterations * numThreads, putTime); - + for (int i = 0; i < numRegions; i++) { if (regions[i] != null) { closeRegion(regions[i]); @@ -451,7 +451,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { private static void logBenchmarkResult(String testName, long numTests, long totalTime) { float tsec = totalTime / 1000.0f; LOG.info(String.format("%s took %.3fs %.3fops/s", testName, tsec, numTests / tsec)); - + } private void printUsageAndExit() { @@ -531,8 +531,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { } }); } - - return HRegion.createHRegion(regionInfo, dir, getConf(), htd, wal); + + return HRegion.createHRegion(getConf(), dir, htd, regionInfo, wal); } private void closeRegion(final HRegion region) throws IOException {