From 814d08a2d663ccdbacb33e84475713a72bf3726a Mon Sep 17 00:00:00 2001 From: zhangduo Date: Thu, 11 Jan 2018 15:47:08 +0800 Subject: [PATCH] HBASE-19751 Use RegionInfo directly instead of an identifier and a namespace when getting WAL --- .../hbase/mapreduce/TestWALRecordReader.java | 52 +++---- .../hbase/regionserver/HRegionServer.java | 14 +- .../apache/hadoop/hbase/util/HBaseFsck.java | 6 +- .../hbase/wal/AbstractFSWALProvider.java | 3 +- .../hadoop/hbase/wal/DisabledWALProvider.java | 2 +- .../hbase/wal/RegionGroupingProvider.java | 40 +++--- .../apache/hadoop/hbase/wal/WALFactory.java | 64 ++++----- .../apache/hadoop/hbase/wal/WALProvider.java | 17 ++- .../hadoop/hbase/HBaseTestingUtility.java | 5 +- .../hbase/coprocessor/TestWALObserver.java | 117 +++++++--------- .../TestCacheOnWriteInSchema.java | 52 +++---- .../TestCompactionArchiveConcurrentClose.java | 31 ++--- .../TestCompactionArchiveIOException.java | 42 +++--- .../regionserver/TestDefaultMemStore.java | 45 +++--- .../hbase/regionserver/TestHMobStore.java | 3 +- .../hbase/regionserver/TestHRegion.java | 13 +- .../regionserver/TestHRegionReplayEvents.java | 43 +++--- .../hadoop/hbase/regionserver/TestHStore.java | 14 +- .../TestStoreFileRefresherChore.java | 51 +++---- .../TestWALMonotonicallyIncreasingSeqId.java | 2 +- .../wal/AbstractTestLogRolling.java | 9 +- .../regionserver/wal/TestDurability.java | 93 +++++++------ .../regionserver/wal/TestLogRollAbort.java | 39 +++--- .../wal/TestLogRollingNoCluster.java | 17 +-- .../wal/TestWALActionsListener.java | 25 ++-- .../TestReplicationSourceManager.java | 2 +- .../regionserver/TestWALEntryStream.java | 13 +- .../hadoop/hbase/wal/IOTestProvider.java | 48 ++++--- .../TestBoundedRegionGroupingStrategy.java | 16 ++- .../hadoop/hbase/wal/TestFSHLogProvider.java | 130 ++++++++---------- .../hadoop/hbase/wal/TestSecureWAL.java | 22 +-- .../hadoop/hbase/wal/TestWALFactory.java | 127 +++++++---------- .../hadoop/hbase/wal/TestWALMethods.java | 2 +- .../hbase/wal/TestWALReaderOnSecureWAL.java | 29 ++-- .../hadoop/hbase/wal/TestWALRootDir.java | 35 ++--- .../apache/hadoop/hbase/wal/TestWALSplit.java | 37 ++--- .../hbase/wal/WALPerformanceEvaluation.java | 47 +++---- 37 files changed, 594 insertions(+), 713 deletions(-) diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java index 18bb1353ee6..c8db903650a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java @@ -24,30 +24,28 @@ import static org.junit.Assert.assertTrue; import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; -import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; 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; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALKeyRecordReader; import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; -import org.apache.hadoop.hbase.wal.WALEdit; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.junit.AfterClass; @@ -61,7 +59,7 @@ import org.slf4j.LoggerFactory; /** * JUnit tests for the WALRecordReader */ -@Category({MapReduceTests.class, MediumTests.class}) +@Category({ MapReduceTests.class, MediumTests.class }) public class TestWALRecordReader { private static final Logger LOG = LoggerFactory.getLogger(TestWALRecordReader.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -74,11 +72,9 @@ public class TestWALRecordReader { static final TableName tableName = TableName.valueOf(getName()); private static final byte [] rowName = tableName.getName(); // visible for TestHLogRecordReader - static final HRegionInfo info = new HRegionInfo(tableName, - Bytes.toBytes(""), Bytes.toBytes(""), false); - private static final byte [] family = Bytes.toBytes("column"); - private static final byte [] value = Bytes.toBytes("value"); - private static HTableDescriptor htd; + static final RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); + private static final byte[] family = Bytes.toBytes("column"); + private static final byte[] value = Bytes.toBytes("value"); private static Path logDir; protected MultiVersionConcurrencyControl mvcc; protected static NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); @@ -93,6 +89,7 @@ public class TestWALRecordReader { walFs.delete(walRootDir, true); mvcc = new MultiVersionConcurrencyControl(); } + @BeforeClass public static void setUpBeforeClass() throws Exception { // Make block sizes small. @@ -108,9 +105,6 @@ public class TestWALRecordReader { walRootDir = TEST_UTIL.createWALRootDir(); walFs = FSUtils.getWALFileSystem(conf); logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME); - - htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(family)); } @AfterClass @@ -127,7 +121,7 @@ public class TestWALRecordReader { @Test public void testPartialRead() throws Exception { final WALFactory walfactory = new WALFactory(conf, null, getName()); - WAL log = walfactory.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()); + WAL log = walfactory.getWAL(info); // This test depends on timestamp being millisecond based and the filename of the WAL also // being millisecond based. long ts = System.currentTimeMillis(); @@ -186,9 +180,8 @@ public class TestWALRecordReader { @Test public void testWALRecordReader() throws Exception { final WALFactory walfactory = new WALFactory(conf, null, getName()); - WAL log = walfactory.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()); + WAL log = walfactory.getWAL(info); byte [] value = Bytes.toBytes("value"); - final AtomicLong sequenceId = new AtomicLong(0); WALEdit edit = new WALEdit(); edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), System.currentTimeMillis(), value)); @@ -245,7 +238,7 @@ public class TestWALRecordReader { return new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, time, mvcc, scopes); } - protected WALRecordReader getReader() { + private WALRecordReader getReader() { return new WALKeyRecordReader(); } @@ -253,7 +246,7 @@ public class TestWALRecordReader { * Create a new reader from the split, and match the edits against the passed columns. */ private void testSplit(InputSplit split, byte[]... columns) throws Exception { - final WALRecordReader reader = getReader(); + WALRecordReader reader = getReader(); reader.initialize(split, MapReduceTestUtil.createDummyMapTaskAttemptContext(conf)); for (byte[] column : columns) { @@ -262,15 +255,12 @@ public class TestWALRecordReader { if (!Bytes.equals(column, 0, column.length, cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength())) { assertTrue( - "expected [" - + Bytes.toString(column) - + "], actual [" - + Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength()) + "]", false); + "expected [" + Bytes.toString(column) + "], actual [" + Bytes.toString( + cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()) + "]", + false); } } assertFalse(reader.nextKeyValue()); reader.close(); } - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 63451c600be..53390bd870c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -2105,8 +2105,6 @@ public class HRegionServer extends HasThread implements return healthy; } - private static final byte[] UNSPECIFIED_REGION = new byte[]{}; - @Override public List getWALs() throws IOException { return walFactory.getWALs(); @@ -2114,17 +2112,7 @@ public class HRegionServer extends HasThread implements @Override public WAL getWAL(RegionInfo regionInfo) throws IOException { - WAL wal; - // _ROOT_ and hbase:meta regions have separate WAL. - if (regionInfo != null && regionInfo.isMetaRegion() - && regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { - wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes()); - } else if (regionInfo == null) { - wal = walFactory.getWAL(UNSPECIFIED_REGION, null); - } else { - byte[] namespace = regionInfo.getTable().getNamespace(); - wal = walFactory.getWAL(regionInfo.getEncodedNameAsBytes(), namespace); - } + WAL wal = walFactory.getWAL(regionInfo); if (this.walRoller != null) { this.walRoller.addWAL(wal); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 5ca1ed671bf..d0276c06b59 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -1480,9 +1480,9 @@ public class HBaseFsck extends Configured implements Closeable { // unless I pass along via the conf. Configuration confForWAL = new Configuration(c); confForWAL.set(HConstants.HBASE_DIR, rootdir.toString()); - WAL wal = (new WALFactory(confForWAL, - Collections. singletonList(new MetricsWAL()), walFactoryID)) - .getWAL(metaHRI.getEncodedNameAsBytes(), metaHRI.getTable().getNamespace()); + WAL wal = + new WALFactory(confForWAL, Collections. singletonList(new MetricsWAL()), + walFactoryID).getWAL(metaHRI); HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c, metaDescriptor, wal); MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, true); return meta; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java index 103b05fad49..74d502e0776 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; @@ -128,7 +129,7 @@ public abstract class AbstractFSWALProvider> implemen } @Override - public T getWAL(byte[] identifier, byte[] namespace) throws IOException { + public T getWAL(RegionInfo region) throws IOException { T walCopy = wal; if (walCopy == null) { // only lock when need to create wal, and need to lock since diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java index 280d95fec30..725f9ff3ec8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java @@ -74,7 +74,7 @@ class DisabledWALProvider implements WALProvider { } @Override - public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException { + public WAL getWAL(RegionInfo region) throws IOException { return disabled; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java index b8c9484ab3b..a3e54a5a432 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java @@ -27,15 +27,17 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; - +import java.util.concurrent.locks.Lock; import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.RegionInfo; // imports for classes still in regionserver.wal import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.IdLock; +import org.apache.hadoop.hbase.util.KeyLocker; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A WAL Provider that returns a WAL per group of regions. @@ -131,7 +133,7 @@ public class RegionGroupingProvider implements WALProvider { /** A group-provider mapping, make sure one-one rather than many-one mapping */ private final ConcurrentMap cached = new ConcurrentHashMap<>(); - private final IdLock createLock = new IdLock(); + private final KeyLocker createLock = new KeyLocker<>(); private RegionGroupingStrategy strategy = null; private WALFactory factory = null; @@ -177,33 +179,39 @@ public class RegionGroupingProvider implements WALProvider { return wals; } - private WAL getWAL(final String group) throws IOException { + private WAL getWAL(String group) throws IOException { WALProvider provider = cached.get(group); if (provider == null) { - IdLock.Entry lockEntry = null; + Lock lock = createLock.acquireLock(group); try { - lockEntry = createLock.getLockEntry(group.hashCode()); provider = cached.get(group); if (provider == null) { provider = createProvider(group); cached.put(group, provider); } } finally { - if (lockEntry != null) { - createLock.releaseLockEntry(lockEntry); - } + lock.unlock(); } } - return provider.getWAL(null, null); + return provider.getWAL(null); } @Override - public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException { - final String group; + public WAL getWAL(RegionInfo region) throws IOException { + String group; if (META_WAL_PROVIDER_ID.equals(this.providerId)) { group = META_WAL_GROUP_NAME; } else { - group = strategy.group(identifier, namespace); + byte[] id; + byte[] namespace; + if (region != null) { + id = region.getEncodedNameAsBytes(); + namespace = region.getTable().getNamespace(); + } else { + id = HConstants.EMPTY_BYTE_ARRAY; + namespace = null; + } + group = strategy.group(id, namespace); } return getWAL(group); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java index 1ed41688d1a..d70b8cdbb4a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java @@ -1,5 +1,4 @@ /** - * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -16,25 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package org.apache.hadoop.hbase.wal; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import java.io.IOException; import java.io.InterruptedIOException; import java.util.Collections; import java.util.List; import java.util.OptionalLong; import java.util.concurrent.atomic.AtomicReference; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.client.RegionInfo; // imports for things that haven't moved from regionserver.wal yet. import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL; import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader; @@ -45,6 +37,11 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.LeaseNotRecoveredException; import org.apache.hadoop.hbase.wal.WAL.Reader; import org.apache.hadoop.hbase.wal.WALProvider.Writer; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Entry point for users of the Write Ahead Log. @@ -91,11 +88,11 @@ public class WALFactory implements WALFileLengthProvider { static final String DEFAULT_META_WAL_PROVIDER = Providers.defaultProvider.name(); final String factoryId; - final WALProvider provider; + private final WALProvider provider; // The meta updates are written to a different wal. If this // regionserver holds meta regions, then this ref will be non-null. // lazily intialized; most RegionServers don't deal with META - final AtomicReference metaProvider = new AtomicReference<>(); + private final AtomicReference metaProvider = new AtomicReference<>(); /** * Configuration-specified WAL Reader used when a custom reader is requested @@ -236,32 +233,35 @@ public class WALFactory implements WALFileLengthProvider { return provider.getWALs(); } - /** - * @param identifier may not be null, contents will not be altered - * @param namespace could be null, and will use default namespace if null - */ - public WAL getWAL(final byte[] identifier, final byte[] namespace) throws IOException { - return provider.getWAL(identifier, namespace); + private WALProvider getMetaProvider() throws IOException { + for (;;) { + WALProvider provider = this.metaProvider.get(); + if (provider != null) { + return provider; + } + provider = getProvider(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER, + Collections. singletonList(new MetricsWAL()), + AbstractFSWALProvider.META_WAL_PROVIDER_ID); + if (metaProvider.compareAndSet(null, provider)) { + return provider; + } else { + // someone is ahead of us, close and try again. + provider.close(); + } + } } /** - * @param identifier may not be null, contents will not be altered + * @param region the region which we want to get a WAL for it. Could be null. */ - public WAL getMetaWAL(final byte[] identifier) throws IOException { - WALProvider metaProvider = this.metaProvider.get(); - if (null == metaProvider) { - final WALProvider temp = getProvider(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER, - Collections.singletonList(new MetricsWAL()), - AbstractFSWALProvider.META_WAL_PROVIDER_ID); - if (this.metaProvider.compareAndSet(null, temp)) { - metaProvider = temp; - } else { - // reference must now be to a provider created in another thread. - temp.close(); - metaProvider = this.metaProvider.get(); - } + public WAL getWAL(RegionInfo region) throws IOException { + // use different WAL for hbase:meta + if (region != null && region.isMetaRegion() && + region.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { + return getMetaProvider().getWAL(region); + } else { + return provider.getWAL(region); } - return metaProvider.getWAL(identifier, null); } public Reader createReader(final FileSystem fs, final Path path) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java index c38f419d89f..0586d1dc365 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java @@ -22,11 +22,10 @@ import java.io.Closeable; import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; - import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; -// imports for things that haven't moved from regionserver.wal yet. +import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; +import org.apache.yetus.audience.InterfaceAudience; /** * The Write Ahead Log (WAL) stores all durable edits to the HRegion. @@ -48,17 +47,17 @@ public interface WALProvider { * @param listeners may be null * @param providerId differentiate between providers from one factory. may be null */ - void init(final WALFactory factory, final Configuration conf, - final List listeners, final String providerId) throws IOException; + void init(WALFactory factory, Configuration conf, List listeners, + String providerId) throws IOException; /** - * @param identifier may not be null. contents will not be altered. - * @param namespace could be null, and will use default namespace if null + * @param region the region which we want to get a WAL for it. Could be null. * @return a WAL for writing entries for the given region. */ - WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException; + WAL getWAL(RegionInfo region) throws IOException; - /** @return the List of WALs that are used by this server + /** + * @return the List of WALs that are used by this server */ List getWALs(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 304b3cb0db0..fe8902a5d29 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -2310,9 +2310,8 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { Configuration confForWAL = new Configuration(conf); confForWAL.set(HConstants.HBASE_DIR, rootDir.toString()); return (new WALFactory(confForWAL, - Collections.singletonList(new MetricsWAL()), - "hregion-" + RandomStringUtils.randomNumeric(8))). - getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()); + Collections. singletonList(new MetricsWAL()), + "hregion-" + RandomStringUtils.randomNumeric(8))).getWAL(hri); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java index 86a0d391f1b..166dfdd596c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java @@ -1,5 +1,4 @@ -/* - * +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertEquals; @@ -31,7 +29,6 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -40,18 +37,19 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseConfiguration; 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.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -61,6 +59,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.wal.WALSplitter; @@ -172,17 +171,17 @@ public class TestWALObserver { */ @Test public void testWALObserverWriteToWAL() throws Exception { - final WAL log = wals.getWAL(UNSPECIFIED_REGION, null); + final WAL log = wals.getWAL(null); verifyWritesSeen(log, getCoprocessor(log, SampleRegionWALCoprocessor.class), false); } private void verifyWritesSeen(final WAL log, final SampleRegionWALCoprocessor cp, final boolean seesLegacy) throws Exception { - HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE)); - final HTableDescriptor htd = createBasic3FamilyHTD(Bytes + RegionInfo hri = createBasicHRegionInfo(Bytes.toString(TEST_TABLE)); + TableDescriptor htd = createBasic3FamilyHTD(Bytes .toString(TEST_TABLE)); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + for (byte[] fam : htd.getColumnFamilyNames()) { scopes.put(fam, 0); } Path basedir = new Path(this.hbaseRootDir, Bytes.toString(TEST_TABLE)); @@ -268,14 +267,14 @@ public class TestWALObserver { */ @Test public void testEmptyWALEditAreNotSeen() throws Exception { - final HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE)); - final HTableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE)); - final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); + RegionInfo hri = createBasicHRegionInfo(Bytes.toString(TEST_TABLE)); + TableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE)); + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + for(byte[] fam : htd.getColumnFamilyNames()) { scopes.put(fam, 0); } - WAL log = wals.getWAL(UNSPECIFIED_REGION, null); + WAL log = wals.getWAL(null); try { SampleRegionWALCoprocessor cp = getCoprocessor(log, SampleRegionWALCoprocessor.class); @@ -304,14 +303,14 @@ public class TestWALObserver { public void testWALCoprocessorReplay() throws Exception { // WAL replay is handled at HRegion::replayRecoveredEdits(), which is // ultimately called by HRegion::initialize() - final TableName tableName = TableName.valueOf(currentTest.getMethodName()); - final HTableDescriptor htd = getBasic3FamilyHTableDescriptor(tableName); + TableName tableName = TableName.valueOf(currentTest.getMethodName()); + TableDescriptor htd = getBasic3FamilyHTableDescriptor(tableName); MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); // final HRegionInfo hri = // createBasic3FamilyHRegionInfo(Bytes.toString(tableName)); // final HRegionInfo hri1 = // createBasic3FamilyHRegionInfo(Bytes.toString(tableName)); - final HRegionInfo hri = new HRegionInfo(tableName, null, null); + RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build(); final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName); @@ -321,21 +320,21 @@ public class TestWALObserver { final Configuration newConf = HBaseConfiguration.create(this.conf); // WAL wal = new WAL(this.fs, this.dir, this.oldLogDir, this.conf); - WAL wal = wals.getWAL(UNSPECIFIED_REGION, null); + WAL wal = wals.getWAL(null); // Put p = creatPutWith2Families(TEST_ROW); WALEdit edit = new WALEdit(); long now = EnvironmentEdgeManager.currentTime(); final int countPerFamily = 1000; NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for (HColumnDescriptor hcd : htd.getFamilies()) { - scopes.put(hcd.getName(), 0); + for (byte[] fam : htd.getColumnFamilyNames()) { + scopes.put(fam, 0); } - for (HColumnDescriptor hcd : htd.getFamilies()) { - addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily, - EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc); + for (byte[] fam : htd.getColumnFamilyNames()) { + addWALEdits(tableName, hri, TEST_ROW, fam, countPerFamily, + EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc); } wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit, - true); + true); // sync to fs. wal.sync(); @@ -345,14 +344,12 @@ public class TestWALObserver { public Void run() throws Exception { Path p = runWALSplit(newConf); LOG.info("WALSplit path == " + p); - FileSystem newFS = FileSystem.get(newConf); // Make a new wal for new region open. final WALFactory wals2 = new WALFactory(conf, null, ServerName.valueOf(currentTest.getMethodName()+"2", 16010, System.currentTimeMillis()).toString()); - WAL wal2 = wals2.getWAL(UNSPECIFIED_REGION, null);; + WAL wal2 = wals2.getWAL(null); HRegion region = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir, hri, htd, wal2, TEST_UTIL.getHBaseCluster().getRegionServer(0), null); - long seqid2 = region.getOpenSeqNum(); SampleRegionWALCoprocessor cp2 = region.getCoprocessorHost().findCoprocessor(SampleRegionWALCoprocessor.class); @@ -374,13 +371,13 @@ public class TestWALObserver { */ @Test public void testWALObserverLoaded() throws Exception { - WAL log = wals.getWAL(UNSPECIFIED_REGION, null); + WAL log = wals.getWAL(null); assertNotNull(getCoprocessor(log, SampleRegionWALCoprocessor.class)); } @Test public void testWALObserverRoll() throws Exception { - final WAL wal = wals.getWAL(UNSPECIFIED_REGION, null); + final WAL wal = wals.getWAL(null); final SampleRegionWALCoprocessor cp = getCoprocessor(wal, SampleRegionWALCoprocessor.class); cp.setTestValues(TEST_TABLE, null, null, null, null, null, null, null); @@ -399,20 +396,12 @@ public class TestWALObserver { return (SampleRegionWALCoprocessor) c; } - /* - * Creates an HRI around an HTD that has tableName and three - * column families named. - * - * @param tableName Name of table to use when we create HTableDescriptor. + /** + * Creates an HRI around an HTD that has tableName. + * @param tableName Name of table to use. */ - private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); - - for (int i = 0; i < TEST_FAMILY.length; i++) { - HColumnDescriptor a = new HColumnDescriptor(TEST_FAMILY[i]); - htd.addFamily(a); - } - return new HRegionInfo(htd.getTableName(), null, null, false); + private RegionInfo createBasicHRegionInfo(String tableName) { + return RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build(); } /* @@ -463,12 +452,10 @@ public class TestWALObserver { return splits.get(0); } - private static final byte[] UNSPECIFIED_REGION = new byte[]{}; - - private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName, + private void addWALEdits(final TableName tableName, final RegionInfo hri, final byte[] rowName, final byte[] family, final int count, EnvironmentEdge ee, final WAL wal, final NavigableMap scopes, final MultiVersionConcurrencyControl mvcc) - throws IOException { + throws IOException { String familyStr = Bytes.toString(family); long txid = -1; for (int j = 0; j < count; j++) { @@ -478,33 +465,25 @@ public class TestWALObserver { edit.add(new KeyValue(rowName, family, qualifierBytes, ee.currentTime(), columnBytes)); // uses WALKeyImpl instead of HLogKey on purpose. will only work for tests where we don't care // about legacy coprocessors - txid = wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, - ee.currentTime(), mvcc), edit, true); + txid = wal.append(hri, + new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, ee.currentTime(), mvcc), edit, true); } if (-1 != txid) { wal.sync(txid); } } - private HTableDescriptor getBasic3FamilyHTableDescriptor( - final TableName tableName) { - HTableDescriptor htd = new HTableDescriptor(tableName); - - for (int i = 0; i < TEST_FAMILY.length; i++) { - HColumnDescriptor a = new HColumnDescriptor(TEST_FAMILY[i]); - htd.addFamily(a); - } - return htd; + private TableDescriptor getBasic3FamilyHTableDescriptor(TableName tableName) { + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); + Arrays.stream(TEST_FAMILY).map(ColumnFamilyDescriptorBuilder::of) + .forEachOrdered(builder::addColumnFamily); + return builder.build(); } - private HTableDescriptor createBasic3FamilyHTD(final String tableName) { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); - HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a")); - htd.addFamily(a); - HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b")); - htd.addFamily(b); - HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c")); - htd.addFamily(c); - return htd; + private TableDescriptor createBasic3FamilyHTD(String tableName) { + return TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("a")) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("b")) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("c")).build(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index 02a21b6aac4..0f4ece6c2a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -26,16 +26,18 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.BlockCache; @@ -106,18 +108,19 @@ public class TestCacheOnWriteInSchema { return blockType == blockType1 || blockType == blockType2; } - public void modifyFamilySchema(HColumnDescriptor family) { + public ColumnFamilyDescriptorBuilder modifyFamilySchema(ColumnFamilyDescriptorBuilder builder) { switch (this) { - case DATA_BLOCKS: - family.setCacheDataOnWrite(true); - break; - case BLOOM_BLOCKS: - family.setCacheBloomsOnWrite(true); - break; - case INDEX_BLOCKS: - family.setCacheIndexesOnWrite(true); - break; + case DATA_BLOCKS: + builder.setCacheDataOnWrite(true); + break; + case BLOOM_BLOCKS: + builder.setCacheBloomsOnWrite(true); + break; + case INDEX_BLOCKS: + builder.setCacheIndexesOnWrite(true); + break; } + return builder; } } @@ -158,23 +161,22 @@ public class TestCacheOnWriteInSchema { fs = HFileSystem.get(conf); // Create the schema - HColumnDescriptor hcd = new HColumnDescriptor(family); - hcd.setBloomFilterType(BloomType.ROWCOL); - cowType.modifyFamilySchema(hcd); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); - htd.addFamily(hcd); + ColumnFamilyDescriptor hcd = cowType + .modifyFamilySchema( + ColumnFamilyDescriptorBuilder.newBuilder(family).setBloomFilterType(BloomType.ROWCOL)) + .build(); + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(table)).addColumnFamily(hcd).build(); // Create a store based on the schema - final String id = TestCacheOnWriteInSchema.class.getName(); - final Path logdir = new Path(FSUtils.getRootDir(conf), - AbstractFSWALProvider.getWALDirectoryName(id)); + String id = TestCacheOnWriteInSchema.class.getName(); + Path logdir = new Path(FSUtils.getRootDir(conf), AbstractFSWALProvider.getWALDirectoryName(id)); fs.delete(logdir, true); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build(); walFactory = new WALFactory(conf, null, id); - region = TEST_UTIL.createLocalHRegion(info, htd, - walFactory.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace())); + region = TEST_UTIL.createLocalHRegion(info, htd, walFactory.getWAL(info)); store = new HStore(region, hcd, conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java index 95dec3dfa5a..d2e18668419 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import java.io.IOException; import java.io.InterruptedIOException; @@ -29,17 +28,18 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -85,10 +85,10 @@ public class TestCompactionArchiveConcurrentClose { byte[] col = Bytes.toBytes("c"); byte[] val = Bytes.toBytes("val"); - final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(fam)); - HRegionInfo info = new HRegionInfo(tableName, null, null, false); + TableName tableName = TableName.valueOf(name.getMethodName()); + TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build(); + RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); HRegion region = initHRegion(htd, info); RegionServerServices rss = mock(RegionServerServices.class); List regions = new ArrayList<>(); @@ -157,20 +157,17 @@ public class TestCompactionArchiveConcurrentClose { } } - private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info) - throws IOException { + private HRegion initHRegion(TableDescriptor htd, RegionInfo info) throws IOException { Configuration conf = testUtil.getConfiguration(); Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName()); - HRegionFileSystem fs = new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf), - tableDir, info); + HRegionFileSystem fs = + new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); final Configuration walConf = new Configuration(conf); FSUtils.setRootDir(walConf, tableDir); final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName()); - HRegion region = - new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), - conf, htd, null); + HRegion region = new HRegion(fs, wals.getWAL(info), conf, htd, null); region.initialize(); @@ -180,7 +177,7 @@ public class TestCompactionArchiveConcurrentClose { private class WaitingHRegionFileSystem extends HRegionFileSystem { public WaitingHRegionFileSystem(final Configuration conf, final FileSystem fs, - final Path tableDir, final HRegionInfo regionInfo) { + final Path tableDir, final RegionInfo regionInfo) { super(conf, fs, tableDir, regionInfo); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java index 2b555a5a295..9c085e5eed2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java @@ -1,4 +1,4 @@ -/* +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -15,38 +15,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; - import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.FailedArchiveException; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -59,6 +57,8 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.mockito.Mockito; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; + /** * Tests that archiving compacted files behaves correctly when encountering exceptions. */ @@ -93,11 +93,11 @@ public class TestCompactionArchiveIOException { byte[] col = Bytes.toBytes("c"); byte[] val = Bytes.toBytes("val"); - final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(fam)); - HRegionInfo info = new HRegionInfo(tableName, null, null, false); - final HRegion region = initHRegion(htd, info); + TableName tableName = TableName.valueOf(name.getMethodName()); + TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build(); + RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); + HRegion region = initHRegion(htd, info); RegionServerServices rss = mock(RegionServerServices.class); List regions = new ArrayList<>(); regions.add(region); @@ -172,29 +172,25 @@ public class TestCompactionArchiveIOException { } } - private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info) - throws IOException { + private HRegion initHRegion(TableDescriptor htd, RegionInfo info) throws IOException { Configuration conf = testUtil.getConfiguration(); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName()); Path regionDir = new Path(tableDir, info.getEncodedName()); Path storeDir = new Path(regionDir, htd.getColumnFamilies()[0].getNameAsString()); - FileSystem errFS = spy(testUtil.getTestFileSystem()); // Prior to HBASE-16964, when an exception is thrown archiving any compacted file, // none of the other files are cleared from the compactedfiles list. // Simulate this condition with a dummy file - doThrow(new IOException("Error for test")) - .when(errFS).rename(eq(new Path(storeDir, ERROR_FILE)), any()); + doThrow(new IOException("Error for test")).when(errFS) + .rename(eq(new Path(storeDir, ERROR_FILE)), any()); HRegionFileSystem fs = new HRegionFileSystem(conf, errFS, tableDir, info); final Configuration walConf = new Configuration(conf); FSUtils.setRootDir(walConf, tableDir); final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName()); - HRegion region = - new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), - conf, htd, null); + HRegion region = new HRegion(fs, wals.getWAL(info), conf, htd, null); region.initialize(); 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 7772f7b05e4..b89fb0e1949 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 @@ -30,7 +30,6 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CategoryBasedTimeout; @@ -39,18 +38,19 @@ import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; 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.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.exceptions.UnexpectedStateException; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -68,6 +68,7 @@ import org.junit.rules.TestName; import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Joiner; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -451,22 +452,16 @@ public class TestDefaultMemStore { final MultiVersionConcurrencyControl mvcc; final MemStore memstore; - final AtomicLong startSeqNum; AtomicReference caughtException; - public ReadOwnWritesTester(int id, - MemStore memstore, - MultiVersionConcurrencyControl mvcc, - AtomicReference caughtException, - AtomicLong startSeqNum) - { + public ReadOwnWritesTester(int id, MemStore memstore, MultiVersionConcurrencyControl mvcc, + AtomicReference caughtException) { this.mvcc = mvcc; this.memstore = memstore; this.caughtException = caughtException; row = Bytes.toBytes(id); - this.startSeqNum = startSeqNum; } @Override @@ -505,14 +500,13 @@ public class TestDefaultMemStore { @Test public void testReadOwnWritesUnderConcurrency() throws Throwable { - int NUM_THREADS = 8; ReadOwnWritesTester threads[] = new ReadOwnWritesTester[NUM_THREADS]; AtomicReference caught = new AtomicReference<>(); for (int i = 0; i < NUM_THREADS; i++) { - threads[i] = new ReadOwnWritesTester(i, memstore, mvcc, caught, this.startSeqNum); + threads[i] = new ReadOwnWritesTester(i, memstore, mvcc, caught); threads[i].start(); } @@ -921,7 +915,8 @@ public class TestDefaultMemStore { EnvironmentEdgeManager.injectEdge(edge); HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf); String cf = "foo"; - HRegion region = hbaseUtility.createTestRegion("foobar", new HColumnDescriptor(cf)); + HRegion region = + hbaseUtility.createTestRegion("foobar", ColumnFamilyDescriptorBuilder.of(cf)); edge.setCurrentTimeMillis(1234); Put p = new Put(Bytes.toBytes("r")); @@ -950,20 +945,16 @@ public class TestDefaultMemStore { EnvironmentEdgeManager.injectEdge(edge); edge.setCurrentTimeMillis(1234); WALFactory wFactory = new WALFactory(conf, null, "1234"); - HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir, + HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir, conf, FSTableDescriptors.createMetaTableDescriptor(conf), - wFactory.getMetaWAL(HRegionInfo.FIRST_META_REGIONINFO. - getEncodedNameAsBytes())); + wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO)); // parameterized tests add [#] suffix get rid of [ and ]. - HRegionInfo hri = - new HRegionInfo(TableName.valueOf(name.getMethodName().replaceAll("[\\[\\]]", "_")), - Bytes.toBytes("row_0200"), Bytes.toBytes("row_0300")); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf( - name.getMethodName().replaceAll("[\\[\\]]", "_"))); - desc.addFamily(new HColumnDescriptor("foo".getBytes())); - HRegion r = - HRegion.createHRegion(hri, testDir, conf, desc, - wFactory.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace())); + TableDescriptor desc = TableDescriptorBuilder + .newBuilder(TableName.valueOf(name.getMethodName().replaceAll("[\\[\\]]", "_"))) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("foo")).build(); + RegionInfo hri = RegionInfoBuilder.newBuilder(desc.getTableName()) + .setStartKey(Bytes.toBytes("row_0200")).setEndKey(Bytes.toBytes("row_0300")).build(); + HRegion r = HRegion.createHRegion(hri, testDir, conf, desc, wFactory.getWAL(hri)); addRegionToMETA(meta, r); edge.setCurrentTimeMillis(1234 + 100); StringBuilder sb = new StringBuilder(); 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 32e3856f0a3..9845d8a4998 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 @@ -162,8 +162,7 @@ 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); + region = new HRegion(tableDir, wals.getWAL(info), fs, conf, info, htd, null); store = new HMobStore(region, hcd, conf); if(testStore) { init(conf, hcd); 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 5ddd4dfb120..3c11b3164c4 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 @@ -63,7 +63,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; - import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -110,6 +109,7 @@ import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; @@ -180,8 +180,10 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor; @@ -378,9 +380,8 @@ public class TestHRegion { final Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log"); final Configuration walConf = new Configuration(conf); FSUtils.setRootDir(walConf, logDir); - return (new WALFactory(walConf, - Collections.singletonList(new MetricsWAL()), callingMethod)) - .getWAL(tableName.toBytes(), tableName.getNamespace()); + return new WALFactory(walConf, Collections. singletonList(new MetricsWAL()), + callingMethod).getWAL(RegionInfoBuilder.newBuilder(tableName).build()); } @Test @@ -960,7 +961,7 @@ public class TestHRegion { final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration()); FSUtils.setRootDir(walConf, logDir); final WALFactory wals = new WALFactory(walConf, null, method); - final WAL wal = wals.getWAL(tableName.getName(), tableName.getNamespace()); + final WAL wal = wals.getWAL(RegionInfoBuilder.newBuilder(tableName).build()); this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family); @@ -4690,7 +4691,7 @@ public class TestHRegion { // deal with classes which have a field of an inner class. See discussions in HBASE-15536. walConf.set(WALFactory.WAL_PROVIDER, "filesystem"); final WALFactory wals = new WALFactory(walConf, null, UUID.randomUUID().toString()); - final WAL wal = spy(wals.getWAL(tableName.getName(), tableName.getNamespace())); + final WAL wal = spy(wals.getWAL(RegionInfoBuilder.newBuilder(tableName).build())); this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, tableDurability, wal, new byte[][] { family }); 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 771523104c7..3c25f6b96b5 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 @@ -1,4 +1,4 @@ -/* +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.regionserver.TestHRegion.assertGet; @@ -27,8 +26,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -43,7 +42,6 @@ import java.util.Map; import java.util.Objects; import java.util.Random; import java.util.UUID; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -51,7 +49,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -59,6 +56,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 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.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.executor.ExecutorService; @@ -89,8 +88,10 @@ import org.junit.rules.TestName; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor; @@ -132,7 +133,7 @@ public class TestHRegionReplayEvents { private TableDescriptor htd; private long time; private RegionServerServices rss; - private HRegionInfo primaryHri, secondaryHri; + private RegionInfo primaryHri, secondaryHri; private HRegion primaryRegion, secondaryRegion; private WALFactory wals; private WAL walPrimary, walSecondary; @@ -156,18 +157,14 @@ public class TestHRegionReplayEvents { time = System.currentTimeMillis(); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); - primaryHri = new HRegionInfo(htd.getTableName(), - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, - false, time, 0); - secondaryHri = new HRegionInfo(htd.getTableName(), - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, - false, time, 1); + primaryHri = + RegionInfoBuilder.newBuilder(htd.getTableName()).setRegionId(time).setReplicaId(0).build(); + secondaryHri = + RegionInfoBuilder.newBuilder(htd.getTableName()).setRegionId(time).setReplicaId(1).build(); wals = TestHRegion.createWALFactory(CONF, rootDir); - walPrimary = wals.getWAL(primaryHri.getEncodedNameAsBytes(), - primaryHri.getTable().getNamespace()); - walSecondary = wals.getWAL(secondaryHri.getEncodedNameAsBytes(), - secondaryHri.getTable().getNamespace()); + walPrimary = wals.getWAL(primaryHri); + walSecondary = wals.getWAL(secondaryHri); rss = mock(RegionServerServices.class); when(rss.getServerName()).thenReturn(ServerName.valueOf("foo", 1, 1)); @@ -1150,8 +1147,8 @@ public class TestHRegionReplayEvents { // test for region open and close secondaryRegion = HRegion.openHRegion(secondaryHri, htd, walSecondary, CONF, rss, null); - verify(walSecondary, times(0)).append((HRegionInfo)any(), - (WALKeyImpl)any(), (WALEdit)any(), anyBoolean()); + verify(walSecondary, times(0)).append(any(RegionInfo.class), any(WALKeyImpl.class), + any(WALEdit.class), anyBoolean()); // test for replay prepare flush putDataByReplay(secondaryRegion, 0, 10, cq, families); @@ -1166,12 +1163,12 @@ public class TestHRegionReplayEvents { primaryRegion.getRegionInfo().getRegionName())) .build()); - verify(walSecondary, times(0)).append((HRegionInfo)any(), - (WALKeyImpl)any(), (WALEdit)any(), anyBoolean()); + verify(walSecondary, times(0)).append(any(RegionInfo.class), any(WALKeyImpl.class), + any(WALEdit.class), anyBoolean()); secondaryRegion.close(); - verify(walSecondary, times(0)).append((HRegionInfo)any(), - (WALKeyImpl)any(), (WALEdit)any(), anyBoolean()); + verify(walSecondary, times(0)).append(any(RegionInfo.class), any(WALKeyImpl.class), + any(WALEdit.class), anyBoolean()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index e28e484500b..b8d3ec7a59f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -47,7 +47,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -65,7 +64,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MemoryCompactionPolicy; import org.apache.hadoop.hbase.PrivateCellUtil; @@ -73,6 +71,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; @@ -112,6 +112,7 @@ import org.junit.rules.TestName; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -205,12 +206,12 @@ public class TestHStore { fs.delete(logdir, true); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); + RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build(); Configuration walConf = new Configuration(conf); FSUtils.setRootDir(walConf, basedir); WALFactory wals = new WALFactory(walConf, null, methodName); - region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info), - wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), conf, htd, null); + region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info), wals.getWAL(info), conf, + htd, null); } private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder, @@ -1006,7 +1007,6 @@ public class TestHStore { assertEquals(0, this.store.getStorefilesCount()); } - @SuppressWarnings("unchecked") @Test public void testRefreshStoreFilesNotChanged() throws IOException { init(name.getMethodName()); 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 416b194088e..35c744f705a 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 @@ -26,31 +26,33 @@ import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; 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.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 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.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.StoppableImplementation; +import org.apache.hadoop.hbase.wal.WALFactory; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -73,20 +75,20 @@ public class TestStoreFileRefresherChore { FSUtils.setRootDir(TEST_UTIL.getConfiguration(), testDir); } - private HTableDescriptor getTableDesc(TableName tableName, byte[]... families) { - HTableDescriptor htd = new HTableDescriptor(tableName); - for (byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family); - // Set default to be three versions. - hcd.setMaxVersions(Integer.MAX_VALUE); - htd.addFamily(hcd); - } - return htd; + private TableDescriptor getTableDesc(TableName tableName, int regionReplication, + byte[]... families) { + TableDescriptorBuilder builder = + TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(regionReplication); + Arrays.stream(families).map(family -> ColumnFamilyDescriptorBuilder.newBuilder(family) + .setMaxVersions(Integer.MAX_VALUE).build()).forEachOrdered(builder::addColumnFamily); + return builder.build(); } static class FailingHRegionFileSystem extends HRegionFileSystem { boolean fail = false; - FailingHRegionFileSystem(Configuration conf, FileSystem fs, Path tableDir, HRegionInfo regionInfo) { + + FailingHRegionFileSystem(Configuration conf, FileSystem fs, Path tableDir, + RegionInfo regionInfo) { super(conf, fs, tableDir, regionInfo); } @@ -99,21 +101,21 @@ public class TestStoreFileRefresherChore { } } - private HRegion initHRegion(HTableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId) + private HRegion initHRegion(TableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId) throws IOException { Configuration conf = TEST_UTIL.getConfiguration(); Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName()); - HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false, 0, replicaId); - - HRegionFileSystem fs = new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, - info); + RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(startKey) + .setEndKey(stopKey).setRegionId(0L).setReplicaId(replicaId).build(); + HRegionFileSystem 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); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); HRegion region = - new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), + new HRegion(fs, wals.getWAL(info), conf, htd, null); region.initialize(); @@ -188,8 +190,7 @@ public class TestStoreFileRefresherChore { when(regionServer.getOnlineRegionsLocalContext()).thenReturn(regions); when(regionServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration()); - HTableDescriptor htd = getTableDesc(TableName.valueOf(name.getMethodName()), families); - htd.setRegionReplication(2); + TableDescriptor htd = getTableDesc(TableName.valueOf(name.getMethodName()), 2, families); HRegion primary = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 0); HRegion replica1 = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 1); regions.add(primary); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java index 99db2087e85..a5148b3f3fc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java @@ -119,7 +119,7 @@ public class TestWALMonotonicallyIncreasingSeqId { wals = new WALFactory(walConf, null, "log_" + replicaId); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); HRegion region = HRegion.createHRegion(info, TEST_UTIL.getDefaultRootDirPath(), conf, htd, - wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace())); + wals.getWAL(info)); return region; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index 264b4c82195..37c3b37e564 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -160,12 +159,12 @@ public abstract class AbstractTestLogRolling { /** * Tests that log rolling doesn't hang when no data is written. */ - @Test(timeout=120000) + @Test(timeout = 120000) public void testLogRollOnNothingWritten() throws Exception { final Configuration conf = TEST_UTIL.getConfiguration(); - final WALFactory wals = new WALFactory(conf, null, - ServerName.valueOf("test.com",8080, 1).toString()); - final WAL newLog = wals.getWAL(new byte[]{}, null); + final WALFactory wals = + new WALFactory(conf, null, ServerName.valueOf("test.com", 8080, 1).toString()); + final WAL newLog = wals.getWAL(null); try { // Now roll the log before we write anything. newLog.rollWriter(true); 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 057b9bfd348..68fa33f47ee 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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.wal; import static org.junit.Assert.assertEquals; @@ -23,20 +22,21 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Arrays; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl; @@ -52,8 +52,10 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; @@ -63,7 +65,7 @@ import org.junit.runners.Parameterized.Parameters; * Tests for WAL write durability */ @RunWith(Parameterized.class) -@Category({RegionServerTests.class, MediumTests.class}) +@Category({ RegionServerTests.class, MediumTests.class }) public class TestDurability { private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static FileSystem FS; @@ -78,6 +80,9 @@ public class TestDurability { @Parameter public String walProvider; + @Rule + public TestName name = new TestName(); + @Parameters(name = "{index}: provider={0}") public static Iterable data() { return Arrays.asList(new Object[] { "defaultProvider" }, new Object[] { "asyncfs" }); @@ -111,12 +116,12 @@ public class TestDurability { @Test public void testDurability() throws Exception { - final WALFactory wals = new WALFactory(CONF, null, ServerName.valueOf("TestDurability", - 16010, System.currentTimeMillis()).toString()); - byte[] tableName = Bytes.toBytes("TestDurability"); - final WAL wal = wals.getWAL(tableName, null); - HRegion region = createHRegion(tableName, "region", wal, Durability.USE_DEFAULT); - HRegion deferredRegion = createHRegion(tableName, "deferredRegion", wal, Durability.ASYNC_WAL); + WALFactory wals = new WALFactory(CONF, null, + ServerName.valueOf("TestDurability", 16010, System.currentTimeMillis()).toString()); + HRegion region = createHRegion(wals, Durability.USE_DEFAULT); + WAL wal = region.getWAL(); + HRegion deferredRegion = createHRegion(region.getTableDescriptor(), region.getRegionInfo(), + "deferredRegion", wal, Durability.ASYNC_WAL); region.put(newPut(null)); verifyWALCount(wals, wal, 1); @@ -175,11 +180,10 @@ public class TestDurability { byte[] col3 = Bytes.toBytes("col3"); // Setting up region - final WALFactory wals = new WALFactory(CONF, null, + WALFactory wals = new WALFactory(CONF, null, ServerName.valueOf("TestIncrement", 16010, System.currentTimeMillis()).toString()); - byte[] tableName = Bytes.toBytes("TestIncrement"); - final WAL wal = wals.getWAL(tableName, null); - HRegion region = createHRegion(tableName, "increment", wal, Durability.USE_DEFAULT); + HRegion region = createHRegion(wals, Durability.USE_DEFAULT); + WAL wal = region.getWAL(); // col1: amount = 0, 1 write back to WAL Increment inc1 = new Increment(row1); @@ -231,8 +235,8 @@ public class TestDurability { assertEquals(3, Bytes.toLong(res.getValue(FAMILY, col3))); verifyWALCount(wals, wal, 4); } - - /* + + /** * Test when returnResults set to false in increment it should not return the result instead it * resturn null. */ @@ -242,12 +246,11 @@ public class TestDurability { byte[] col1 = Bytes.toBytes("col1"); // Setting up region - final WALFactory wals = new WALFactory(CONF, null, - ServerName.valueOf("testIncrementWithReturnResultsSetToFalse", 16010, - System.currentTimeMillis()).toString()); - byte[] tableName = Bytes.toBytes("testIncrementWithReturnResultsSetToFalse"); - final WAL wal = wals.getWAL(tableName, null); - HRegion region = createHRegion(tableName, "increment", wal, Durability.USE_DEFAULT); + WALFactory wals = new WALFactory(CONF, null, + ServerName + .valueOf("testIncrementWithReturnResultsSetToFalse", 16010, System.currentTimeMillis()) + .toString()); + HRegion region = createHRegion(wals, Durability.USE_DEFAULT); Increment inc1 = new Increment(row1); inc1.setReturnResults(false); @@ -270,28 +273,38 @@ public class TestDurability { WAL.Reader reader = wals.createReader(FS, walPath); int count = 0; WAL.Entry entry = new WAL.Entry(); - while (reader.next(entry) != null) count++; + while (reader.next(entry) != null) { + count++; + } reader.close(); assertEquals(expected, count); } // lifted from TestAtomicOperation - private HRegion createHRegion (byte [] tableName, String callingMethod, - WAL log, Durability durability) - throws IOException { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); - htd.setDurability(durability); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); - htd.addFamily(hcd); - HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - Path path = new Path(DIR + callingMethod); - if (FS.exists(path)) { - if (!FS.delete(path, true)) { - throw new IOException("Failed delete of " + path); - } + private HRegion createHRegion(WALFactory wals, Durability durability) throws IOException { + TableName tableName = TableName.valueOf(name.getMethodName().replaceAll("[^A-Za-z0-9-_]", "_")); + TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); + RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); + Path path = new Path(DIR, tableName.getNameAsString()); + if (FS.exists(path)) { + if (!FS.delete(path, true)) { + throw new IOException("Failed delete of " + path); } - ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); - return HRegion.createHRegion(info, path, CONF, htd, log); } + ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); + return HRegion.createHRegion(info, path, CONF, htd, wals.getWAL(info)); + } + private HRegion createHRegion(TableDescriptor td, RegionInfo info, String dir, WAL wal, + Durability durability) throws IOException { + Path path = new Path(DIR, dir); + if (FS.exists(path)) { + if (!FS.delete(path, true)) { + throw new IOException("Failed delete of " + path); + } + } + ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); + return HRegion.createHRegion(info, path, CONF, td, wal); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 6c2fd9f4fd1..57679c40585 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -21,22 +21,23 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.NavigableMap; import java.util.TreeMap; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; 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.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; @@ -136,11 +137,11 @@ public class TestLogRollAbort { // Create the test table and open it TableName tableName = TableName.valueOf(this.getClass().getSimpleName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); admin.createTable(desc); - Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(tableName); try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); WAL log = server.getWAL(null); @@ -189,32 +190,26 @@ public class TestLogRollAbort { // put some entries in an WAL TableName tableName = TableName.valueOf(this.getClass().getName()); - HRegionInfo regioninfo = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - final WAL log = wals.getWAL(regioninfo.getEncodedNameAsBytes(), - regioninfo.getTable().getNamespace()); + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build(); + WAL log = wals.getWAL(regionInfo); MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); - final int total = 20; + int total = 20; for (int i = 0; i < total; i++) { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor("column")); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { - scopes.put(fam, 0); - } - log.append(regioninfo, new WALKeyImpl(regioninfo.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis(), mvcc, scopes), kvs, true); + scopes.put(Bytes.toBytes("column"), 0); + log.append(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName, + System.currentTimeMillis(), mvcc, scopes), + kvs, true); } // Send the data to HDFS datanodes and close the HDFS writer log.sync(); ((AbstractFSWAL) log).replaceWriter(((FSHLog)log).getOldPath(), null, null); - /* code taken from MasterFileSystem.getLogDirs(), which is called from MasterFileSystem.splitLog() - * handles RS shutdowns (as observed by the splitting process) - */ + // code taken from MasterFileSystem.getLogDirs(), which is called from + // MasterFileSystem.splitLog() handles RS shutdowns (as observed by the splitting process) // rename the directory so a rogue RS doesn't create more WALs Path rsSplitDir = thisTestsDir.suffix(AbstractFSWALProvider.SPLITTING_EXT); if (!fs.rename(thisTestsDir, rsSplitDir)) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index 054910056a4..2766b319802 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -18,20 +18,21 @@ package org.apache.hadoop.hbase.regionserver.wal; import static org.junit.Assert.assertFalse; + import java.io.IOException; import java.util.NavigableMap; import java.util.TreeMap; import java.util.concurrent.ThreadLocalRandom; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -89,8 +90,8 @@ public class TestLogRollingNoCluster { FSUtils.setRootDir(conf, dir); conf.set("hbase.regionserver.hlog.writer.impl", HighLatencySyncWriter.class.getName()); final WALFactory wals = new WALFactory(conf, null, TestLogRollingNoCluster.class.getName()); - final WAL wal = wals.getWAL(new byte[]{}, null); - + final WAL wal = wals.getWAL(null); + Appender [] appenders = null; final int numThreads = NUM_THREADS; @@ -157,10 +158,10 @@ public class TestLogRollingNoCluster { WALEdit edit = new WALEdit(); byte[] bytes = Bytes.toBytes(i); edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY)); - final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO; - final HTableDescriptor htd = TEST_UTIL.getMetaTableDescriptor(); + RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO; + TableDescriptor htd = TEST_UTIL.getMetaTableDescriptorBuilder().build(); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + for(byte[] fam : htd.getColumnFamilyNames()) { scopes.put(fam, 0); } final long txid = wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java index 6696ce3fbdc..380ea7429d3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java @@ -24,17 +24,15 @@ import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; 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.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -100,23 +98,20 @@ public class TestWALActionsListener { list.add(observer); final WALFactory wals = new WALFactory(conf, list, "testActionListener"); DummyWALActionsListener laterobserver = new DummyWALActionsListener(); - HRegionInfo hri = new HRegionInfo(TableName.valueOf(SOME_BYTES), - SOME_BYTES, SOME_BYTES, false); - final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()); + RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(SOME_BYTES)) + .setStartKey(SOME_BYTES).setEndKey(SOME_BYTES).build(); + final WAL wal = wals.getWAL(hri); MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); for (int i = 0; i < 20; i++) { byte[] b = Bytes.toBytes(i + ""); - KeyValue kv = new KeyValue(b,b,b); + KeyValue kv = new KeyValue(b, b, b); WALEdit edit = new WALEdit(); edit.add(kv); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(SOME_BYTES)); - htd.addFamily(new HColumnDescriptor(b)); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { - scopes.put(fam, 0); - } - final long txid = wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), - TableName.valueOf(b), 0, mvcc, scopes), edit, true); + scopes.put(b, 0); + long txid = wal.append(hri, + new WALKeyImpl(hri.getEncodedNameAsBytes(), TableName.valueOf(b), 0, mvcc, scopes), edit, + true); wal.sync(txid); if (i == 10) { wal.registerWALActionsListener(laterobserver); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index 27e4cf1fb8e..057c64d2664 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -261,7 +261,7 @@ public abstract class TestReplicationSourceManager { listeners.add(replication); final WALFactory wals = new WALFactory(utility.getConfiguration(), listeners, URLEncoder.encode("regionserver:60020", "UTF8")); - final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()); + final WAL wal = wals.getWAL(hri); manager.init(); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame")); htd.addFamily(new HColumnDescriptor(f1)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java index 6f9c2d3499c..43140bc9a50 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java @@ -1,5 +1,4 @@ -/* - * +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -35,16 +34,16 @@ import java.util.OptionalLong; import java.util.TreeMap; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.replication.WALEntryFilter; @@ -78,8 +77,8 @@ public class TestWALEntryStream { private static final TableName tableName = TableName.valueOf("tablename"); private static final byte[] family = Bytes.toBytes("column"); private static final byte[] qualifier = Bytes.toBytes("qualifier"); - private static final HRegionInfo info = - new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.LAST_ROW, false); + private static final RegionInfo info = RegionInfoBuilder.newBuilder(tableName) + .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.LAST_ROW).build(); private static final NavigableMap scopes = getScopes(); private static NavigableMap getScopes() { @@ -118,7 +117,7 @@ public class TestWALEntryStream { pathWatcher = new PathWatcher(); listeners.add(pathWatcher); final WALFactory wals = new WALFactory(conf, listeners, tn.getMethodName()); - log = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()); + log = wals.getWAL(info); } @After diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java index bfd18cf5f4a..54495179564 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java @@ -26,47 +26,45 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.client.RegionInfo; // imports for things that haven't moved from regionserver.wal yet. import org.apache.hadoop.hbase.regionserver.wal.FSHLog; import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * A WAL Provider that returns a single thread safe WAL that optionally can skip parts of our - * normal interactions with HDFS. - * - * This implementation picks a directory in HDFS based on the same mechanisms as the - * {@link FSHLogProvider}. Users can configure how much interaction - * we have with HDFS with the configuration property "hbase.wal.iotestprovider.operations". - * The value should be a comma separated list of allowed operations: + * A WAL Provider that returns a single thread safe WAL that optionally can skip parts of our normal + * interactions with HDFS. + *

+ * This implementation picks a directory in HDFS based on the same mechanisms as the + * {@link FSHLogProvider}. Users can configure how much interaction we have with HDFS with the + * configuration property "hbase.wal.iotestprovider.operations". The value should be a comma + * separated list of allowed operations: *

    - *
  • append : edits will be written to the underlying filesystem - *
  • sync : wal syncs will result in hflush calls - *
  • fileroll : roll requests will result in creating a new file on the underlying - * filesystem. + *
  • append : edits will be written to the underlying filesystem
  • + *
  • sync : wal syncs will result in hflush calls
  • + *
  • fileroll : roll requests will result in creating a new file on the underlying + * filesystem.
  • *
- * Additionally, the special cases "all" and "none" are recognized. - * If ommited, the value defaults to "all." - * Behavior is undefined if "all" or "none" are paired with additional values. Behavior is also - * undefined if values not listed above are included. - * - * Only those operations listed will occur between the returned WAL and HDFS. All others - * will be no-ops. - * + * Additionally, the special cases "all" and "none" are recognized. If ommited, the value defaults + * to "all." Behavior is undefined if "all" or "none" are paired with additional values. Behavior is + * also undefined if values not listed above are included. + *

+ * Only those operations listed will occur between the returned WAL and HDFS. All others will be + * no-ops. + *

* Note that in the case of allowing "append" operations but not allowing "fileroll", the returned * WAL will just keep writing to the same file. This won't avoid all costs associated with file * management over time, becaue the data set size may result in additional HDFS block allocations. - * */ @InterfaceAudience.Private public class IOTestProvider implements WALProvider { @@ -114,7 +112,7 @@ public class IOTestProvider implements WALProvider { } @Override - public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException { + public WAL getWAL(RegionInfo region) throws IOException { return log; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java index be65ba3b501..3cf65c59799 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java @@ -20,23 +20,24 @@ package org.apache.hadoop.hbase.wal; import static org.apache.hadoop.hbase.wal.BoundedGroupingStrategy.DEFAULT_NUM_REGION_GROUPS; import static org.apache.hadoop.hbase.wal.BoundedGroupingStrategy.NUM_REGION_GROUPS; -import static org.apache.hadoop.hbase.wal.RegionGroupingProvider.*; +import static org.apache.hadoop.hbase.wal.RegionGroupingProvider.DELEGATE_PROVIDER; +import static org.apache.hadoop.hbase.wal.RegionGroupingProvider.REGION_GROUPING_STRATEGY; import static org.apache.hadoop.hbase.wal.WALFactory.WAL_PROVIDER; import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.Arrays; import java.util.HashSet; -import java.util.Random; import java.util.Set; - +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.junit.After; @@ -179,12 +180,13 @@ public class TestBoundedRegionGroupingStrategy { FSUtils.setRootDir(CONF, TEST_UTIL.getDataTestDirOnTestFS()); wals = new WALFactory(CONF, null, "setMembershipDedups"); - final Set seen = new HashSet<>(temp * 4); - final Random random = new Random(); + Set seen = new HashSet<>(temp * 4); int count = 0; // we know that this should see one of the wals more than once for (int i = 0; i < temp * 8; i++) { - final WAL maybeNewWAL = wals.getWAL(Bytes.toBytes(random.nextInt()), null); + WAL maybeNewWAL = wals.getWAL(RegionInfoBuilder + .newBuilder(TableName.valueOf("Table-" + ThreadLocalRandom.current().nextInt())) + .build()); LOG.info("Iteration " + i + ", checking wal " + maybeNewWAL); if (seen.add(maybeNewWAL)) { count++; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java index da2965e5ce8..d9ee9eb3922 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java @@ -26,29 +26,28 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.HashSet; import java.util.NavigableMap; -import java.util.Random; import java.util.Set; import java.util.TreeMap; - +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; 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.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; // imports for things that haven't moved from regionserver.wal yet. import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; -import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -63,10 +62,10 @@ import org.slf4j.LoggerFactory; public class TestFSHLogProvider { private static final Logger LOG = LoggerFactory.getLogger(TestFSHLogProvider.class); - protected static Configuration conf; - protected static FileSystem fs; - protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - protected MultiVersionConcurrencyControl mvcc; + private static Configuration conf; + private static FileSystem fs; + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private MultiVersionConcurrencyControl mvcc; @Rule public final TestName currentTest = new TestName(); @@ -80,10 +79,6 @@ public class TestFSHLogProvider { } } - @After - public void tearDown() throws Exception { - } - @BeforeClass public static void setUpBeforeClass() throws Exception { // Make block sizes small. @@ -149,15 +144,15 @@ public class TestFSHLogProvider { } - protected void addEdits(WAL log, HRegionInfo hri, HTableDescriptor htd, - int times, NavigableMap scopes) throws IOException { + private void addEdits(WAL log, RegionInfo hri, TableDescriptor htd, int times, + NavigableMap scopes) throws IOException { final byte[] row = Bytes.toBytes("row"); for (int i = 0; i < times; i++) { long timestamp = System.currentTimeMillis(); WALEdit cols = new WALEdit(); cols.add(new KeyValue(row, row, row, timestamp, row)); log.append(hri, getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp, scopes), - cols, true); + cols, true); } log.sync(); } @@ -181,37 +176,31 @@ public class TestFSHLogProvider { wal.completeCacheFlush(regionEncodedName); } - private static final byte[] UNSPECIFIED_REGION = new byte[]{}; - @Test public void testLogCleaning() throws Exception { LOG.info(currentTest.getMethodName()); - final HTableDescriptor htd = - new HTableDescriptor(TableName.valueOf(currentTest.getMethodName())).addFamily(new HColumnDescriptor( - "row")); - final HTableDescriptor htd2 = - new HTableDescriptor(TableName.valueOf(currentTest.getMethodName() + "2")) - .addFamily(new HColumnDescriptor("row")); - NavigableMap scopes1 = new TreeMap<>( - Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName())) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build(); + TableDescriptor htd2 = + TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName() + "2")) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build(); + NavigableMap scopes1 = new TreeMap<>(Bytes.BYTES_COMPARATOR); + for (byte[] fam : htd.getColumnFamilyNames()) { scopes1.put(fam, 0); } - NavigableMap scopes2 = new TreeMap<>( - Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd2.getFamiliesKeys()) { + NavigableMap scopes2 = new TreeMap<>(Bytes.BYTES_COMPARATOR); + for (byte[] fam : htd2.getColumnFamilyNames()) { scopes2.put(fam, 0); } - final Configuration localConf = new Configuration(conf); + Configuration localConf = new Configuration(conf); localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName()); - final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName()); + WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName()); try { - HRegionInfo hri = new HRegionInfo(htd.getTableName(), - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - HRegionInfo hri2 = new HRegionInfo(htd2.getTableName(), - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); + RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build(); + RegionInfo hri2 = RegionInfoBuilder.newBuilder(htd2.getTableName()).build(); // we want to mix edits from regions, so pick our own identifier. - final WAL log = wals.getWAL(UNSPECIFIED_REGION, null); + WAL log = wals.getWAL(null); // Add a single edit and make sure that rolling won't remove the file // Before HBASE-3198 it used to delete it @@ -235,7 +224,7 @@ public class TestFSHLogProvider { // Flush the first region, we expect to see the first two files getting // archived. We need to append something or writer won't be rolled. addEdits(log, hri2, htd2, 1, scopes2); - log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getFamiliesKeys()); + log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getColumnFamilyNames()); log.completeCacheFlush(hri.getEncodedNameAsBytes()); log.rollWriter(); assertEquals(2, AbstractFSWALProvider.getNumRolledLogFiles(log)); @@ -244,7 +233,7 @@ public class TestFSHLogProvider { // since the oldest was completely flushed and the two others only contain // flush information addEdits(log, hri2, htd2, 1, scopes2); - log.startCacheFlush(hri2.getEncodedNameAsBytes(), htd2.getFamiliesKeys()); + log.startCacheFlush(hri2.getEncodedNameAsBytes(), htd2.getColumnFamilyNames()); log.completeCacheFlush(hri2.getEncodedNameAsBytes()); log.rollWriter(); assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(log)); @@ -270,35 +259,28 @@ public class TestFSHLogProvider { @Test public void testWALArchiving() throws IOException { LOG.debug(currentTest.getMethodName()); - HTableDescriptor table1 = - new HTableDescriptor(TableName.valueOf(currentTest.getMethodName() + "1")).addFamily(new HColumnDescriptor("row")); - HTableDescriptor table2 = - new HTableDescriptor(TableName.valueOf(currentTest.getMethodName() + "2")).addFamily(new HColumnDescriptor("row")); - NavigableMap scopes1 = new TreeMap<>( - Bytes.BYTES_COMPARATOR); - for(byte[] fam : table1.getFamiliesKeys()) { + TableDescriptor table1 = + TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName() + "1")) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build(); + TableDescriptor table2 = + TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName() + "2")) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build(); + NavigableMap scopes1 = new TreeMap<>(Bytes.BYTES_COMPARATOR); + for (byte[] fam : table1.getColumnFamilyNames()) { scopes1.put(fam, 0); } - NavigableMap scopes2 = new TreeMap<>( - Bytes.BYTES_COMPARATOR); - for(byte[] fam : table2.getFamiliesKeys()) { + NavigableMap scopes2 = new TreeMap<>(Bytes.BYTES_COMPARATOR); + for (byte[] fam : table2.getColumnFamilyNames()) { scopes2.put(fam, 0); } - final Configuration localConf = new Configuration(conf); + Configuration localConf = new Configuration(conf); localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName()); - final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName()); + WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName()); try { - final WAL wal = wals.getWAL(UNSPECIFIED_REGION, null); + WAL wal = wals.getWAL(null); assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(wal)); - HRegionInfo hri1 = - new HRegionInfo(table1.getTableName(), HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW); - HRegionInfo hri2 = - new HRegionInfo(table2.getTableName(), HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW); - // ensure that we don't split the regions. - hri1.setSplit(false); - hri2.setSplit(false); + RegionInfo hri1 = RegionInfoBuilder.newBuilder(table1.getTableName()).build(); + RegionInfo hri2 = RegionInfoBuilder.newBuilder(table2.getTableName()).build(); // variables to mock region sequenceIds. // start with the testing logic: insert a waledit, and roll writer addEdits(wal, hri1, table1, 1, scopes1); @@ -312,7 +294,7 @@ public class TestFSHLogProvider { assertEquals(2, AbstractFSWALProvider.getNumRolledLogFiles(wal)); // add a waledit to table1, and flush the region. addEdits(wal, hri1, table1, 3, scopes1); - flushRegion(wal, hri1.getEncodedNameAsBytes(), table1.getFamiliesKeys()); + flushRegion(wal, hri1.getEncodedNameAsBytes(), table1.getColumnFamilyNames()); // roll log; all old logs should be archived. wal.rollWriter(); assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(wal)); @@ -326,7 +308,7 @@ public class TestFSHLogProvider { assertEquals(2, AbstractFSWALProvider.getNumRolledLogFiles(wal)); // add edits for table2, and flush hri1. addEdits(wal, hri2, table2, 2, scopes2); - flushRegion(wal, hri1.getEncodedNameAsBytes(), table2.getFamiliesKeys()); + flushRegion(wal, hri1.getEncodedNameAsBytes(), table2.getColumnFamilyNames()); // the log : region-sequenceId map is // log1: region2 (unflushed) // log2: region1 (flushed) @@ -336,7 +318,7 @@ public class TestFSHLogProvider { assertEquals(2, AbstractFSWALProvider.getNumRolledLogFiles(wal)); // flush region2, and all logs should be archived. addEdits(wal, hri2, table2, 2, scopes2); - flushRegion(wal, hri2.getEncodedNameAsBytes(), table2.getFamiliesKeys()); + flushRegion(wal, hri2.getEncodedNameAsBytes(), table2.getColumnFamilyNames()); wal.rollWriter(); assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(wal)); } finally { @@ -365,18 +347,20 @@ public class TestFSHLogProvider { */ @Test public void setMembershipDedups() throws IOException { - final Configuration localConf = new Configuration(conf); + Configuration localConf = new Configuration(conf); localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName()); - final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName()); + WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName()); try { final Set seen = new HashSet<>(1); - final Random random = new Random(); assertTrue("first attempt to add WAL from default provider should work.", - seen.add(wals.getWAL(Bytes.toBytes(random.nextInt()), null))); + seen.add(wals.getWAL(null))); for (int i = 0; i < 1000; i++) { - assertFalse("default wal provider is only supposed to return a single wal, which should " - + "compare as .equals itself.", - seen.add(wals.getWAL(Bytes.toBytes(random.nextInt()), null))); + assertFalse( + "default wal provider is only supposed to return a single wal, which should " + + "compare as .equals itself.", + seen.add(wals.getWAL(RegionInfoBuilder + .newBuilder(TableName.valueOf("Table-" + ThreadLocalRandom.current().nextInt())) + .build()))); } } finally { wals.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java index 0814fcb6c0d..b4160e99aa2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java @@ -1,4 +1,4 @@ -/* +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -25,7 +25,6 @@ import java.util.Arrays; import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -33,12 +32,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; 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.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.SecureAsyncProtobufLogWriter; @@ -106,14 +104,9 @@ public class TestSecureWAL { @Test public void testSecureWAL() throws Exception { TableName tableName = TableName.valueOf(name.getMethodName().replaceAll("[^a-zA-Z0-9]", "_")); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(tableName.getName())); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { - scopes.put(fam, 0); - } - HRegionInfo regioninfo = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false); + scopes.put(tableName.getName(), 0); + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build(); final int total = 10; final byte[] row = Bytes.toBytes("row"); final byte[] family = Bytes.toBytes("family"); @@ -123,15 +116,14 @@ public class TestSecureWAL { new WALFactory(TEST_UTIL.getConfiguration(), null, tableName.getNameAsString()); // Write the WAL - final WAL wal = - wals.getWAL(regioninfo.getEncodedNameAsBytes(), regioninfo.getTable().getNamespace()); + final WAL wal = wals.getWAL(regionInfo); MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); for (int i = 0; i < total; i++) { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value)); - wal.append(regioninfo, new WALKeyImpl(regioninfo.getEncodedNameAsBytes(), tableName, + wal.append(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), mvcc, scopes), kvs, true); } wal.sync(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java index 2c19c12965a..f0f35e75dea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java @@ -1,5 +1,4 @@ -/* - * +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -30,7 +29,6 @@ import java.net.BindException; import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -41,14 +39,15 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.Coprocessor; 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.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.SampleRegionWALCoprocessor; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; @@ -169,27 +168,24 @@ public class TestWALFactory { final byte [] rowName = tableName.getName(); final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); final int howmany = 3; - HRegionInfo[] infos = new HRegionInfo[3]; + RegionInfo[] infos = new RegionInfo[3]; Path tabledir = FSUtils.getTableDir(hbaseWALDir, tableName); fs.mkdirs(tabledir); - for(int i = 0; i < howmany; i++) { - infos[i] = new HRegionInfo(tableName, - Bytes.toBytes("" + i), Bytes.toBytes("" + (i+1)), false); + for (int i = 0; i < howmany; i++) { + infos[i] = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("" + i)) + .setEndKey(Bytes.toBytes("" + (i + 1))).build(); fs.mkdirs(new Path(tabledir, infos[i].getEncodedName())); LOG.info("allo " + new Path(tabledir, infos[i].getEncodedName()).toString()); } - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor("column")); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { - scopes.put(fam, 0); - } + scopes.put(Bytes.toBytes("column"), 0); + // Add edits for three regions. for (int ii = 0; ii < howmany; ii++) { for (int i = 0; i < howmany; i++) { final WAL log = - wals.getWAL(infos[i].getEncodedNameAsBytes(), infos[i].getTable().getNamespace()); + wals.getWAL(infos[i]); for (int j = 0; j < howmany; j++) { WALEdit edit = new WALEdit(); byte [] family = Bytes.toBytes("column"); @@ -254,15 +250,10 @@ public class TestWALFactory { WAL.Reader reader = null; try { - HRegionInfo info = new HRegionInfo(tableName, - null,null, false); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(tableName.getName())); + RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { - scopes.put(fam, 0); - } - final WAL wal = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()); + scopes.put(tableName.getName(), 0); + final WAL wal = wals.getWAL(info); for (int i = 0; i < total; i++) { WALEdit kvs = new WALEdit(); @@ -374,24 +365,18 @@ public class TestWALFactory { public void testAppendClose() throws Exception { TableName tableName = TableName.valueOf(currentTest.getMethodName()); - HRegionInfo regioninfo = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false); + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build(); - final WAL wal = - wals.getWAL(regioninfo.getEncodedNameAsBytes(), regioninfo.getTable().getNamespace()); - final int total = 20; + WAL wal = wals.getWAL(regionInfo); + int total = 20; - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(tableName.getName())); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { - scopes.put(fam, 0); - } + scopes.put(tableName.getName(), 0); MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); for (int i = 0; i < total; i++) { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); - wal.append(regioninfo, new WALKeyImpl(regioninfo.getEncodedNameAsBytes(), tableName, + wal.append(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), mvcc, scopes), kvs, true); } @@ -496,20 +481,18 @@ public class TestWALFactory { /** * Tests that we can write out an edit, close, and then read it back in again. - * @throws IOException */ @Test public void testEditAdd() throws IOException { - final int COL_COUNT = 10; - final HTableDescriptor htd = - new HTableDescriptor(TableName.valueOf(currentTest.getMethodName())).addFamily(new HColumnDescriptor( - "column")); - NavigableMap scopes = new TreeMap( - Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + int colCount = 10; + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName())) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("column")).build(); + NavigableMap scopes = new TreeMap(Bytes.BYTES_COMPARATOR); + for (byte[] fam : htd.getColumnFamilyNames()) { scopes.put(fam, 0); } - final byte [] row = Bytes.toBytes("row"); + byte[] row = Bytes.toBytes("row"); WAL.Reader reader = null; try { final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); @@ -518,21 +501,21 @@ public class TestWALFactory { // 1, 2, 3... long timestamp = System.currentTimeMillis(); WALEdit cols = new WALEdit(); - for (int i = 0; i < COL_COUNT; i++) { + for (int i = 0; i < colCount; i++) { cols.add(new KeyValue(row, Bytes.toBytes("column"), Bytes.toBytes(Integer.toString(i)), timestamp, new byte[] { (byte)(i + '0') })); } - HRegionInfo info = new HRegionInfo(htd.getTableName(), - row,Bytes.toBytes(Bytes.toString(row) + "1"), false); - final WAL log = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()); + RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(row) + .setEndKey(Bytes.toBytes(Bytes.toString(row) + "1")).build(); + final WAL log = wals.getWAL(info); final long txid = log.append(info, new WALKeyImpl(info.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis(), mvcc, scopes), cols, true); log.sync(txid); - log.startCacheFlush(info.getEncodedNameAsBytes(), htd.getFamiliesKeys()); + log.startCacheFlush(info.getEncodedNameAsBytes(), htd.getColumnFamilyNames()); log.completeCacheFlush(info.getEncodedNameAsBytes()); log.shutdown(); Path filename = AbstractFSWALProvider.getCurrentFileName(log); @@ -560,21 +543,17 @@ public class TestWALFactory { } } - /** - * @throws IOException - */ @Test public void testAppend() throws IOException { - final int COL_COUNT = 10; - final HTableDescriptor htd = - new HTableDescriptor(TableName.valueOf(currentTest.getMethodName())).addFamily(new HColumnDescriptor( - "column")); - NavigableMap scopes = new TreeMap( - Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + int colCount = 10; + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName())) + .addColumnFamily(ColumnFamilyDescriptorBuilder.of("column")).build(); + NavigableMap scopes = new TreeMap(Bytes.BYTES_COMPARATOR); + for (byte[] fam : htd.getColumnFamilyNames()) { scopes.put(fam, 0); } - final byte [] row = Bytes.toBytes("row"); + byte[] row = Bytes.toBytes("row"); WAL.Reader reader = null; final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); try { @@ -582,27 +561,26 @@ public class TestWALFactory { // 1, 2, 3... long timestamp = System.currentTimeMillis(); WALEdit cols = new WALEdit(); - for (int i = 0; i < COL_COUNT; i++) { + for (int i = 0; i < colCount; i++) { cols.add(new KeyValue(row, Bytes.toBytes("column"), Bytes.toBytes(Integer.toString(i)), timestamp, new byte[] { (byte)(i + '0') })); } - HRegionInfo hri = new HRegionInfo(htd.getTableName(), - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - final WAL log = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()); + RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build(); + final WAL log = wals.getWAL(hri); final long txid = log.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis(), mvcc, scopes), cols, true); log.sync(txid); - log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getFamiliesKeys()); + log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getColumnFamilyNames()); log.completeCacheFlush(hri.getEncodedNameAsBytes()); log.shutdown(); Path filename = AbstractFSWALProvider.getCurrentFileName(log); // Now open a reader on the log and assert append worked. reader = wals.createReader(fs, filename); WAL.Entry entry = reader.next(); - assertEquals(COL_COUNT, entry.getEdit().size()); + assertEquals(colCount, entry.getEdit().size()); int idx = 0; for (Cell val : entry.getEdit().getCells()) { assertTrue(Bytes.equals(hri.getEncodedNameAsBytes(), @@ -633,15 +611,11 @@ public class TestWALFactory { final DumbWALActionsListener visitor = new DumbWALActionsListener(); final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); long timestamp = System.currentTimeMillis(); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor("column")); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { - scopes.put(fam, 0); - } - HRegionInfo hri = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - final WAL log = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()); + scopes.put(Bytes.toBytes("column"), 0); + + RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build(); + final WAL log = wals.getWAL(hri); log.registerWALActionsListener(visitor); for (int i = 0; i < COL_COUNT; i++) { WALEdit cols = new WALEdit(); @@ -670,7 +644,7 @@ public class TestWALFactory { @Test public void testWALCoprocessorLoaded() throws Exception { // test to see whether the coprocessor is loaded or not. - WALCoprocessorHost host = wals.getWAL(UNSPECIFIED_REGION, null).getCoprocessorHost(); + WALCoprocessorHost host = wals.getWAL(null).getCoprocessorHost(); Coprocessor c = host.findCoprocessor(SampleRegionWALCoprocessor.class); assertNotNull(c); } @@ -690,7 +664,4 @@ public class TestWALFactory { increments++; } } - - private static final byte[] UNSPECIFIED_REGION = new byte[]{}; - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java index a6d245698f9..994f008b5d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java @@ -99,7 +99,7 @@ public class TestWALMethods { final Configuration walConf = new Configuration(util.getConfiguration()); FSUtils.setRootDir(walConf, regiondir); - (new WALFactory(walConf, null, "dummyLogName")).getWAL(new byte[] {}, null); + (new WALFactory(walConf, null, "dummyLogName")).getWAL(null); NavigableSet files = WALSplitter.getSplitEditFilesSorted(fs, regiondir); assertEquals(7, files.size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java index 3672f9b146d..dcb6f5a46e9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java @@ -1,4 +1,4 @@ -/* +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -25,22 +25,20 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.NavigableMap; import java.util.TreeMap; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -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.KeyValue; import org.apache.hadoop.hbase.ByteBufferKeyValue; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; // imports for things that haven't moved from regionserver.wal yet. @@ -85,7 +83,6 @@ public class TestWALReaderOnSecureWAL { FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir()); } - @SuppressWarnings("deprecation") private Path writeWAL(final WALFactory wals, final String tblName, boolean offheap) throws IOException { Configuration conf = TEST_UTIL.getConfiguration(); String clsName = conf.get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName()); @@ -93,22 +90,16 @@ public class TestWALReaderOnSecureWAL { WALCellCodec.class); try { TableName tableName = TableName.valueOf(tblName); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(tableName.getName())); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { - scopes.put(fam, 0); - } - HRegionInfo regioninfo = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false); + scopes.put(tableName.getName(), 0); + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build(); final int total = 10; final byte[] row = Bytes.toBytes("row"); final byte[] family = Bytes.toBytes("family"); final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); // Write the WAL - WAL wal = - wals.getWAL(regioninfo.getEncodedNameAsBytes(), regioninfo.getTable().getNamespace()); + WAL wal = wals.getWAL(regionInfo); for (int i = 0; i < total; i++) { WALEdit kvs = new WALEdit(); KeyValue kv = new KeyValue(row, family, Bytes.toBytes(i), value); @@ -120,7 +111,7 @@ public class TestWALReaderOnSecureWAL { } else { kvs.add(kv); } - wal.append(regioninfo, new WALKeyImpl(regioninfo.getEncodedNameAsBytes(), tableName, + wal.append(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), mvcc, scopes), kvs, true); } wal.sync(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java index c71fb4b36aa..e46dc21c4a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java @@ -1,4 +1,4 @@ -/* +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -17,17 +17,22 @@ */ package org.apache.hadoop.hbase.wal; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -40,13 +45,6 @@ import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static org.junit.Assert.assertEquals; - @Category(MediumTests.class) public class TestWALRootDir { private static final Logger LOG = LoggerFactory.getLogger(TestWALRootDir.class); @@ -54,10 +52,9 @@ public class TestWALRootDir { private static Configuration conf; private static FileSystem fs; private static FileSystem walFs; - static final TableName tableName = TableName.valueOf("TestWALWALDir"); + private static final TableName tableName = TableName.valueOf("TestWALWALDir"); private static final byte [] rowName = Bytes.toBytes("row"); private static final byte [] family = Bytes.toBytes("column"); - private static HTableDescriptor htd; private static Path walRootDir; private static Path rootDir; private static WALFactory wals; @@ -75,8 +72,6 @@ public class TestWALRootDir { walRootDir = TEST_UTIL.createWALRootDir(); fs = FSUtils.getRootDirFileSystem(conf); walFs = FSUtils.getWALFileSystem(conf); - htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(family)); } @AfterClass @@ -87,9 +82,9 @@ public class TestWALRootDir { @Test public void testWALRootDir() throws Exception { - HRegionInfo regionInfo = new HRegionInfo(tableName); + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build(); wals = new WALFactory(conf, null, "testWALRootDir"); - WAL log = wals.getWAL(regionInfo.getEncodedNameAsBytes(), regionInfo.getTable().getNamespace()); + WAL log = wals.getWAL(regionInfo); assertEquals(1, getWALFiles(walFs, walRootDir).size()); byte [] value = Bytes.toBytes("value"); @@ -117,7 +112,7 @@ public class TestWALRootDir { new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME)).size()); } - protected WALKeyImpl getWalKey(final long time, HRegionInfo hri, final long startPoint) { + private WALKeyImpl getWalKey(final long time, RegionInfo hri, final long startPoint) { return new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, time, new MultiVersionConcurrencyControl(startPoint)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index 556afd97518..a1206aa775a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -23,10 +23,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import org.apache.hbase.thirdparty.com.google.common.base.Joiner; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; - import java.io.FileNotFoundException; import java.io.IOException; import java.lang.reflect.Method; @@ -45,7 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -58,18 +53,16 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.wal.FaultyProtobufLogReader; import org.apache.hadoop.hbase.regionserver.wal.InstrumentedLogWriter; import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader; import org.apache.hadoop.hbase.security.User; -import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -98,6 +91,14 @@ import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Joiner; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; + /** * Testing {@link WAL} splitting code. */ @@ -373,10 +374,10 @@ public class TestWALSplit { */ @Test (timeout=300000) public void testRecoveredEditsPathForMeta() throws IOException { - byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); + byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME); Path regiondir = new Path(tdir, - HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); + RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); long now = System.currentTimeMillis(); Entry entry = @@ -386,7 +387,7 @@ public class TestWALSplit { Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR, FILENAME_BEING_SPLIT); String parentOfParent = p.getParent().getParent().getName(); - assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); + assertEquals(parentOfParent, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); } /** @@ -395,10 +396,10 @@ public class TestWALSplit { */ @Test (timeout=300000) public void testOldRecoveredEditsFileSidelined() throws IOException { - byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); + byte [] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME); Path regiondir = new Path(tdir, - HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); + RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); long now = System.currentTimeMillis(); Entry entry = @@ -412,7 +413,7 @@ public class TestWALSplit { Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR, FILENAME_BEING_SPLIT); String parentOfParent = p.getParent().getParent().getName(); - assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); + assertEquals(parentOfParent, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); WALFactory.createRecoveredEditsWriter(fs, p, conf).close(); } @@ -459,7 +460,7 @@ public class TestWALSplit { @Test (timeout=300000) public void testSplitLeavesCompactionEventsEdits() throws IOException{ - HRegionInfo hri = new HRegionInfo(TABLE_NAME); + RegionInfo hri = RegionInfoBuilder.newBuilder(TABLE_NAME).build(); REGIONS.clear(); REGIONS.add(hri.getEncodedName()); Path regionDir = new Path(FSUtils.getTableDir(HBASEDIR, TABLE_NAME), hri.getEncodedName()); @@ -1132,7 +1133,7 @@ public class TestWALSplit { REGIONS.add(regionName); generateWALs(-1); - wals.getWAL(Bytes.toBytes(regionName), null); + wals.getWAL(null); FileStatus[] logfiles = fs.listStatus(WALDIR); assertTrue("There should be some log file", logfiles != null && logfiles.length > 0); @@ -1337,7 +1338,7 @@ public class TestWALSplit { return count; } - private static void appendCompactionEvent(Writer w, HRegionInfo hri, String[] inputs, + private static void appendCompactionEvent(Writer w, RegionInfo hri, String[] inputs, String output) throws IOException { WALProtos.CompactionDescriptor.Builder desc = WALProtos.CompactionDescriptor.newBuilder(); desc.setTableName(ByteString.copyFrom(hri.getTable().toBytes())) 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 93e63735457..a9bad00272d 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 @@ -20,6 +20,11 @@ package org.apache.hadoop.hbase.wal; import static com.codahale.metrics.MetricRegistry.name; +import com.codahale.metrics.ConsoleReporter; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; import java.io.IOException; import java.util.HashMap; import java.util.HashSet; @@ -30,7 +35,7 @@ import java.util.Random; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.TimeUnit; - +import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileStatus; @@ -39,14 +44,15 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MockRegionServerServices; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.LogRoller; @@ -71,12 +77,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.codahale.metrics.ConsoleReporter; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.MetricFilter; -import com.codahale.metrics.MetricRegistry; - // imports for things that haven't moved from regionserver.wal yet. /** @@ -131,11 +131,10 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { private final boolean noSync; private final HRegion region; private final int syncInterval; - private final HTableDescriptor htd; private final Sampler loopSampler; private final NavigableMap scopes; - WALPutBenchmark(final HRegion region, final HTableDescriptor htd, + WALPutBenchmark(final HRegion region, final TableDescriptor htd, final long numIterations, final boolean noSync, final int syncInterval, final double traceFreq) { this.numIterations = numIterations; @@ -143,9 +142,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { this.syncInterval = syncInterval; this.numFamilies = htd.getColumnFamilyCount(); this.region = region; - this.htd = htd; scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + for(byte[] fam : htd.getColumnFamilyNames()) { scopes.put(fam, 0); } String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes"); @@ -320,7 +318,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { if (rootRegionDir == null) { rootRegionDir = TEST_UTIL.getDataTestDirOnTestFS("WALPerformanceEvaluation"); } - rootRegionDir = rootRegionDir.makeQualified(fs); + rootRegionDir = rootRegionDir.makeQualified(fs.getUri(), fs.getWorkingDirectory()); cleanRegionRootDir(fs, rootRegionDir); FSUtils.setRootDir(getConf(), rootRegionDir); final WALFactory wals = new WALFactory(getConf(), null, "wals"); @@ -334,7 +332,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { for(int i = 0; i < numRegions; i++) { // Initialize Table Descriptor // a table per desired region means we can avoid carving up the key space - final HTableDescriptor htd = createHTableDescriptor(i, numFamilies); + final TableDescriptor htd = createHTableDescriptor(i, numFamilies); regions[i] = openRegion(fs, rootRegionDir, htd, wals, roll, roller); benchmarks[i] = TraceUtil.wrap(new WALPutBenchmark(regions[i], htd, numIterations, noSync, syncInterval, traceFreq), ""); @@ -401,14 +399,14 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { return(0); } - private static HTableDescriptor createHTableDescriptor(final int regionNum, + private static TableDescriptor createHTableDescriptor(final int regionNum, final int numFamilies) { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE_NAME + ":" + regionNum)); - for (int i = 0; i < numFamilies; ++i) { - HColumnDescriptor colDef = new HColumnDescriptor(FAMILY_PREFIX + i); - htd.addFamily(colDef); - } - return htd; + TableDescriptorBuilder builder = + TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE_NAME + ":" + regionNum)); + IntStream.range(0, numFamilies) + .mapToObj(i -> ColumnFamilyDescriptorBuilder.of(FAMILY_PREFIX + i)) + .forEachOrdered(builder::addColumnFamily); + return builder.build(); } /** @@ -495,13 +493,12 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { private final Set walsListenedTo = new HashSet<>(); - private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd, + private HRegion openRegion(final FileSystem fs, final Path dir, final TableDescriptor htd, final WALFactory wals, final long whenToRoll, final LogRoller roller) throws IOException { // Initialize HRegion RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build(); // Initialize WAL - final WAL wal = - wals.getWAL(regionInfo.getEncodedNameAsBytes(), regionInfo.getTable().getNamespace()); + final WAL wal = wals.getWAL(regionInfo); // If we haven't already, attach a listener to this wal to handle rolls and metrics. if (walsListenedTo.add(wal)) { roller.addWAL(wal);