From 731fc3ee401db5a8624dfa40a03bdfa2550e24ae Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 5 Jun 2026 10:10:00 +0800 Subject: [PATCH 1/2] HBASE-30190 Upgrade hbase-server to use junit5 Part21 (#8301) (#8309) (cherry picked from commit 8464d77af403fe19cf17b79034cbc0dac05bcb25) Signed-off-by: Xiao Liu --- .../hadoop/hbase/TableNameTestExtension.java | 4 + .../replication/TestVerifyReplication.java | 1 - .../TestPassCustomCellViaRegionObserver.java | 61 +++---- .../TestFavoredNodeAssignmentHelper.java | 97 ++++++----- .../TestStartcodeAgnosticServerName.java | 17 +- ...FirstKeyValueMatchingQualifiersFilter.java | 25 ++- .../hbase/master/TestMasterNotCarryTable.java | 23 ++- .../TestMirroringTableStateManager.java | 31 ++-- .../hadoop/hbase/master/TestRegionPlan.java | 40 ++--- .../hbase/master/TestTableStateManager.java | 35 ++-- .../balancer/TestCacheAwareLoadBalancer.java | 2 +- .../balancer/TestRegionLocationFinder.java | 31 ++-- .../balancer/TestRegionsOnMasterOptions.java | 45 +++--- .../master/balancer/TestServerAndLoad.java | 17 +- .../hbase/master/cleaner/TestLogsCleaner.java | 6 +- .../hadoop/hbase/mob/MobStressToolRunner.java | 4 +- .../hbase/protobuf/TestProtobufUtil.java | 17 +- ...bstractTestDateTieredCompactionPolicy.java | 2 +- .../regionserver/TestShortCircuitGet.java | 43 ++--- .../hbase/regionserver/TestWALLockup.java | 48 +++--- .../regionserver/wal/TestWALOpenError.java | 53 +++--- .../ReplicationDroppedTablesTestBase.java | 8 +- .../ReplicationKillRSTestBase.java | 9 +- .../SerialReplicationTestBase.java | 35 ++-- .../TestAddToSerialReplicationPeer.java | 19 +-- ...TestBidirectionSerialReplicationStuck.java | 17 +- .../TestBulkLoadReplication.java | 152 ++++++++---------- .../TestBulkLoadReplicationHFileRefs.java | 71 ++++---- .../TestClaimReplicationQueue.java | 39 ++--- .../TestEditsBehindDroppedTableTiming.java | 106 ++++++------ .../TestEditsDroppedWithDroppedTable.java | 17 +- .../TestEditsDroppedWithDroppedTableNS.java | 17 +- .../TestEditsStuckBehindDroppedTable.java | 17 +- .../replication/TestMasterReplication.java | 69 ++++---- ...estMigrateRepliationPeerStorageOnline.java | 25 ++- .../TestMultiSlaveReplication.java | 35 ++-- .../replication/TestNamespaceReplication.java | 56 +++---- ...amespaceReplicationWithBulkLoadedData.java | 122 ++++---------- .../TestNonHBaseReplicationEndpoint.java | 27 ++-- .../TestPerTableCFReplication.java | 70 ++++---- .../TestRemoveFromSerialReplicationPeer.java | 19 +-- .../replication/TestReplicationBase.java | 2 - .../TestReplicationBaseNoBeforeAll.java | 14 +- ...tReplicationChangingPeerRegionservers.java | 50 +++--- .../TestReplicationDisableInactivePeer.java | 17 +- ...cationEditsDroppedWithDeletedTableCFs.java | 32 ++-- ...plicationEditsDroppedWithDroppedTable.java | 32 ++-- .../TestReplicationEmptyWALRecovery.java | 39 +++-- .../TestReplicationKillSlaveRS.java | 3 - .../TestReplicationMetricsforUI.java | 46 +++--- .../TestReplicationProcedureRetry.java | 31 ++-- .../TestReplicationSmallTests.java | 70 ++++---- .../replication/TestReplicationStatus.java | 34 ++-- .../TestReplicationStatusAfterLagging.java | 17 +- ...ionStatusBothNormalAndRecoveryLagging.java | 21 +-- .../TestReplicationStatusSink.java | 21 ++- ...StatusSourceStartedTargetStoppedNewOp.java | 19 +-- ...StatusSourceStartedTargetStoppedNoOps.java | 17 +- ...ourceStartedTargetStoppedWithRecovery.java | 23 ++- ...stReplicationStuckWithDeletedTableCFs.java | 24 ++- .../TestReplicationStuckWithDroppedTable.java | 24 ++- .../TestReplicationWALEntryFilters.java | 111 ++++++------- .../TestReplicationWithFSPeerStorage.java | 26 ++- .../replication/TestReplicationWithTags.java | 26 ++- ...tReplicationWithWALExtendedAttributes.java | 25 ++- .../replication/TestSerialReplication.java | 51 +++--- .../TestSerialReplicationFailover.java | 21 +-- .../TestVerifyCellsReplicationEndpoint.java | 23 ++- .../master/TestTableCFsUpdater.java | 43 +++-- .../hbase/util/HFileArchiveTestingUtil.java | 8 +- .../hadoop/hbase/util/HFileTestUtil.java | 6 +- .../hbase/util/TestServerHttpUtils.java | 2 +- .../hbase/util/hbck/HbckTestingUtil.java | 2 +- .../TestBoundedRegionGroupingStrategy.java | 56 +++---- 74 files changed, 1052 insertions(+), 1416 deletions(-) rename hbase-server/src/test/java/org/apache/hadoop/hbase/{regionserver => replication}/TestBulkLoadReplication.java (80%) rename hbase-server/src/test/java/org/apache/hadoop/hbase/{regionserver => replication}/TestBulkLoadReplicationHFileRefs.java (86%) diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestExtension.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestExtension.java index a628d34f76fc..535e4b0140cd 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestExtension.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestExtension.java @@ -45,6 +45,10 @@ public TableName getTableName() { return tableName; } + public TableName getTableName(String suffix) { + return TableName.valueOf(tableName.getNameAsString() + suffix); + } + @Override public void beforeEach(ExtensionContext context) { tableName = TableName.valueOf(cleanUpTestName(context.getRequiredTestMethod().getName())); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java index 6a75a029f02a..38088f3e4da6 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java @@ -85,7 +85,6 @@ public void setUp() throws Exception { @BeforeAll public static void setUpBeforeClass() throws Exception { - TableDescriptor peerTable = TableDescriptorBuilder.newBuilder(peerTableName) .setColumnFamily( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java index 1e939b0c4e60..6ea6836878c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hbase.coprocessor; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.nio.ByteBuffer; @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValueUtil; @@ -53,25 +52,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WALEdit; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; - -@Category({ CoprocessorTests.class, MediumTests.class }) +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; + +@Tag(CoprocessorTests.TAG) +@Tag(MediumTests.TAG) public class TestPassCustomCellViaRegionObserver { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestPassCustomCellViaRegionObserver.class); - - @Rule - public TestName testName = new TestName(); - private TableName tableName; private Table table = null; @@ -85,22 +76,22 @@ public class TestPassCustomCellViaRegionObserver { private static final byte[] QUALIFIER_FROM_CP = Bytes.toBytes("QUALIFIER_FROM_CP"); - @BeforeClass + @BeforeAll public static void setupBeforeClass() throws Exception { // small retry number can speed up the failed tests. UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); UTIL.startMiniCluster(); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { UTIL.shutdownMiniCluster(); } - @Before - public void clearTable() throws IOException { + @BeforeEach + public void clearTable(TestInfo testInfo) throws IOException { RegionObserverImpl.COUNT.set(0); - tableName = TableName.valueOf(testName.getMethodName()); + tableName = TableName.valueOf(testInfo.getTestMethod().get().getName()); if (table != null) { table.close(); } @@ -149,8 +140,8 @@ public void testMutation() throws Exception { Delete delete = new Delete(ROW); delete.addColumns(FAMILY, QUALIFIER); table.delete(delete); - assertTrue(Arrays.asList(table.get(new Get(ROW)).rawCells()).toString(), - table.get(new Get(ROW)).isEmpty()); + assertTrue(table.get(new Get(ROW)).isEmpty(), + Arrays.asList(table.get(new Get(ROW)).rawCells()).toString()); assertObserverHasExecuted(); assertTrue(table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put)); @@ -186,21 +177,21 @@ private static void assertObserverHasExecuted() { private static void assertResult(Result result, byte[] expectedValue) { assertFalse(result.isEmpty()); for (Cell c : result.rawCells()) { - assertTrue(c.toString(), Bytes.equals(ROW, CellUtil.cloneRow(c))); - assertTrue(c.toString(), Bytes.equals(FAMILY, CellUtil.cloneFamily(c))); - assertTrue(c.toString(), Bytes.equals(expectedValue, CellUtil.cloneValue(c))); + assertTrue(Bytes.equals(ROW, CellUtil.cloneRow(c)), c.toString()); + assertTrue(Bytes.equals(FAMILY, CellUtil.cloneFamily(c)), c.toString()); + assertTrue(Bytes.equals(expectedValue, CellUtil.cloneValue(c)), c.toString()); } } private static void assertResult(Result result, byte[] expectedValue, byte[] expectedFromCp) { assertFalse(result.isEmpty()); for (Cell c : result.rawCells()) { - assertTrue(c.toString(), Bytes.equals(ROW, CellUtil.cloneRow(c))); - assertTrue(c.toString(), Bytes.equals(FAMILY, CellUtil.cloneFamily(c))); + assertTrue(Bytes.equals(ROW, CellUtil.cloneRow(c)), c.toString()); + assertTrue(Bytes.equals(FAMILY, CellUtil.cloneFamily(c)), c.toString()); if (Bytes.equals(QUALIFIER, CellUtil.cloneQualifier(c))) { - assertTrue(c.toString(), Bytes.equals(expectedValue, CellUtil.cloneValue(c))); + assertTrue(Bytes.equals(expectedValue, CellUtil.cloneValue(c)), c.toString()); } else if (Bytes.equals(QUALIFIER_FROM_CP, CellUtil.cloneQualifier(c))) { - assertTrue(c.toString(), Bytes.equals(expectedFromCp, CellUtil.cloneValue(c))); + assertTrue(Bytes.equals(expectedFromCp, CellUtil.cloneValue(c)), c.toString()); } else { fail("No valid qualifier"); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java index 35db0a19f6f9..76da67df95dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.hbase.favored; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.when; @@ -33,10 +33,10 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestExtension; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.master.RackManager; @@ -45,24 +45,19 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Triple; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import org.mockito.Mockito; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; -@Category({ MasterTests.class, MediumTests.class }) +@Tag(MasterTests.TAG) +@Tag(MediumTests.TAG) public class TestFavoredNodeAssignmentHelper { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestFavoredNodeAssignmentHelper.class); - private static List servers = new ArrayList<>(); private static Map> rackToServers = new HashMap<>(); private static RackManager rackManager = Mockito.mock(RackManager.class); @@ -70,8 +65,8 @@ public class TestFavoredNodeAssignmentHelper { // Some tests have randomness, so we run them multiple times private static final int MAX_ATTEMPTS = 100; - @Rule - public TestName name = new TestName(); + @RegisterExtension + private final TableNameTestExtension tableNameExt = new TableNameTestExtension(); private static String getRack(int index) { if (index < 10) { @@ -85,7 +80,7 @@ private static String getRack(int index) { } } - @BeforeClass + @BeforeAll public static void setupBeforeClass() throws Exception { // Set up some server -> rack mappings // Have three racks in the cluster with 10 hosts each. @@ -301,7 +296,7 @@ public void testSecondaryAndTertiaryPlacementWithMoreThanOneServerInPrimaryRack( // create regions List regions = new ArrayList<>(regionCount); for (int i = 0; i < regionCount; i++) { - regions.add(RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())) + regions.add(RegionInfoBuilder.newBuilder(tableNameExt.getTableName()) .setStartKey(Bytes.toBytes(i)).setEndKey(Bytes.toBytes(i + 1)).build()); } // place the regions @@ -366,12 +361,15 @@ private void checkNumRegions(int regionCount, int firstRackSize, int secondRackS regionMap.put(regionsOnRack1, 1); regionMap.put(regionsOnRack2, 2); regionMap.put(regionsOnRack3, 3); - assertEquals(printProportions(firstRackSize, secondRackSize, thirdRackSize, regionsOnRack1, - regionsOnRack2, regionsOnRack3), rackMap.get(firstRackSize), regionMap.get(regionsOnRack1)); - assertEquals(printProportions(firstRackSize, secondRackSize, thirdRackSize, regionsOnRack1, - regionsOnRack2, regionsOnRack3), rackMap.get(secondRackSize), regionMap.get(regionsOnRack2)); - assertEquals(printProportions(firstRackSize, secondRackSize, thirdRackSize, regionsOnRack1, - regionsOnRack2, regionsOnRack3), rackMap.get(thirdRackSize), regionMap.get(regionsOnRack3)); + assertEquals(rackMap.get(firstRackSize), regionMap.get(regionsOnRack1), + printProportions(firstRackSize, secondRackSize, thirdRackSize, regionsOnRack1, regionsOnRack2, + regionsOnRack3)); + assertEquals(rackMap.get(secondRackSize), regionMap.get(regionsOnRack2), + printProportions(firstRackSize, secondRackSize, thirdRackSize, regionsOnRack1, regionsOnRack2, + regionsOnRack3)); + assertEquals(rackMap.get(thirdRackSize), regionMap.get(regionsOnRack3), + printProportions(firstRackSize, secondRackSize, thirdRackSize, regionsOnRack1, regionsOnRack2, + regionsOnRack3)); } private String printProportions(int firstRackSize, int secondRackSize, int thirdRackSize, @@ -392,7 +390,7 @@ public void testConstrainedPlacement() throws Exception { List regions = new ArrayList<>(20); for (int i = 0; i < 20; i++) { - regions.add(RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())) + regions.add(RegionInfoBuilder.newBuilder(tableNameExt.getTableName()) .setStartKey(Bytes.toBytes(i)).setEndKey(Bytes.toBytes(i + 1)).build()); } Map> assignmentMap = new HashMap>(); @@ -452,7 +450,7 @@ public void testGetRandomServerSingleRack() throws IOException { // Check we don't get a bad node on any number of attempts for (int attempts = 0; attempts < MAX_ATTEMPTS; attempts++) { ServerName sn = helper.getOneRandomServer(rack, Sets.newHashSet()); - assertTrue("Server:" + sn + " does not belong to list: " + servers, servers.contains(sn)); + assertTrue(servers.contains(sn), "Server:" + sn + " does not belong to list: " + servers); } // Check skipServers multiple times when an invalid server is specified @@ -460,7 +458,7 @@ public void testGetRandomServerSingleRack() throws IOException { Sets.newHashSet(ServerName.valueOf("invalidnode:1234", ServerName.NON_STARTCODE)); for (int attempts = 0; attempts < MAX_ATTEMPTS; attempts++) { ServerName sn = helper.getOneRandomServer(rack, skipServers); - assertTrue("Server:" + sn + " does not belong to list: " + servers, servers.contains(sn)); + assertTrue(servers.contains(sn), "Server:" + sn + " does not belong to list: " + servers); } // Check skipRack multiple times when an valid servers are specified @@ -468,8 +466,8 @@ public void testGetRandomServerSingleRack() throws IOException { skipServers = Sets.newHashSet(skipSN); for (int attempts = 0; attempts < MAX_ATTEMPTS; attempts++) { ServerName sn = helper.getOneRandomServer(rack, skipServers); - assertNotEquals("Skip server should not be selected ", skipSN.getAddress(), sn.getAddress()); - assertTrue("Server:" + sn + " does not belong to list: " + servers, servers.contains(sn)); + assertNotEquals(skipSN.getAddress(), sn.getAddress(), "Skip server should not be selected"); + assertTrue(servers.contains(sn), "Server:" + sn + " does not belong to list: " + servers); } } @@ -490,8 +488,8 @@ public void testGetRandomServerMultiRack() throws IOException { for (int attempts = 0; attempts < MAX_ATTEMPTS; attempts++) { for (String rack : rackList) { ServerName sn = helper.getOneRandomServer(rack, Sets.newHashSet()); - assertTrue("Server:" + sn + " does not belong to rack servers: " + rackToServers.get(rack), - rackToServers.get(rack).contains(sn)); + assertTrue(rackToServers.get(rack).contains(sn), + "Server:" + sn + " does not belong to rack servers: " + rackToServers.get(rack)); } } @@ -501,8 +499,8 @@ public void testGetRandomServerMultiRack() throws IOException { for (int attempts = 0; attempts < MAX_ATTEMPTS; attempts++) { for (String rack : rackList) { ServerName sn = helper.getOneRandomServer(rack, skipServers); - assertTrue("Server:" + sn + " does not belong to rack servers: " + rackToServers.get(rack), - rackToServers.get(rack).contains(sn)); + assertTrue(rackToServers.get(rack).contains(sn), + "Server:" + sn + " does not belong to rack servers: " + rackToServers.get(rack)); } } @@ -514,9 +512,9 @@ public void testGetRandomServerMultiRack() throws IOException { for (int attempts = 0; attempts < MAX_ATTEMPTS; attempts++) { for (String rack : rackList) { ServerName sn = helper.getOneRandomServer(rack, skipServers); - assertFalse("Skip server should not be selected ", skipServers.contains(sn)); - assertTrue("Server:" + sn + " does not belong to rack servers: " + rackToServers.get(rack), - rackToServers.get(rack).contains(sn)); + assertFalse(skipServers.contains(sn), "Skip server should not be selected"); + assertTrue(rackToServers.get(rack).contains(sn), + "Server:" + sn + " does not belong to rack servers: " + rackToServers.get(rack)); } } } @@ -534,7 +532,7 @@ public void testGetFavoredNodes() throws IOException { helper.initialize(); assertTrue(helper.canPlaceFavoredNodes()); - RegionInfo region = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())) + RegionInfo region = RegionInfoBuilder.newBuilder(tableNameExt.getTableName()) .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).build(); for (int maxattempts = 0; maxattempts < MAX_ATTEMPTS; maxattempts++) { @@ -569,13 +567,12 @@ public void testGenMissingFavoredNodeOneRack() throws IOException { for (int attempts = 0; attempts < MAX_ATTEMPTS; attempts++) { ServerName genSN = helper.generateMissingFavoredNode(fn, skipServers); checkDuplicateFN(fn, genSN); - assertNotEquals("Generated FN should not match excluded one", snRack1SN3, genSN); + assertNotEquals(snRack1SN3, genSN, "Generated FN should not match excluded one"); } } @Test public void testGenMissingFavoredNodeMultiRack() throws IOException { - ServerName snRack1SN1 = ServerName.valueOf("foo1:1234", ServerName.NON_STARTCODE); ServerName snRack1SN2 = ServerName.valueOf("foo2:1234", ServerName.NON_STARTCODE); ServerName snRack2SN1 = ServerName.valueOf("foo10:1234", ServerName.NON_STARTCODE); @@ -612,22 +609,22 @@ public void testGenMissingFavoredNodeMultiRack() throws IOException { ServerName genSN = helper.generateMissingFavoredNode(fn, skipServers); checkDuplicateFN(fn, genSN); checkFNRacks(fn, genSN); - assertNotEquals("Generated FN should not match excluded one", snRack2SN2, genSN); + assertNotEquals(snRack2SN2, genSN, "Generated FN should not match excluded one"); } } private void checkDuplicateFN(List fnList, ServerName genFN) { Set favoredNodes = Sets.newHashSet(fnList); - assertNotNull("Generated FN can't be null", genFN); + assertNotNull(genFN, "Generated FN can't be null"); favoredNodes.add(genFN); - assertEquals("Did not find expected number of favored nodes", - FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, favoredNodes.size()); + assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, favoredNodes.size(), + "Did not find expected number of favored nodes"); } private void checkDuplicateFN(List fnList) { Set favoredNodes = Sets.newHashSet(fnList); - assertEquals("Did not find expected number of favored nodes", - FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, favoredNodes.size()); + assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, favoredNodes.size(), + "Did not find expected number of favored nodes"); } private void checkFNRacks(List fnList, ServerName genFN) { @@ -637,7 +634,7 @@ private void checkFNRacks(List fnList, ServerName genFN) { for (ServerName sn : favoredNodes) { racks.add(rackManager.getRack(sn)); } - assertTrue("FN should be spread atleast across 2 racks", racks.size() >= 2); + assertTrue(racks.size() >= 2, "FN should be spread atleast across 2 racks"); } private void checkFNRacks(List fnList) { @@ -646,6 +643,6 @@ private void checkFNRacks(List fnList) { for (ServerName sn : favoredNodes) { racks.add(rackManager.getRack(sn)); } - assertTrue("FN should be spread atleast across 2 racks", racks.size() >= 2); + assertTrue(racks.size() >= 2, "FN should be spread atleast across 2 racks"); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestStartcodeAgnosticServerName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestStartcodeAgnosticServerName.java index 095495729f26..cd756cac1516 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestStartcodeAgnosticServerName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestStartcodeAgnosticServerName.java @@ -17,24 +17,19 @@ */ package org.apache.hadoop.hbase.favored; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ MiscTests.class, SmallTests.class }) +@Tag(MiscTests.TAG) +@Tag(SmallTests.TAG) public class TestStartcodeAgnosticServerName { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestStartcodeAgnosticServerName.class); - @Test public void testStartCodeServerName() { ServerName sn = ServerName.valueOf("www.example.org", 1234, 5678); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java index dcad0e503281..a3547f940cf0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java @@ -17,26 +17,22 @@ */ package org.apache.hadoop.hbase.filter; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Set; import java.util.TreeSet; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.ClassRule; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; @SuppressWarnings("deprecation") -@Category({ FilterTests.class, SmallTests.class }) +@Tag(FilterTests.TAG) +@Tag(SmallTests.TAG) public class TestFirstKeyValueMatchingQualifiersFilter { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestFirstKeyValueMatchingQualifiersFilter.class); - private static final byte[] ROW = Bytes.toBytes("test"); private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test"); private static final byte[] COLUMN_QUALIFIER_1 = Bytes.toBytes("foo"); @@ -48,6 +44,7 @@ public class TestFirstKeyValueMatchingQualifiersFilter { * Test the functionality of * {@link FirstKeyValueMatchingQualifiersFilter#filterCell(org.apache.hadoop.hbase.Cell)} */ + @Test public void testFirstKeyMatchingQualifierFilter() throws Exception { Set quals = new TreeSet<>(Bytes.BYTES_COMPARATOR); quals.add(COLUMN_QUALIFIER_1); @@ -57,19 +54,19 @@ public void testFirstKeyMatchingQualifierFilter() throws Exception { // Match in first attempt KeyValue cell; cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1); - assertTrue("includeAndSetFlag", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE); + assertTrue(filter.filterCell(cell) == Filter.ReturnCode.INCLUDE, "includeAndSetFlag"); cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); - assertTrue("flagIsSetSkipToNextRow", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW); + assertTrue(filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW, "flagIsSetSkipToNextRow"); // A mismatch in first attempt and match in second attempt. filter.reset(); cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_3, VAL_1); System.out.println(filter.filterCell(cell)); - assertTrue("includeFlagIsUnset", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE); + assertTrue(filter.filterCell(cell) == Filter.ReturnCode.INCLUDE, "includeFlagIsUnset"); cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); - assertTrue("includeAndSetFlag", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE); + assertTrue(filter.filterCell(cell) == Filter.ReturnCode.INCLUDE, "includeAndSetFlag"); cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1); - assertTrue("flagIsSetSkipToNextRow", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW); + assertTrue(filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW, "flagIsSetSkipToNextRow"); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java index 35969c54afcd..e683e984ad25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java @@ -17,36 +17,31 @@ */ package org.apache.hadoop.hbase.master; -import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.assertFalse; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.CommonFSUtils; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ MasterTests.class, MediumTests.class }) +@Tag(MasterTests.TAG) +@Tag(MediumTests.TAG) public class TestMasterNotCarryTable { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMasterNotCarryTable.class); - private static final Logger LOG = LoggerFactory.getLogger(TestMasterNotCarryTable.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static HMaster master; - @BeforeClass + @BeforeAll public static void setUp() throws Exception { Configuration c = UTIL.getConfiguration(); // We use local filesystem. Set it so it writes into the testdir. @@ -61,7 +56,7 @@ public static void setUp() throws Exception { } } - @AfterClass + @AfterAll public static void tearDown() throws Exception { master.stop("Shutdown"); UTIL.shutdownMiniZKCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java index 454a22eab2b0..961fb70ee1b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java @@ -17,10 +17,9 @@ */ package org.apache.hadoop.hbase.master; -import static junit.framework.TestCase.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -32,13 +31,11 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.zookeeper.KeeperException; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; @@ -47,29 +44,25 @@ * Tests that table state is mirrored out to zookeeper for hbase-1.x clients. Also tests that table * state gets migrated from zookeeper on master start. */ -@Category({ MasterTests.class, LargeTests.class }) +@Tag(MasterTests.TAG) +@Tag(LargeTests.TAG) public class TestMirroringTableStateManager { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMirroringTableStateManager.class); - @Rule - public TestName name = new TestName(); private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - @Before + @BeforeEach public void before() throws Exception { TEST_UTIL.startMiniCluster(); } - @After + @AfterEach public void after() throws Exception { TEST_UTIL.shutdownMiniCluster(); } @Test - public void testMirroring() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + public void testMirroring(TestInfo testInfo) throws Exception { + final TableName tableName = TableName.valueOf(testInfo.getTestMethod().get().getName()); TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY_STR); ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); assertTrue(TableState.State.ENABLED.equals(getTableStateInZK(zkw, tableName))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlan.java index c65198b825bc..64e83367bb55 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlan.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlan.java @@ -17,39 +17,34 @@ */ package org.apache.hadoop.hbase.master; -import static junit.framework.TestCase.assertFalse; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestExtension; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; -@Category({ MasterTests.class, SmallTests.class }) +@Tag(MasterTests.TAG) +@Tag(SmallTests.TAG) public class TestRegionPlan { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestRegionPlan.class); - private final ServerName SRC = ServerName.valueOf("source", 1234, 2345); private final ServerName DEST = ServerName.valueOf("dest", 1234, 2345); - @Rule - public TestName name = new TestName(); + + @RegisterExtension + private final TableNameTestExtension tableNameExt = new TableNameTestExtension(); @Test public void testCompareTo() { - RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); + RegionInfo hri = RegionInfoBuilder.newBuilder(tableNameExt.getTableName()).build(); RegionPlan a = new RegionPlan(hri, null, null); RegionPlan b = new RegionPlan(hri, null, null); assertEquals(0, a.compareTo(b)); @@ -72,7 +67,7 @@ public void testCompareTo() { @Test public void testEqualsWithNulls() { - RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); + RegionInfo hri = RegionInfoBuilder.newBuilder(tableNameExt.getTableName()).build(); RegionPlan a = new RegionPlan(hri, null, null); RegionPlan b = new RegionPlan(hri, null, null); assertTrue(a.equals(b)); @@ -89,7 +84,7 @@ public void testEqualsWithNulls() { @Test public void testEquals() { - RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); + RegionInfo hri = RegionInfoBuilder.newBuilder(tableNameExt.getTableName()).build(); // Identity equality RegionPlan plan = new RegionPlan(hri, SRC, DEST); @@ -97,8 +92,7 @@ public void testEquals() { assertEquals(plan, new RegionPlan(hri, SRC, DEST)); // HRI is used for equality - RegionInfo other = - RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName() + "other")).build(); + RegionInfo other = RegionInfoBuilder.newBuilder(tableNameExt.getTableName("other")).build(); assertNotEquals(plan.hashCode(), new RegionPlan(other, SRC, DEST).hashCode()); assertNotEquals(plan, new RegionPlan(other, SRC, DEST)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java index 8bfe0464509a..4c89dbcc9f27 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java @@ -17,9 +17,8 @@ */ package org.apache.hadoop.hbase.master; -import static junit.framework.TestCase.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; @@ -29,49 +28,41 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; /** * Tests the default table lock manager */ -@Category({ MasterTests.class, LargeTests.class }) +@Tag(MasterTests.TAG) +@Tag(LargeTests.TAG) public class TestTableStateManager { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestTableStateManager.class); - private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - @Rule - public TestName name = new TestName(); - - @Before + @BeforeEach public void before() throws Exception { TEST_UTIL.startMiniCluster(); } - @After + @AfterEach public void after() throws Exception { TEST_UTIL.shutdownMiniCluster(); } @Test - public void testMigration() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + public void testMigration(TestInfo testInfo) throws Exception { + final TableName tableName = TableName.valueOf(testInfo.getTestMethod().get().getName()); TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY_STR); TEST_UTIL.getAdmin().disableTable(tableName); // Table is disabled. Now remove the DISABLED column from the hbase:meta for this table's // region. We want to see if Master will read the DISABLED from zk and make use of it as // though it were reading the zk table state written by a hbase-1.x cluster. TableState state = MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), tableName); - assertTrue("State=" + state, state.getState().equals(TableState.State.DISABLED)); + assertTrue(state.getState().equals(TableState.State.DISABLED), "State=" + state); MetaTableAccessor.deleteTableState(TEST_UTIL.getConnection(), tableName); assertTrue(MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), tableName) == null); // Now kill Master so a new one can come up and run through the zk migration. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancer.java index 4e0232417d4d..a6b92da24407 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancer.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hbase.master.balancer; -import static org.junit.Assert.assertNotNull; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java index 5496ff6b547b..1a82a840a9ab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.hbase.master.balancer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -32,7 +32,6 @@ import java.util.Random; import java.util.TreeMap; import org.apache.hadoop.hbase.ClusterMetrics; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HDFSBlocksDistribution; @@ -51,19 +50,15 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ MasterTests.class, MediumTests.class }) +@Tag(MasterTests.TAG) +@Tag(MediumTests.TAG) public class TestRegionLocationFinder { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestRegionLocationFinder.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static MiniHBaseCluster cluster; @@ -74,7 +69,7 @@ public class TestRegionLocationFinder { private static RegionLocationFinder finder = new RegionLocationFinder(); - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { cluster = TEST_UTIL.startMiniCluster(ServerNum); table = TEST_UTIL.createTable(tableName, FAMILY, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); @@ -93,7 +88,7 @@ public static void setUpBeforeClass() throws Exception { finder.setClusterMetrics(cluster.getMaster().getClusterMetrics()); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { table.close(); TEST_UTIL.deleteTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java index 759f0ab2cf8b..d3c8e869997f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java @@ -17,17 +17,17 @@ */ package org.apache.hadoop.hbase.master.balancer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.StartMiniClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestExtension; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.LoadBalancer; @@ -35,14 +35,12 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,17 +50,11 @@ * kill combinations to make sure the distribution is more than just for startup. NOTE: Regions on * Master does not work well. See HBASE-19828. Until addressed, disabling this test. */ -@Ignore -@Category({ MediumTests.class }) +@Disabled +@Tag(MediumTests.TAG) public class TestRegionsOnMasterOptions { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestRegionsOnMasterOptions.class); - private static final Logger LOG = LoggerFactory.getLogger(TestRegionsOnMasterOptions.class); - @Rule - public TestName name = new TestName(); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Configuration c; private String tablesOnMasterOldValue; @@ -75,14 +67,17 @@ public class TestRegionsOnMasterOptions { private static final int REGIONS = 12; private static final int SYSTEM_REGIONS = 2; // ns and meta -- no acl unless enabled. - @Before + @RegisterExtension + private final TableNameTestExtension tableNameExt = new TableNameTestExtension(); + + @BeforeEach public void setup() { this.c = TEST_UTIL.getConfiguration(); this.tablesOnMasterOldValue = c.get(LoadBalancer.TABLES_ON_MASTER); this.systemTablesOnMasterOldValue = c.get(LoadBalancer.SYSTEM_TABLES_ON_MASTER); } - @After + @AfterEach public void tearDown() { unset(LoadBalancer.TABLES_ON_MASTER, this.tablesOnMasterOldValue); unset(LoadBalancer.SYSTEM_TABLES_ON_MASTER, this.systemTablesOnMasterOldValue); @@ -112,7 +107,7 @@ public void testNoRegionOnMaster() throws Exception { checkBalance(0, rsCount); } - @Ignore // Fix this. The Master startup doesn't allow Master reporting as a RegionServer, not + // Fix this. The Master startup doesn't allow Master reporting as a RegionServer, not // until way late after the Master startup finishes. Needs more work. @Test public void testSystemTablesOnMaster() throws Exception { @@ -172,7 +167,7 @@ private void checkBalance(int masterCount, int rsCount) throws Exception { StartMiniClusterOption option = StartMiniClusterOption.builder().numMasters(MASTERS) .numRegionServers(SLAVES).numDataNodes(SLAVES).build(); MiniHBaseCluster cluster = TEST_UTIL.startMiniCluster(option); - TableName tn = TableName.valueOf(this.name.getMethodName()); + TableName tn = tableNameExt.getTableName(); try { Table t = TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY, REGIONS); LOG.info("Server: " + cluster.getMaster().getServerManager().getOnlineServersList()); @@ -229,7 +224,7 @@ private void checkBalance(int masterCount, int rsCount) throws Exception { } private void checkCount(int actual, int expected) { - assertTrue("Actual=" + actual + ", expected=" + expected, - actual >= (expected - 2) && actual <= (expected + 2)); // Lots of slop +/- 2 + assertTrue(actual >= (expected - 2) && actual <= (expected + 2), + "Actual=" + actual + ", expected=" + expected); // Lots of slop +/- 2 } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestServerAndLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestServerAndLoad.java index 9568edad74cd..e4c974c29633 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestServerAndLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestServerAndLoad.java @@ -17,24 +17,19 @@ */ package org.apache.hadoop.hbase.master.balancer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ MasterTests.class, SmallTests.class }) +@Tag(MasterTests.TAG) +@Tag(SmallTests.TAG) public class TestServerAndLoad { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestServerAndLoad.class); - @Test public void test() { ServerName server = ServerName.valueOf("host", 12345, 112244); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java index 244651cd9bb5..bb9a73c78c50 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hbase.master.cleaner; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java index 705cd90284f6..a42698d20a32 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hbase.mob; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.Arrays; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java index c4137fd71196..8f293374b8d1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hbase.protobuf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import com.google.protobuf.ByteString; import java.io.IOException; @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; @@ -52,22 +51,18 @@ import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer; /** * Class to test ProtobufUtil. */ -@Category({ MiscTests.class, SmallTests.class }) +@Tag(MiscTests.TAG) +@Tag(SmallTests.TAG) public class TestProtobufUtil { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestProtobufUtil.class); - @Test public void testException() throws IOException { NameBytesPair.Builder builder = NameBytesPair.newBuilder(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java index 48a4e7b2984c..f5226411163d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.IOException; import java.util.ArrayList; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShortCircuitGet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShortCircuitGet.java index 4836effb4332..dad8b52d0dcb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShortCircuitGet.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShortCircuitGet.java @@ -17,15 +17,14 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.Arrays; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer; @@ -47,13 +46,10 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; @@ -65,11 +61,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; -@Category({ RegionServerTests.class, MediumTests.class }) +@Tag(RegionServerTests.TAG) +@Tag(MediumTests.TAG) public class TestShortCircuitGet { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestShortCircuitGet.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] FAMILY = Bytes.toBytes("testFamily"); @@ -85,10 +79,7 @@ public class TestShortCircuitGet { static final byte[] r6 = Bytes.toBytes("row-6"); static final TableName tableName = TableName.valueOf("TestShortCircuitGet"); - @Rule - public TestName name = new TestName(); - - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 30 * 60 * 1000); @@ -101,7 +92,7 @@ public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(1); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); } @@ -141,9 +132,9 @@ public void testScannerCloseWhenScanAndGetInCP() throws Exception { scan.withStartRow(r0, true).withStopRow(r1, true); resultScanner = table.getScanner(scan); Result result = resultScanner.next(); - assertTrue("Expected row: row-0", Bytes.equals(r0, result.getRow())); + assertTrue(Bytes.equals(r0, result.getRow()), "Expected row: row-0"); result = resultScanner.next(); - assertTrue("Expected row: row-1", Bytes.equals(r1, result.getRow())); + assertTrue(Bytes.equals(r1, result.getRow()), "Expected row: row-1"); assertNull(resultScanner.next()); } finally { if (resultScanner != null) { @@ -283,7 +274,7 @@ public RegionScanner postScannerOpen( try { table1 = connection.getTable(tableName); Result result = table1.get(get2); - assertTrue("Expected row: row-2", Bytes.equals(r2, result.getRow())); + assertTrue(Bytes.equals(r2, result.getRow()), "Expected row: row-2"); } finally { if (table1 != null) { @@ -304,8 +295,8 @@ public RegionScanner postScannerOpen( try { table2 = connection.getTable(tableName); Result[] results = table2.get(Arrays.asList(get3, get4)); - assertTrue("Expected row: row-3", Bytes.equals(r3, results[0].getRow())); - assertTrue("Expected row: row-4", Bytes.equals(r4, results[1].getRow())); + assertTrue(Bytes.equals(r3, results[0].getRow()), "Expected row: row-3"); + assertTrue(Bytes.equals(r4, results[1].getRow()), "Expected row: row-4"); } finally { if (table2 != null) { table2.close(); @@ -328,9 +319,9 @@ public RegionScanner postScannerOpen( table3 = connection.getTable(tableName); resultScanner = table3.getScanner(newScan); Result result = resultScanner.next(); - assertTrue("Expected row: row-5", Bytes.equals(r5, result.getRow())); + assertTrue(Bytes.equals(r5, result.getRow()), "Expected row: row-5"); result = resultScanner.next(); - assertTrue("Expected row: row-6", Bytes.equals(r6, result.getRow())); + assertTrue(Bytes.equals(r6, result.getRow()), "Expected row: row-6"); result = resultScanner.next(); assertNull(result); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java index 731781bfc1b2..a5d2f81dd308 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java @@ -17,7 +17,8 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.NavigableMap; @@ -30,7 +31,6 @@ import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; @@ -52,14 +52,11 @@ import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,18 +66,12 @@ /** * Testing for lock up of FSHLog. */ -@Category({ RegionServerTests.class, MediumTests.class }) +@Tag(RegionServerTests.TAG) +@Tag(MediumTests.TAG) public class TestWALLockup { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestWALLockup.class); - private static final Logger LOG = LoggerFactory.getLogger(TestWALLockup.class); - @Rule - public TestName name = new TestName(); - private static final String COLUMN_FAMILY = "MyCF"; private static final byte[] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY); @@ -90,18 +81,21 @@ public class TestWALLockup { private String dir; // Test names - protected TableName tableName; + private TableName tableName; + + private String methodName; - @Before - public void setup() throws IOException { + @BeforeEach + public void setup(TestInfo testInfo) throws IOException { CONF = TEST_UTIL.getConfiguration(); // Disable block cache. CONF.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); dir = TEST_UTIL.getDataTestDir("TestHRegion").toString(); - tableName = TableName.valueOf(name.getMethodName()); + methodName = testInfo.getTestMethod().get().getName(); + tableName = TableName.valueOf(methodName); } - @After + @AfterEach public void tearDown() throws Exception { EnvironmentEdgeManagerTestHelper.reset(); LOG.info("Cleaning test directory: " + TEST_UTIL.getDataTestDir()); @@ -109,7 +103,7 @@ public void tearDown() throws Exception { } private String getName() { - return name.getMethodName(); + return methodName; } // A WAL that we can have throw exceptions when a flag is set. @@ -259,7 +253,7 @@ public void testLockupWhenSyncInMiddleOfZigZagSetup() throws IOException { } catch (Exception e) { exception = true; } - assertTrue("Did not get sync exception", exception); + assertTrue(exception, "Did not get sync exception"); // Get a memstore flush going too so we have same hung profile as up in the issue over // in HBASE-14317. Flush hangs trying to get sequenceid because the ringbuffer is held up @@ -348,7 +342,7 @@ protected long getSequenceOnRingBuffer() { protected void publishSyncOnRingBufferAndBlock(long sequence) { try { super.blockOnSync(super.publishSyncOnRingBuffer(sequence, false)); - Assert.fail("Expect an IOException here."); + fail("Expect an IOException here."); } catch (IOException ignore) { // Here, we will get an IOException. } @@ -435,7 +429,7 @@ public void run() { LOG.info("Call sync for testing whether RingBufferEventHandler is hanging."); dodgyWAL.sync(false); // Should not get a hang here, otherwise we will see timeout in this // test. - Assert.fail("Expect an IOException here."); + fail("Expect an IOException here."); } catch (IOException ignore) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALOpenError.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALOpenError.java index eafafd3d708f..f6b635603903 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALOpenError.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALOpenError.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.regionserver.wal; +import static org.junit.jupiter.api.Assertions.assertTrue; + import java.io.IOException; import java.util.List; import org.apache.hadoop.conf.Configuration; @@ -24,7 +26,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.FailedCloseWALAfterInitializedErrorException; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; @@ -34,29 +35,23 @@ import org.apache.hadoop.hbase.wal.FSHLogProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Test WAL Init ERROR */ -@Category({ RegionServerTests.class, MediumTests.class }) +@Tag(RegionServerTests.TAG) +@Tag(MediumTests.TAG) public class TestWALOpenError { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestWALOpenError.class); - private static final Logger LOG = LoggerFactory.getLogger(TestWALOpenError.class); protected static Configuration conf; @@ -70,18 +65,16 @@ public class TestWALOpenError { protected WALFactory wals; private ServerName currentServername; - @Rule - public final TestName currentTest = new TestName(); - - @Before - public void setUp() throws Exception { + @BeforeEach + public void setUp(TestInfo testInfo) throws Exception { fs = cluster.getFileSystem(); - dir = new Path(hbaseDir, currentTest.getMethodName()); - this.currentServername = ServerName.valueOf(currentTest.getMethodName(), 16010, 1); + String methodName = testInfo.getTestMethod().get().getName(); + dir = new Path(hbaseDir, methodName); + this.currentServername = ServerName.valueOf(methodName, 16010, 1); wals = new WALFactory(conf, this.currentServername.toString()); } - @After + @AfterEach public void tearDown() throws Exception { // testAppendClose closes the FileSystem, which will prevent us from closing cleanly here. try { @@ -97,7 +90,7 @@ public void tearDown() throws Exception { } } - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniDFSCluster(3); conf = TEST_UTIL.getConfiguration(); @@ -109,7 +102,7 @@ public static void setUpBeforeClass() throws Exception { hbaseWALDir = TEST_UTIL.createWALRootDir(); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); } @@ -128,11 +121,11 @@ public void testWALClosedIfOpenError() throws IOException { } catch (IOException ex) { hasFakeInitException = ex.getMessage().contains("Fake init exception"); } - Assert.assertTrue(hasFakeInitException); - Assert.assertTrue(myFSLogCreated.closed); + assertTrue(hasFakeInitException); + assertTrue(myFSLogCreated.closed); FileStatus[] fileStatuses = CommonFSUtils.listStatus(fs, myFSLogCreated.walDir); - Assert.assertTrue(fileStatuses == null || fileStatuses.length == 0); + assertTrue(fileStatuses == null || fileStatuses.length == 0); } @Test @@ -144,7 +137,7 @@ public void testThrowFailedCloseWalException() throws IOException { } catch (FailedCloseWALAfterInitializedErrorException ex) { failedCloseWalException = true; } - Assert.assertTrue(failedCloseWalException); + assertTrue(failedCloseWalException); } public static class MyFSWalProvider extends FSHLogProvider { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationDroppedTablesTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationDroppedTablesTestBase.java index c01df9545e40..3152717fcd75 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationDroppedTablesTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationDroppedTablesTestBase.java @@ -18,7 +18,8 @@ package org.apache.hadoop.hbase.replication; import static org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint.REPLICATION_DROP_ON_DELETED_TABLE_KEY; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.concurrent.ThreadLocalRandom; @@ -37,14 +38,13 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Base class for testing replication for dropped tables. */ -public class ReplicationDroppedTablesTestBase extends TestReplicationBase { +public class ReplicationDroppedTablesTestBase extends TestReplicationBaseNoBeforeAll { private static final Logger LOG = LoggerFactory.getLogger(ReplicationDroppedTablesTestBase.class); @@ -149,7 +149,7 @@ private boolean peerHasAllNormalRows() throws IOException { return false; } for (int i = 0; i < results.length; i++) { - Assert.assertArrayEquals(generateRowKey(i), results[i].getRow()); + assertArrayEquals(generateRowKey(i), results[i].getRow()); } return true; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationKillRSTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationKillRSTestBase.java index 5ad90b9a5e60..ba5b3c5cc46f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationKillRSTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationKillRSTestBase.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Threads; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -116,8 +115,12 @@ private static Thread killARegionServer(final HBaseTestingUtility utility, final Thread killer = new Thread() { @Override public void run() { - Threads.sleepWithoutInterrupt(timeout); - utility.getHBaseCluster().getRegionServer(rs).abort("Stopping as part of the test"); + try { + Thread.sleep(timeout); + utility.getHBaseCluster().getRegionServer(rs).abort("Stopping as part of the test"); + } catch (Exception e) { + LOG.error("Couldn't kill a region server", e); + } } }; killer.setDaemon(true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java index d4f76584f00a..448b076028b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.io.UncheckedIOException; @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestExtension; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -41,11 +42,10 @@ import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hbase.wal.WALStreamReader; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.RegisterExtension; /** * Base class for testing serial replication. @@ -66,14 +66,14 @@ public class SerialReplicationTestBase { protected static WALProvider.Writer WRITER; - @Rule - public final TestName name = new TestName(); + @RegisterExtension + protected final TableNameTestExtension tableNameExt = new TableNameTestExtension(); protected Path logPath; public static final class LocalReplicationEndpoint extends BaseReplicationEndpoint { - private static final UUID PEER_UUID = UTIL.getRandomUUID(); + private static final UUID PEER_UUID = HBaseTestingUtility.getRandomUUID(); @Override public UUID getPeerUUID() { @@ -121,7 +121,7 @@ public boolean canReplicateToSameCluster() { } } - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { UTIL.getConfiguration().setInt("replication.source.nb.capacity", 10); UTIL.getConfiguration().setLong("replication.sleep.before.failover", 1000); @@ -134,12 +134,12 @@ public static void setUpBeforeClass() throws Exception { FS.mkdirs(LOG_DIR); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { UTIL.shutdownMiniCluster(); } - @After + @AfterEach public void tearDown() throws Exception { Admin admin = UTIL.getAdmin(); for (ReplicationPeerDescription pd : admin.listReplicationPeers()) { @@ -188,7 +188,7 @@ public String explainFailure() throws Exception { } protected final void setupWALWriter() throws IOException { - logPath = new Path(LOG_DIR, name.getMethodName()); + logPath = new Path(LOG_DIR, tableNameExt.getTableName().getNameAsString()); WRITER = WALFactory.createWALWriter(FS, logPath, UTIL.getConfiguration()); } @@ -235,9 +235,8 @@ protected final void checkOrder(int expectedEntries) throws IOException { if (entry == null) { break; } - assertTrue( - "Sequence id go backwards from " + seqId + " to " + entry.getKey().getSequenceId(), - entry.getKey().getSequenceId() >= seqId); + assertTrue(entry.getKey().getSequenceId() >= seqId, + "Sequence id go backwards from " + seqId + " to " + entry.getKey().getSequenceId()); seqId = entry.getKey().getSequenceId(); count++; } @@ -246,7 +245,7 @@ protected final void checkOrder(int expectedEntries) throws IOException { } protected final TableName createTable() throws IOException, InterruptedException { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = tableNameExt.getTableName(); UTIL.getAdmin().createTable( TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java index 229da6b07129..a642a608bd2a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java @@ -17,12 +17,11 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.Collections; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.client.Put; @@ -39,24 +38,20 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; /** * Testcase for HBASE-20147. */ -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestAddToSerialReplicationPeer extends SerialReplicationTestBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestAddToSerialReplicationPeer.class); - - @Before + @BeforeEach public void setUp() throws IOException, StreamLacksCapabilityException { setupWALWriter(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBidirectionSerialReplicationStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBidirectionSerialReplicationStuck.java index f069d6b1095b..28c85eb66d8a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBidirectionSerialReplicationStuck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBidirectionSerialReplicationStuck.java @@ -17,28 +17,26 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestBidirectionSerialReplicationStuck extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestBidirectionSerialReplicationStuck.class); - @Override protected boolean isSerialPeer() { return true; } + @BeforeEach @Override public void setUpBase() throws Exception { UTIL1.ensureSomeRegionServersAvailable(2); @@ -47,6 +45,7 @@ public void setUpBase() throws Exception { addPeer(PEER_ID2, tableName, UTIL2, UTIL1); } + @AfterEach @Override public void tearDownBase() throws Exception { removePeer(PEER_ID2, UTIL1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBulkLoadReplication.java similarity index 80% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBulkLoadReplication.java index 9e3759311c32..b347efa02865 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBulkLoadReplication.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.regionserver; +package org.apache.hadoop.hbase.replication; import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID; import static org.apache.hadoop.hbase.HConstants.REPLICATION_CONF_DIR; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; import java.io.FileOutputStream; @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.ExtendedCellBuilder; import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -63,24 +62,18 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; -import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; -import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; -import org.apache.hadoop.hbase.replication.TestReplicationBase; -import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,12 +87,9 @@ * of the clusters. This CP counts the amount of times bulk load actually gets invoked, certifying * we are not entering the infinite loop condition addressed by HBASE-22380. */ -@Category({ ReplicationTests.class, MediumTests.class }) -public class TestBulkLoadReplication extends TestReplicationBase { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestBulkLoadReplication.class); +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) +public class TestBulkLoadReplication extends TestReplicationBaseNoBeforeAll { protected static final Logger LOG = LoggerFactory.getLogger(TestBulkLoadReplication.class); @@ -118,26 +108,41 @@ public class TestBulkLoadReplication extends TestReplicationBase { private static final Path BULK_LOAD_BASE_DIR = new Path("/bulk_dir"); - @Rule - public TestName name = new TestName(); - - @ClassRule - public static TemporaryFolder testFolder = new TemporaryFolder(); + private static File SOURCE_DIR; private static ReplicationQueueStorage queueStorage; - private static boolean replicationPeersAdded = false; - - @BeforeClass - public static void setUpBeforeClass() throws Exception { + @BeforeAll + public static void setUpBeforeAll() throws Exception { + setupConfig(UTIL3, "/3"); + configureClusters(UTIL1, UTIL2); + SOURCE_DIR = new File(UTIL1.getRandomDir().toString()).getAbsoluteFile(); setupBulkLoadConfigsForCluster(CONF1, PEER1_CLUSTER_ID); setupBulkLoadConfigsForCluster(CONF2, PEER2_CLUSTER_ID); setupBulkLoadConfigsForCluster(CONF3, PEER3_CLUSTER_ID); - setupConfig(UTIL3, "/3"); - TestReplicationBase.setUpBeforeClass(); + startClusters(); startThirdCluster(); queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(UTIL1.getZooKeeperWatcher(), UTIL1.getConfiguration()); + setupCoprocessor(UTIL1); + setupCoprocessor(UTIL2); + setupCoprocessor(UTIL3); + + ReplicationPeerConfig peer1Config = getPeerConfigForCluster(UTIL1); + ReplicationPeerConfig peer2Config = getPeerConfigForCluster(UTIL2); + ReplicationPeerConfig peer3Config = getPeerConfigForCluster(UTIL3); + // Setup following topology: "1 <-> 2 <-> 3", 1 -> 2 will be added by setUpBase method in parent + // class + UTIL2.getAdmin().addReplicationPeer(PEER_ID1, peer1Config); + // adds cluster3 as a remote peer on cluster2 + UTIL2.getAdmin().addReplicationPeer(PEER_ID3, peer3Config); + // adds cluster2 as a remote peer on cluster3 + UTIL3.getAdmin().addReplicationPeer(PEER_ID2, peer2Config); + } + + @AfterAll + public static void tearDownAfterAll() throws Exception { + UTIL3.shutdownMiniCluster(); } private static void startThirdCluster() throws Exception { @@ -157,65 +162,44 @@ private static void startThirdCluster() throws Exception { UTIL3.waitUntilAllRegionsAssigned(tableName); } - @Before - @Override - public void setUpBase() throws Exception { + @BeforeEach + public void resetBulkLoadCount() throws Exception { // removing the peer and adding again causing the previously completed bulk load jobs getting - // submitted again, adding a check to add the peers only once. - if (!replicationPeersAdded) { - // "super.setUpBase()" already sets replication from 1->2, - // then on the subsequent lines, sets 2->1, 2->3 and 3->2. - // So we have following topology: "1 <-> 2 <->3" - super.setUpBase(); - ReplicationPeerConfig peer1Config = getPeerConfigForCluster(UTIL1); - ReplicationPeerConfig peer2Config = getPeerConfigForCluster(UTIL2); - ReplicationPeerConfig peer3Config = getPeerConfigForCluster(UTIL3); - // adds cluster1 as a remote peer on cluster2 - UTIL2.getAdmin().addReplicationPeer(PEER_ID1, peer1Config); - // adds cluster3 as a remote peer on cluster2 - UTIL2.getAdmin().addReplicationPeer(PEER_ID3, peer3Config); - // adds cluster2 as a remote peer on cluster3 - UTIL3.getAdmin().addReplicationPeer(PEER_ID2, peer2Config); - setupCoprocessor(UTIL1); - setupCoprocessor(UTIL2); - setupCoprocessor(UTIL3); - replicationPeersAdded = true; - } - + // submitted again, so here we override the setUpBase and tearDownBase to not adding/removing + // peers between each tests, we will add peers in beforeAll BULK_LOADS_COUNT = new AtomicInteger(0); } - private ReplicationPeerConfig getPeerConfigForCluster(HBaseTestingUtility util) { - return ReplicationPeerConfig.newBuilder().setClusterKey(util.getClusterKey()) - .setSerial(isSerialPeer()).build(); + private static ReplicationPeerConfig getPeerConfigForCluster(HBaseTestingUtility util) { + return ReplicationPeerConfig.newBuilder().setClusterKey(util.getClusterKey()).setSerial(false) + .build(); } - private void setupCoprocessor(HBaseTestingUtility cluster) { - cluster.getHBaseCluster().getRegions(tableName).forEach(r -> { - try { - TestBulkLoadReplication.BulkReplicationTestObserver cp = r.getCoprocessorHost() + private static void setupCoprocessor(HBaseTestingUtility cluster) throws IOException { + for (HRegion region : cluster.getHBaseCluster().getRegions(tableName)) { + TestBulkLoadReplication.BulkReplicationTestObserver cp = region.getCoprocessorHost() + .findCoprocessor(TestBulkLoadReplication.BulkReplicationTestObserver.class); + if (cp == null) { + region.getCoprocessorHost().load(TestBulkLoadReplication.BulkReplicationTestObserver.class, + 0, cluster.getConfiguration()); + cp = region.getCoprocessorHost() .findCoprocessor(TestBulkLoadReplication.BulkReplicationTestObserver.class); - if (cp == null) { - r.getCoprocessorHost().load(TestBulkLoadReplication.BulkReplicationTestObserver.class, 0, - cluster.getConfiguration()); - cp = r.getCoprocessorHost() - .findCoprocessor(TestBulkLoadReplication.BulkReplicationTestObserver.class); - cp.clusterName = cluster.getClusterKey(); - } - } catch (Exception e) { - LOG.error(e.getMessage(), e); + cp.clusterName = cluster.getClusterKey(); } - }); + } } protected static void setupBulkLoadConfigsForCluster(Configuration config, String clusterReplicationId) throws Exception { config.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true); config.set(REPLICATION_CLUSTER_ID, clusterReplicationId); - File sourceConfigFolder = testFolder.newFolder(clusterReplicationId); - File sourceConfigFile = new File(sourceConfigFolder.getAbsolutePath() + "/hbase-site.xml"); - config.writeXml(new FileOutputStream(sourceConfigFile)); - config.set(REPLICATION_CONF_DIR, testFolder.getRoot().getAbsolutePath()); + File sourceConfigFolder = new File(SOURCE_DIR, clusterReplicationId); + sourceConfigFolder.mkdirs(); + File sourceConfigFile = new File(sourceConfigFolder.getAbsolutePath(), "hbase-site.xml"); + try (FileOutputStream out = new FileOutputStream(sourceConfigFile)) { + config.writeXml(out); + } + config.set(REPLICATION_CONF_DIR, SOURCE_DIR.getAbsolutePath()); } @Test @@ -288,7 +272,9 @@ private String createHFileForFamilies(byte[] row, byte[] value, Configuration cl HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(clusterConfig); // TODO We need a way to do this without creating files - File hFileLocation = testFolder.newFile(); + File randomDir = new File(UTIL1.getRandomDir().toString()).getAbsoluteFile(); + randomDir.mkdirs(); + File hFileLocation = new File(randomDir, "hfile"); FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation), null); try { hFileFactory.withOutputStream(out); @@ -374,7 +360,9 @@ private String createHFileForNoRepFamilies(byte[] row, byte[] value, Configurati HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(clusterConfig); // TODO We need a way to do this without creating files - File hFileLocation = testFolder.newFile(); + File randomDir = new File(UTIL1.getRandomDir().toString()).getAbsoluteFile(); + randomDir.mkdirs(); + File hFileLocation = new File(randomDir, "hfile"); FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation), null); try { hFileFactory.withOutputStream(out); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplicationHFileRefs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBulkLoadReplicationHFileRefs.java similarity index 86% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplicationHFileRefs.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBulkLoadReplicationHFileRefs.java index 70a6e88552bd..48a81f2180bc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplicationHFileRefs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestBulkLoadReplicationHFileRefs.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.regionserver; +package org.apache.hadoop.hbase.replication; import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID; import static org.apache.hadoop.hbase.HConstants.REPLICATION_CONF_DIR; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; import java.io.FileOutputStream; @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.CellBuilder; import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -49,36 +48,25 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; -import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; -import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; -import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; -import org.apache.hadoop.hbase.replication.TestReplicationBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; -@Category({ ReplicationTests.class, SmallTests.class }) -public class TestBulkLoadReplicationHFileRefs extends TestReplicationBase { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestBulkLoadReplicationHFileRefs.class); +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) +public class TestBulkLoadReplicationHFileRefs extends TestReplicationBaseNoBeforeAll { private static final String PEER1_CLUSTER_ID = "peer1"; private static final String PEER2_CLUSTER_ID = "peer2"; @@ -96,9 +84,6 @@ public class TestBulkLoadReplicationHFileRefs extends TestReplicationBase { private byte[] qualifier = Bytes.toBytes("q1"); private byte[] value = Bytes.toBytes("v1"); - @ClassRule - public static TemporaryFolder testFolder = new TemporaryFolder(); - private static final Path BULK_LOAD_BASE_DIR = new Path("/bulk_dir"); private static Admin admin1; @@ -106,11 +91,15 @@ public class TestBulkLoadReplicationHFileRefs extends TestReplicationBase { private static ReplicationQueueStorage queueStorage; - @BeforeClass + private static File sourceDir; + + @BeforeAll public static void setUpBeforeClass() throws Exception { + configureClusters(UTIL1, UTIL2); + sourceDir = new File(UTIL1.getRandomDir().toString()).getAbsoluteFile(); setupBulkLoadConfigsForCluster(CONF1, PEER1_CLUSTER_ID); setupBulkLoadConfigsForCluster(CONF2, PEER2_CLUSTER_ID); - TestReplicationBase.setUpBeforeClass(); + startClusters(); admin1 = UTIL1.getConnection().getAdmin(); admin2 = UTIL2.getConnection().getAdmin(); @@ -127,20 +116,21 @@ protected static void setupBulkLoadConfigsForCluster(Configuration config, String clusterReplicationId) throws Exception { config.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true); config.set(REPLICATION_CLUSTER_ID, clusterReplicationId); - File sourceConfigFolder = testFolder.newFolder(clusterReplicationId); - File sourceConfigFile = new File(sourceConfigFolder.getAbsolutePath() + "/hbase-site.xml"); + File sourceConfigFolder = new File(sourceDir, clusterReplicationId); + sourceConfigFolder.mkdirs(); + File sourceConfigFile = new File(sourceConfigFolder.getAbsolutePath(), "hbase-site.xml"); config.writeXml(new FileOutputStream(sourceConfigFile)); - config.set(REPLICATION_CONF_DIR, testFolder.getRoot().getAbsolutePath()); + config.set(REPLICATION_CONF_DIR, sourceDir.getAbsolutePath()); } - @Before + @BeforeEach public void setUp() throws Exception { for (ReplicationPeerDescription peer : admin1.listReplicationPeers()) { admin1.removeReplicationPeer(peer.getPeerId()); } } - @After + @AfterEach public void teardown() throws Exception { for (ReplicationPeerDescription peer : admin1.listReplicationPeers()) { admin1.removeReplicationPeer(peer.getPeerId()); @@ -164,9 +154,9 @@ public void testWhenExcludeCF() throws Exception { ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey()) .setReplicateAllUserTables(true).setExcludeTableCFsMap(excludeTableCFs).build(); admin1.addReplicationPeer(PEER_ID2, peerConfig); - Assert.assertTrue(peerConfig.needToReplicate(REPLICATE_TABLE)); - Assert.assertTrue(peerConfig.needToReplicate(REPLICATE_TABLE, CF_A)); - Assert.assertFalse(peerConfig.needToReplicate(REPLICATE_TABLE, CF_B)); + assertTrue(peerConfig.needToReplicate(REPLICATE_TABLE)); + assertTrue(peerConfig.needToReplicate(REPLICATE_TABLE, CF_A)); + assertFalse(peerConfig.needToReplicate(REPLICATE_TABLE, CF_B)); assertEquals(0, queueStorage.getAllHFileRefs().size()); @@ -235,7 +225,6 @@ public void testWhenExcludeNamespace() throws Exception { // Bulk load data into the table of the namespace that is not replicated. byte[] row = Bytes.toBytes("001"); - byte[] value = Bytes.toBytes("v1"); bulkLoadOnCluster(NO_REPLICATE_TABLE, CF_A); Threads.sleep(1000); @@ -261,7 +250,9 @@ private String createHFileForFamilies(byte[] family) throws IOException { .setType(Cell.Type.Put); HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(UTIL1.getConfiguration()); - File hFileLocation = testFolder.newFile(); + File randomDir = new File(UTIL1.getRandomDir().toString()).getAbsoluteFile(); + randomDir.mkdirs(); + File hFileLocation = new File(randomDir, "hfile"); FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation), null); try { hFileFactory.withOutputStream(out); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestClaimReplicationQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestClaimReplicationQueue.java index b8718f3526bc..2d25e8105772 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestClaimReplicationQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestClaimReplicationQueue.java @@ -21,7 +21,6 @@ import java.util.Collections; import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -35,11 +34,12 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.apache.hbase.thirdparty.com.google.common.io.Closeables; @@ -49,12 +49,9 @@ * In HBASE-26029, we reimplement the claim queue operation with proc-v2 and make it a step in SCP, * this is a UT to make sure the {@link ClaimReplicationQueuesProcedure} works correctly. */ -@Category({ ReplicationTests.class, LargeTests.class }) -public class TestClaimReplicationQueue extends TestReplicationBase { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestClaimReplicationQueue.class); +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) +public class TestClaimReplicationQueue extends TestReplicationBaseNoBeforeAll { private static final TableName tableName3 = TableName.valueOf("test3"); @@ -100,33 +97,31 @@ protected ServerManager createServerManager(MasterServices master, RegionServerL } } - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { CONF1.setClass(HConstants.MASTER_IMPL, HMasterForTest.class, HMaster.class); - TestReplicationBase.setUpBeforeClass(); + configureClusters(UTIL1, UTIL2); + startClusters(); createTable(tableName3); table3 = connection1.getTable(tableName3); table4 = connection2.getTable(tableName3); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { Closeables.close(table3, true); Closeables.close(table4, true); - TestReplicationBase.tearDownAfterClass(); } - @Override - public void setUpBase() throws Exception { - super.setUpBase(); + @BeforeEach + public void addExtraPeer() throws Exception { // set up two replication peers and only 1 rs to test claim replication queue with multiple // round addPeer(PEER_ID3, tableName3); } - @Override - public void tearDownBase() throws Exception { - super.tearDownBase(); + @AfterEach + public void removeExtraPeer() throws Exception { removePeer(PEER_ID3); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsBehindDroppedTableTiming.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsBehindDroppedTableTiming.java index 76ddbf5b6470..92c07ee0b3ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsBehindDroppedTableTiming.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsBehindDroppedTableTiming.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -31,19 +30,15 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestEditsBehindDroppedTableTiming extends ReplicationDroppedTablesTestBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestEditsBehindDroppedTableTiming.class); - - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { setupClusters(true); } @@ -58,58 +53,59 @@ public void testEditsBehindDroppedTableTiming() throws Exception { TableDescriptorBuilder.newBuilder(tablename).setColumnFamily(ColumnFamilyDescriptorBuilder .newBuilder(familyName).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build(); - Connection connection1 = ConnectionFactory.createConnection(CONF1); - Connection connection2 = ConnectionFactory.createConnection(CONF2); - try (Admin admin1 = connection1.getAdmin()) { - admin1.createTable(table); - } - try (Admin admin2 = connection2.getAdmin()) { - admin2.createTable(table); - } - UTIL1.waitUntilAllRegionsAssigned(tablename); - UTIL2.waitUntilAllRegionsAssigned(tablename); + try (Connection connection1 = ConnectionFactory.createConnection(CONF1); + Connection connection2 = ConnectionFactory.createConnection(CONF2)) { + try (Admin admin1 = connection1.getAdmin()) { + admin1.createTable(table); + } + try (Admin admin2 = connection2.getAdmin()) { + admin2.createTable(table); + } + UTIL1.waitUntilAllRegionsAssigned(tablename); + UTIL2.waitUntilAllRegionsAssigned(tablename); - // now suspend replication - try (Admin admin1 = connection1.getAdmin()) { - admin1.disableReplicationPeer(PEER_ID2); - } + // now suspend replication + try (Admin admin1 = connection1.getAdmin()) { + admin1.disableReplicationPeer(PEER_ID2); + } - // put some data (lead with 0 so the edit gets sorted before the other table's edits - // in the replication batch) write a bunch of edits, making sure we fill a batch - try (Table droppedTable = connection1.getTable(tablename)) { - byte[] rowKey = Bytes.toBytes(0 + " put on table to be dropped"); - Put put = new Put(rowKey); - put.addColumn(familyName, row, VALUE); - droppedTable.put(put); - } + // put some data (lead with 0 so the edit gets sorted before the other table's edits + // in the replication batch) write a bunch of edits, making sure we fill a batch + try (Table droppedTable = connection1.getTable(tablename)) { + byte[] rowKey = Bytes.toBytes(0 + " put on table to be dropped"); + Put put = new Put(rowKey); + put.addColumn(familyName, row, VALUE); + droppedTable.put(put); + } - try (Table table1 = connection1.getTable(tableName)) { - for (int i = 0; i < ROWS_COUNT; i++) { - Put put = new Put(generateRowKey(i)).addColumn(famName, row, VALUE); - table1.put(put); + try (Table table1 = connection1.getTable(tableName)) { + for (int i = 0; i < ROWS_COUNT; i++) { + Put put = new Put(generateRowKey(i)).addColumn(famName, row, VALUE); + table1.put(put); + } } - } - try (Admin admin2 = connection2.getAdmin()) { - admin2.disableTable(tablename); - admin2.deleteTable(tablename); - } + try (Admin admin2 = connection2.getAdmin()) { + admin2.disableTable(tablename); + admin2.deleteTable(tablename); + } - // edit should still be stuck - try (Admin admin1 = connection1.getAdmin()) { - // enable the replication peer. - admin1.enableReplicationPeer(PEER_ID2); - // the source table still exists, replication should be stalled - verifyReplicationStuck(); + // edit should still be stuck + try (Admin admin1 = connection1.getAdmin()) { + // enable the replication peer. + admin1.enableReplicationPeer(PEER_ID2); + // the source table still exists, replication should be stalled + verifyReplicationStuck(); - admin1.disableTable(tablename); - // still stuck, source table still exists - verifyReplicationStuck(); + admin1.disableTable(tablename); + // still stuck, source table still exists + verifyReplicationStuck(); - admin1.deleteTable(tablename); - // now the source table is gone, replication should proceed, the - // offending edits be dropped - verifyReplicationProceeded(); + admin1.deleteTable(tablename); + // now the source table is gone, replication should proceed, the + // offending edits be dropped + verifyReplicationProceeded(); + } } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsDroppedWithDroppedTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsDroppedWithDroppedTable.java index 9548d3a56b55..188a8ea14005 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsDroppedWithDroppedTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsDroppedWithDroppedTable.java @@ -17,22 +17,17 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestEditsDroppedWithDroppedTable extends ReplicationDroppedTablesTestBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestEditsDroppedWithDroppedTable.class); - - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { setupClusters(true); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsDroppedWithDroppedTableNS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsDroppedWithDroppedTableNS.java index aa153471ca13..a6e14098416b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsDroppedWithDroppedTableNS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsDroppedWithDroppedTableNS.java @@ -17,23 +17,18 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestEditsDroppedWithDroppedTableNS extends ReplicationDroppedTablesTestBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestEditsDroppedWithDroppedTableNS.class); - - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { setupClusters(true); // also try with a namespace diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsStuckBehindDroppedTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsStuckBehindDroppedTable.java index 924ba72861bd..f86199e8bac6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsStuckBehindDroppedTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestEditsStuckBehindDroppedTable.java @@ -17,22 +17,17 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestEditsStuckBehindDroppedTable extends ReplicationDroppedTablesTestBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestEditsStuckBehindDroppedTable.class); - - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { setupClusters(false); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 939f02a8fa42..c2241a636618 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -17,10 +17,12 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.fail; import java.io.Closeable; import java.io.IOException; @@ -34,7 +36,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -70,22 +71,17 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestMasterReplication { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMasterReplication.class); - private static final Logger LOG = LoggerFactory.getLogger(TestMasterReplication.class); private Configuration baseConfiguration; @@ -113,7 +109,7 @@ public class TestMasterReplication { private TableDescriptor table; - @Before + @BeforeEach public void setUp() throws Exception { baseConfiguration = HBaseConfiguration.create(); // smaller block size and capacity to trigger more operations @@ -173,12 +169,12 @@ public void testCyclicReplication1() throws Exception { * {@link org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint}, * the replication peer should not be added. */ - @Test(expected = DoNotRetryIOException.class) + @Test public void testLoopedReplication() throws Exception { LOG.info("testLoopedReplication"); startMiniClusters(1); createTableOnClusters(table); - addPeer("1", 0, 0); + assertThrows(DoNotRetryIOException.class, () -> addPeer("1", 0, 0)); } /** @@ -462,9 +458,9 @@ public void testBasePeerConfigsForReplicationPeer() throws Exception { Admin admin = utilities[0].getAdmin(); // Validates base configs 1 is present for both peer. - Assert.assertEquals(firstCustomPeerConfigValue, + assertEquals(firstCustomPeerConfigValue, admin.getReplicationPeerConfig("1").getConfiguration().get(firstCustomPeerConfigKey)); - Assert.assertEquals(firstCustomPeerConfigValue, + assertEquals(firstCustomPeerConfigValue, admin.getReplicationPeerConfig("2").getConfiguration().get(firstCustomPeerConfigKey)); // override value of configuration 1 for peer "1". @@ -481,9 +477,9 @@ public void testBasePeerConfigsForReplicationPeer() throws Exception { admin.updateReplicationPeerConfig("2", updatedReplicationConfigForPeer2); // validates configuration is overridden by updateReplicationPeerConfig - Assert.assertEquals(firstCustomPeerConfigUpdatedValue, + assertEquals(firstCustomPeerConfigUpdatedValue, admin.getReplicationPeerConfig("1").getConfiguration().get(firstCustomPeerConfigKey)); - Assert.assertEquals(secondCustomPeerConfigUpdatedValue, + assertEquals(secondCustomPeerConfigUpdatedValue, admin.getReplicationPeerConfig("2").getConfiguration().get(secondCustomPeerConfigKey)); // Add second config to base config and perform restart. @@ -497,14 +493,14 @@ public void testBasePeerConfigsForReplicationPeer() throws Exception { admin = utilities[0].getAdmin(); // Configurations should be updated after restart again - Assert.assertEquals(firstCustomPeerConfigValue, + assertEquals(firstCustomPeerConfigValue, admin.getReplicationPeerConfig("1").getConfiguration().get(firstCustomPeerConfigKey)); - Assert.assertEquals(firstCustomPeerConfigValue, + assertEquals(firstCustomPeerConfigValue, admin.getReplicationPeerConfig("2").getConfiguration().get(firstCustomPeerConfigKey)); - Assert.assertEquals(secondCustomPeerConfigValue, + assertEquals(secondCustomPeerConfigValue, admin.getReplicationPeerConfig("1").getConfiguration().get(secondCustomPeerConfigKey)); - Assert.assertEquals(secondCustomPeerConfigValue, + assertEquals(secondCustomPeerConfigValue, admin.getReplicationPeerConfig("2").getConfiguration().get(secondCustomPeerConfigKey)); } finally { shutDownMiniClusters(); @@ -526,7 +522,7 @@ public void testBasePeerConfigsRemovalForReplicationPeer() throws Exception { Admin admin = utilities[0].getAdmin(); // Validates base configs 1 is present for both peer. - Assert.assertEquals(firstCustomPeerConfigValue, + assertEquals(firstCustomPeerConfigValue, admin.getReplicationPeerConfig("1").getConfiguration().get(firstCustomPeerConfigKey)); utilities[0].getConfiguration() @@ -540,7 +536,7 @@ public void testBasePeerConfigsRemovalForReplicationPeer() throws Exception { admin = utilities[0].getAdmin(); // Configurations should be removed after restart again - Assert.assertNull( + assertNull( admin.getReplicationPeerConfig("1").getConfiguration().get(firstCustomPeerConfigKey)); } finally { shutDownMiniClusters(); @@ -560,15 +556,16 @@ public void testRemoveBasePeerConfigWithoutExistingConfigForReplicationPeer() th addPeer("1", 0, 1); Admin admin = utilities[0].getAdmin(); - Assert.assertNull("Config should not be there", - admin.getReplicationPeerConfig("1").getConfiguration().get(firstCustomPeerConfigKey)); + assertNull( + admin.getReplicationPeerConfig("1").getConfiguration().get(firstCustomPeerConfigKey), + "Config should not be there"); } finally { shutDownMiniClusters(); baseConfiguration.unset(ReplicationPeerConfigUtil.HBASE_REPLICATION_PEER_BASE_CONFIG); } } - @After + @AfterEach public void tearDown() throws IOException { configurations = null; utilities = null; @@ -671,8 +668,8 @@ private Table[] getHTablesOnClusters(TableName tableName) throws Exception { private void validateCounts(Table[] htables, byte[] type, int[] expectedCounts) throws IOException { for (int i = 0; i < htables.length; i++) { - assertEquals(Bytes.toString(type) + " were replicated back ", expectedCounts[i], - getCount(htables[i], type)); + assertEquals(expectedCounts[i], getCount(htables[i], type), + Bytes.toString(type) + " were replicated back"); } } @@ -735,7 +732,7 @@ private void wait(int slaveNumber, Table target, int expectedCount) fail("Waited too much time for bulkloaded data replication. Current count=" + count + ", expected count=" + expectedCount); } - count = utilities[slaveNumber].countRows(target); + count = utilities[0].countRows(target); if (count != expectedCount) { LOG.info("Waiting more time for bulkloaded data replication."); Thread.sleep(SLEEP_TIME); @@ -781,7 +778,7 @@ private void rollWALAndWait(final HBaseTestingUtility utility, final TableName t break; } } - assertNotNull("Couldn't find the region for row '" + Arrays.toString(row) + "'", region); + assertNotNull(region, "Couldn't find the region for row '" + Arrays.toString(row) + "'"); final CountDownLatch latch = new CountDownLatch(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMigrateRepliationPeerStorageOnline.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMigrateRepliationPeerStorageOnline.java index 7b0b10f2008f..59c5cc868c38 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMigrateRepliationPeerStorageOnline.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMigrateRepliationPeerStorageOnline.java @@ -19,12 +19,11 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.empty; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -32,22 +31,18 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.util.ToolRunner; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestMigrateRepliationPeerStorageOnline { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMigrateRepliationPeerStorageOnline.class); - private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - @BeforeClass + @BeforeAll public static void setUp() throws Exception { // use zookeeper first, and then migrate to filesystem UTIL.getConfiguration().set(ReplicationStorageFactory.REPLICATION_PEER_STORAGE_IMPL, @@ -55,7 +50,7 @@ public static void setUp() throws Exception { UTIL.startMiniCluster(1); } - @AfterClass + @AfterAll public static void tearDown() throws IOException { UTIL.shutdownMiniCluster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index 4a86f5e647ee..26e6dc5a2a15 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -17,17 +17,16 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.Arrays; import java.util.concurrent.CountDownLatch; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -50,20 +49,16 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestMultiSlaveReplication { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMultiSlaveReplication.class); - private static final Logger LOG = LoggerFactory.getLogger(TestMultiSlaveReplication.class); private static Configuration conf1; @@ -86,7 +81,7 @@ public class TestMultiSlaveReplication { private static HTableDescriptor table; - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { conf1 = HBaseConfiguration.create(); conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); @@ -218,7 +213,7 @@ private void rollWALAndWait(final HBaseTestingUtility utility, final TableName t break; } } - assertNotNull("Couldn't find the region for row '" + Arrays.toString(row) + "'", region); + assertNotNull(region, "Couldn't find the region for row '" + Arrays.toString(row) + "'"); final CountDownLatch latch = new CountDownLatch(1); @@ -257,8 +252,8 @@ private void checkWithWait(byte[] row, int count, Table table) throws Exception if (res.size() >= 1) { LOG.info("Row is replicated"); rowReplicated = true; - assertEquals("Table '" + table + "' did not have the expected number of results.", count, - res.size()); + assertEquals(count, res.size(), + "Table '" + table + "' did not have the expected number of results."); break; } if (rowReplicated) { @@ -273,8 +268,8 @@ private void checkRow(byte[] row, int count, Table... tables) throws IOException Get get = new Get(row); for (Table table : tables) { Result res = table.get(get); - assertEquals("Table '" + table + "' did not have the expected number of results.", count, - res.size()); + assertEquals(count, res.size(), + "Table '" + table + "' did not have the expected number of results."); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java index 8fe01dcd17da..a2a2bc9015cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.util.ArrayList; import java.util.HashMap; @@ -27,7 +27,8 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.hadoop.hbase.HBaseClassTestRule; +import java.util.stream.Stream; +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; @@ -43,29 +44,21 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.params.provider.Arguments; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; - -@RunWith(Parameterized.class) -@Category({ LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: serialPeer={0}") public class TestNamespaceReplication extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestNamespaceReplication.class); - private static final Logger LOG = LoggerFactory.getLogger(TestNamespaceReplication.class); private static String ns1 = "ns1"; @@ -84,23 +77,23 @@ public class TestNamespaceReplication extends TestReplicationBase { private static Admin admin1; private static Admin admin2; - @Parameter - public boolean serialPeer; + private boolean serialPeer; + + public TestNamespaceReplication(boolean serialPeer) { + this.serialPeer = serialPeer; + } @Override protected boolean isSerialPeer() { return serialPeer; } - @Parameters(name = "{index}: serialPeer={0}") - public static List parameters() { - return ImmutableList.of(true, false); + public static Stream parameters() { + return Stream.of(Arguments.of(true), Arguments.of(false)); } - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { - TestReplicationBase.setUpBeforeClass(); - connection1 = ConnectionFactory.createConnection(CONF1); connection2 = ConnectionFactory.createConnection(CONF2); admin1 = connection1.getAdmin(); @@ -130,7 +123,7 @@ public static void setUpBeforeClass() throws Exception { admin2.createTable(tabB); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { admin1.disableTable(tabAName); admin1.deleteTable(tabAName); @@ -148,10 +141,9 @@ public static void tearDownAfterClass() throws Exception { connection1.close(); connection2.close(); - TestReplicationBase.tearDownAfterClass(); } - @Test + @TestTemplate public void testNamespaceReplication() throws Exception { String peerId = "2"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplicationWithBulkLoadedData.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplicationWithBulkLoadedData.java index 1552b277792f..fb5204fbd90c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplicationWithBulkLoadedData.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplicationWithBulkLoadedData.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.HashMap; import java.util.HashSet; @@ -25,7 +25,6 @@ import java.util.Map; import java.util.Set; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -37,31 +36,24 @@ 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.regionserver.TestBulkLoadReplication; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.junit.After; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Testcase for HBASE-23098 */ -// LargeTest because spins up four clusters. -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public final class TestNamespaceReplicationWithBulkLoadedData extends TestBulkLoadReplication { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestNamespaceReplicationWithBulkLoadedData.class); + private static final Logger LOG = LoggerFactory.getLogger(TestNamespaceReplicationWithBulkLoadedData.class); @@ -78,44 +70,20 @@ public final class TestNamespaceReplicationWithBulkLoadedData extends TestBulkLo private static final TableName NS1_TABLE = TableName.valueOf(NS1 + ":t1_syncup"); private static final TableName NS2_TABLE = TableName.valueOf(NS2 + ":t2_syncup"); - @BeforeClass - public static void setUpBeforeClass() throws Exception { + @BeforeAll + public static void setUpBeforeAll() throws Exception { setupBulkLoadConfigsForCluster(CONF4, PEER4_CLUSTER_ID); setupConfig(UTIL4, "/4"); - TestBulkLoadReplication.setUpBeforeClass(); startFourthCluster(); - } - - private static void startFourthCluster() throws Exception { - LOG.info("Setup Zk to same one from UTIL1 and UTIL2 and UTIL3"); - UTIL4.setZkCluster(UTIL1.getZkCluster()); - UTIL4.startMiniCluster(NUM_SLAVES1); - - TableDescriptor table = TableDescriptorBuilder.newBuilder(tableName) - .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName).setMaxVersions(100) - .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()) - .setColumnFamily(ColumnFamilyDescriptorBuilder.of(noRepfamName)).build(); - Connection connection4 = ConnectionFactory.createConnection(CONF4); - try (Admin admin4 = connection4.getAdmin()) { - admin4.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); - } - UTIL4.waitUntilAllRegionsAssigned(tableName); - } - - @Before - @Override - public void setUpBase() throws Exception { /** - * "super.setUpBase()" already sets peer1 from 1 <-> 2 <-> 3 and this test add the fourth - * cluster. So we have following topology: 1 / \ 2 4 / 3 The 1 -> 4 has two peers, ns_peer1: ns1 - * -> ns1 (validate this peer hfile-refs) ns_peer1 configuration is NAMESPACES => ["ns1"] - * ns_peer2: ns2:t2_syncup -> ns2:t2_syncup, this peers is ns_peer2 configuration is NAMESPACES - * => ["ns2"], TABLE_CFS => { "ns2:t2_syncup" => []} The 1 -> 2 has one peer, this peer + * Parent class already sets peer1 from 1 <-> 2 <-> 3 and this test add the fourth cluster. So + * we have following topology: 1 / \ 2 4 / 3 The 1 -> 4 has two peers, ns_peer1: ns1 -> ns1 + * (validate this peer hfile-refs) ns_peer1 configuration is NAMESPACES => ["ns1"] ns_peer2: + * ns2:t2_syncup -> ns2:t2_syncup, this peers is ns_peer2 configuration is NAMESPACES => + * ["ns2"], TABLE_CFS => { "ns2:t2_syncup" => []} The 1 -> 2 has one peer, this peer * configuration is add_peer '2', CLUSTER_KEY => "server1.cie.com:2181:/hbase" */ - super.setUpBase(); - // Create tables TableDescriptor table1 = TableDescriptorBuilder.newBuilder(NS1_TABLE) .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName) @@ -174,52 +142,26 @@ public void setUpBase() throws Exception { admin1.addReplicationPeer(PEER4_NS_TABLE, rpc4_ns_table); } - @After - @Override - public void tearDownBase() throws Exception { - super.tearDownBase(); - TableDescriptor table1 = TableDescriptorBuilder.newBuilder(NS1_TABLE) - .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName) - .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()) - .setColumnFamily(ColumnFamilyDescriptorBuilder.of(noRepfamName)).build(); + @AfterAll + public static void tearDownAfterAll() throws Exception { + UTIL4.shutdownMiniCluster(); + } - TableDescriptor table2 = TableDescriptorBuilder.newBuilder(NS2_TABLE) - .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName) + private static void startFourthCluster() throws Exception { + LOG.info("Setup Zk to same one from UTIL1 and UTIL2 and UTIL3"); + UTIL4.setZkCluster(UTIL1.getZkCluster()); + UTIL4.startMiniCluster(NUM_SLAVES1); + + TableDescriptor table = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName).setMaxVersions(100) .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(noRepfamName)).build(); - Admin admin1 = UTIL1.getAdmin(); - admin1.disableTable(table1.getTableName()); - admin1.deleteTable(table1.getTableName()); - admin1.disableTable(table2.getTableName()); - admin1.deleteTable(table2.getTableName()); - admin1.deleteNamespace(NS1); - admin1.deleteNamespace(NS2); - - Admin admin2 = UTIL2.getAdmin(); - admin2.disableTable(table1.getTableName()); - admin2.deleteTable(table1.getTableName()); - admin2.disableTable(table2.getTableName()); - admin2.deleteTable(table2.getTableName()); - admin2.deleteNamespace(NS1); - admin2.deleteNamespace(NS2); - Admin admin3 = UTIL3.getAdmin(); - admin3.disableTable(table1.getTableName()); - admin3.deleteTable(table1.getTableName()); - admin3.disableTable(table2.getTableName()); - admin3.deleteTable(table2.getTableName()); - admin3.deleteNamespace(NS1); - admin3.deleteNamespace(NS2); - - Admin admin4 = UTIL4.getAdmin(); - admin4.disableTable(table1.getTableName()); - admin4.deleteTable(table1.getTableName()); - admin4.disableTable(table2.getTableName()); - admin4.deleteTable(table2.getTableName()); - admin4.deleteNamespace(NS1); - admin4.deleteNamespace(NS2); - UTIL1.getAdmin().removeReplicationPeer(PEER4_NS); - UTIL1.getAdmin().removeReplicationPeer(PEER4_NS_TABLE); + Connection connection4 = ConnectionFactory.createConnection(CONF4); + try (Admin admin4 = connection4.getAdmin()) { + admin4.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); + } + UTIL4.waitUntilAllRegionsAssigned(tableName); } @Test @@ -256,9 +198,7 @@ public void testBulkLoadReplicationActiveActive() throws Exception { assertTableNoValue(notPeerTable, row, value); // 1 -> 4, table is empty // Verify hfile-refs for 1:ns_peer1, expect is empty - MiniZooKeeperCluster zkCluster = UTIL1.getZkCluster(); ZKWatcher watcher = new ZKWatcher(UTIL1.getConfiguration(), "TestZnodeHFiles-refs", null); - RecoverableZooKeeper zk = RecoverableZooKeeper.connect(UTIL1.getConfiguration(), watcher); ZKReplicationQueueStorage replicationQueueStorage = new ZKReplicationQueueStorage(watcher, UTIL1.getConfiguration()); Set hfiles = replicationQueueStorage.getAllHFileRefs(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java index 080489f97022..65e7d32560a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java @@ -22,7 +22,6 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -36,23 +35,19 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; -@Category({ MediumTests.class, ReplicationTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestNonHBaseReplicationEndpoint { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestNonHBaseReplicationEndpoint.class); - - protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static Admin ADMIN; @@ -61,18 +56,18 @@ public class TestNonHBaseReplicationEndpoint { private static final AtomicBoolean REPLICATED = new AtomicBoolean(); - @BeforeClass + @BeforeAll public static void setupBeforeClass() throws Exception { UTIL.startMiniCluster(); ADMIN = UTIL.getAdmin(); } - @AfterClass + @AfterAll public static void teardownAfterClass() throws Exception { UTIL.shutdownMiniCluster(); } - @Before + @BeforeEach public void setup() { REPLICATED.set(false); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java index 6964a9ad7eab..e42e7affbf67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java @@ -17,7 +17,13 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.ArrayList; @@ -25,13 +31,13 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; 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.TableName; +import org.apache.hadoop.hbase.TableNameTestExtension; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -43,30 +49,25 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; -@Category({ FlakeyTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestPerTableCFReplication { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestPerTableCFReplication.class); - private static final Logger LOG = LoggerFactory.getLogger(TestPerTableCFReplication.class); private static Configuration conf1; @@ -79,28 +80,24 @@ public class TestPerTableCFReplication { private static final long SLEEP_TIME = 500; private static final int NB_RETRIES = 100; - private static final TableName tableName = TableName.valueOf("test"); private static final TableName tabAName = TableName.valueOf("TA"); private static final TableName tabBName = TableName.valueOf("TB"); private static final TableName tabCName = TableName.valueOf("TC"); - private static final byte[] famName = Bytes.toBytes("f"); private static final byte[] f1Name = Bytes.toBytes("f1"); private static final byte[] f2Name = Bytes.toBytes("f2"); private static final byte[] f3Name = Bytes.toBytes("f3"); private static final byte[] row1 = Bytes.toBytes("row1"); private static final byte[] row2 = Bytes.toBytes("row2"); - private static final byte[] noRepfamName = Bytes.toBytes("norep"); private static final byte[] val = Bytes.toBytes("myval"); - private static HTableDescriptor table; private static HTableDescriptor tabA; private static HTableDescriptor tabB; private static HTableDescriptor tabC; - @Rule - public TestName name = new TestName(); + @RegisterExtension + private final TableNameTestExtension tableNameExt = new TableNameTestExtension(); - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { conf1 = HBaseConfiguration.create(); conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); @@ -118,7 +115,7 @@ public static void setUpBeforeClass() throws Exception { utility1 = new HBaseTestingUtility(conf1); utility1.startMiniZKCluster(); MiniZooKeeperCluster miniZK = utility1.getZkCluster(); - new ZKWatcher(conf1, "cluster1", null, true); + new ZKWatcher(conf1, "cluster1", null, true).close(); conf2 = new Configuration(conf1); conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2"); @@ -128,21 +125,14 @@ public static void setUpBeforeClass() throws Exception { utility2 = new HBaseTestingUtility(conf2); utility2.setZkCluster(miniZK); - new ZKWatcher(conf2, "cluster3", null, true); + new ZKWatcher(conf2, "cluster3", null, true).close(); utility3 = new HBaseTestingUtility(conf3); utility3.setZkCluster(miniZK); - new ZKWatcher(conf3, "cluster3", null, true); - - table = new HTableDescriptor(tableName); - HColumnDescriptor fam = new HColumnDescriptor(famName); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - table.addFamily(fam); - fam = new HColumnDescriptor(noRepfamName); - table.addFamily(fam); + new ZKWatcher(conf3, "cluster3", null, true).close(); tabA = new HTableDescriptor(tabAName); - fam = new HColumnDescriptor(f1Name); + HColumnDescriptor fam = new HColumnDescriptor(f1Name); fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); tabA.addFamily(fam); fam = new HColumnDescriptor(f2Name); @@ -179,7 +169,7 @@ public static void setUpBeforeClass() throws Exception { utility3.startMiniCluster(); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { utility3.shutdownMiniCluster(); utility2.shutdownMiniCluster(); @@ -200,9 +190,9 @@ public void testParseTableCFsFromConfig() { tabCFsMap = ReplicationPeerConfigUtil.parseTableCFsFromConfig(" "); assertEquals(null, tabCFsMap); - final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1"); - final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2"); - final TableName tableName3 = TableName.valueOf(name.getMethodName() + "3"); + final TableName tableName1 = tableNameExt.getTableName("1"); + final TableName tableName2 = tableNameExt.getTableName("2"); + final TableName tableName3 = tableNameExt.getTableName("3"); // 2. single table: "tableName1" / "tableName2:cf1" / "tableName3:cf1,cf3" tabCFsMap = ReplicationPeerConfigUtil.parseTableCFsFromConfig(tableName1.getNameAsString()); @@ -291,9 +281,9 @@ public void testTableCFsHelperConverter() { tableCFs = ReplicationPeerConfigUtil.convert(tabCFsMap); assertEquals(0, tableCFs.length); - final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1"); - final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2"); - final TableName tableName3 = TableName.valueOf(name.getMethodName() + "3"); + final TableName tableName1 = tableNameExt.getTableName("1"); + final TableName tableName2 = tableNameExt.getTableName("2"); + final TableName tableName3 = tableNameExt.getTableName("3"); // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3" tabCFsMap.clear(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java index 23f4dca33447..8d73b6138cb4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java @@ -17,11 +17,10 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.IOException; import java.util.Collections; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; @@ -32,24 +31,20 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; /** * Testcase for HBASE-20296. */ -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestRemoveFromSerialReplicationPeer extends SerialReplicationTestBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestRemoveFromSerialReplicationPeer.class); - - @Before + @BeforeEach public void setUp() throws IOException, StreamLacksCapabilityException { setupWALWriter(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java index 02924dcaa616..636825e6157a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.replication; -import org.junit.BeforeClass; import org.junit.jupiter.api.BeforeAll; /** @@ -28,7 +27,6 @@ public class TestReplicationBase extends TestReplicationBaseNoBeforeAll { @BeforeAll - @BeforeClass public static void setUpBeforeClass() throws Exception { configureClusters(UTIL1, UTIL2); startClusters(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBaseNoBeforeAll.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBaseNoBeforeAll.java index 96cc8e23ac7a..33e1b178d8d5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBaseNoBeforeAll.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBaseNoBeforeAll.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.ArrayList; @@ -49,9 +49,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.slf4j.Logger; @@ -289,7 +287,6 @@ protected final void addPeer(String peerId, TableName tableName, HBaseTestingUti source.getAdmin().addReplicationPeer(peerId, builder.build()); } - @Before @BeforeEach public void setUpBase() throws Exception { addPeer(PEER_ID2, tableName); @@ -305,7 +302,6 @@ protected final void removePeer(String peerId, HBaseTestingUtility util) throws } } - @After @AfterEach public void tearDownBase() throws Exception { removePeer(PEER_ID2); @@ -365,7 +361,7 @@ protected static void runSmallBatchTest() throws IOException, InterruptedExcepti waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { if (htable2 != null) { htable2.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java index 1ccc4bd9e3ed..3805d17a5b21 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java @@ -17,13 +17,13 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; -import java.util.List; -import org.apache.hadoop.hbase.HBaseClassTestRule; +import java.util.stream.Stream; +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Get; @@ -35,48 +35,40 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.params.provider.Arguments; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; - /** * Test handling of changes to the number of a peer's regionservers. */ -@RunWith(Parameterized.class) -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: serialPeer={0}") public class TestReplicationChangingPeerRegionservers extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationChangingPeerRegionservers.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationChangingPeerRegionservers.class); - @SuppressWarnings("checkstyle:VisibilityModifier") - @Parameter - public boolean serialPeer; + private boolean serialPeer; + + public TestReplicationChangingPeerRegionservers(boolean serialPeer) { + this.serialPeer = serialPeer; + } @Override protected boolean isSerialPeer() { return serialPeer; } - @Parameters(name = "{index}: serialPeer={0}") - public static List parameters() { - return ImmutableList.of(true, false); + public static Stream parameters() { + return Stream.of(Arguments.of(true), Arguments.of(false)); } - @Before + @BeforeEach public void setUp() throws Exception { // Starting and stopping replication can make us miss new logs, // rolling like this makes sure the most recent one gets added to the queue @@ -111,7 +103,7 @@ public void setUp() throws Exception { } } - @Test + @TestTemplate public void testChangingNumberOfPeerRegionServers() throws IOException, InterruptedException { LOG.info("testSimplePutDelete"); MiniHBaseCluster peerCluster = UTIL2.getMiniHBaseCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java index 8c7da7e0900f..aabd0f4c126f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java @@ -17,29 +17,24 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.fail; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestReplicationDisableInactivePeer extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationDisableInactivePeer.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationDisableInactivePeer.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEditsDroppedWithDeletedTableCFs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEditsDroppedWithDeletedTableCFs.java index ed19f233ab05..310269369967 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEditsDroppedWithDeletedTableCFs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEditsDroppedWithDeletedTableCFs.java @@ -20,13 +20,12 @@ import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_GLOBAL; import static org.apache.hadoop.hbase.HConstants.ZOOKEEPER_ZNODE_PARENT; import static org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint.REPLICATION_DROP_ON_DELETED_COLUMN_FAMILY_KEY; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.Arrays; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; @@ -40,26 +39,23 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestReplicationEditsDroppedWithDeletedTableCFs { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationEditsDroppedWithDeletedTableCFs.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationEditsDroppedWithDeletedTableCFs.class); @@ -84,7 +80,7 @@ public class TestReplicationEditsDroppedWithDeletedTableCFs { private static final long SLEEP_TIME = 1000; private static final int NB_RETRIES = 10; - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { // Set true to filter replication edits for dropped table conf1.setBoolean(REPLICATION_DROP_ON_DELETED_COLUMN_FAMILY_KEY, true); @@ -107,13 +103,13 @@ public static void setUpBeforeClass() throws Exception { admin2 = utility2.getAdmin(); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { utility2.shutdownMiniCluster(); utility1.shutdownMiniCluster(); } - @Before + @BeforeEach public void setup() throws Exception { // Roll log for (JVMClusterUtil.RegionServerThread r : utility1.getHBaseCluster() @@ -128,7 +124,7 @@ public void setup() throws Exception { createTable(); } - @After + @AfterEach public void tearDown() throws Exception { // Remove peer admin1.removeReplicationPeer(PEER_ID); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEditsDroppedWithDroppedTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEditsDroppedWithDroppedTable.java index 3724ddd8aed2..d3b33d3ba382 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEditsDroppedWithDroppedTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEditsDroppedWithDroppedTable.java @@ -17,10 +17,9 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -37,26 +36,23 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestReplicationEditsDroppedWithDroppedTable { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationEditsDroppedWithDroppedTable.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationEditsDroppedWithDroppedTable.class); @@ -82,7 +78,7 @@ public class TestReplicationEditsDroppedWithDroppedTable { private static final long SLEEP_TIME = 1000; private static final int NB_RETRIES = 10; - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { // Set true to filter replication edits for dropped table conf1.setBoolean(HBaseInterClusterReplicationEndpoint.REPLICATION_DROP_ON_DELETED_TABLE_KEY, @@ -110,13 +106,13 @@ public static void setUpBeforeClass() throws Exception { admin2.createNamespace(nsDesc); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { utility2.shutdownMiniCluster(); utility1.shutdownMiniCluster(); } - @Before + @BeforeEach public void setup() throws Exception { // Roll log for (JVMClusterUtil.RegionServerThread r : utility1.getHBaseCluster() @@ -131,7 +127,7 @@ public void setup() throws Exception { createTable(NORMAL_TABLE); } - @After + @AfterEach public void tearDown() throws Exception { // Remove peer admin1.removeReplicationPeer(PEER_ID); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java index 63cbfe3119c4..ff38ce9f0d5d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java @@ -17,13 +17,14 @@ */ package org.apache.hadoop.hbase.replication; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Waiter; @@ -43,23 +44,19 @@ import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKeyImpl; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestReplicationEmptyWALRecovery extends TestReplicationBase { + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); static final RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationEmptyWALRecovery.class); - - @Before + @BeforeEach public void setUp() throws IOException, InterruptedException { cleanUp(); scopes.put(famName, HConstants.REPLICATION_SCOPE_GLOBAL); @@ -198,12 +195,12 @@ public void testReplicationOfEmptyWALFollowingNonEmptyWAL() throws Exception { // Now we should expect numOfEntriesToReplicate entries // replicated from each region server. This makes sure we didn't loose data // from any previous batch when we encounter EOF exception for empty file. - Assert.assertEquals("Replicated entries are not correct", numOfEntriesToReplicate * numRs, - replicatedEntries.size()); + assertEquals(numOfEntriesToReplicate * numRs, replicatedEntries.size(), + "Replicated entries are not correct"); // We expect just one batch of replication which will // be from when we handle the EOF exception. - Assert.assertEquals("Replicated batches are not correct", 1, replicateCount.intValue()); + assertEquals(1, replicateCount.intValue(), "Replicated batches are not correct"); verifyNumberOfLogsInQueue(1, numRs); // we're now writing to the new wal // if everything works, the source should've stopped reading from the empty wal, and start @@ -253,12 +250,12 @@ public void testReplicationOfEmptyWALFollowedByNonEmptyWAL() throws Exception { // Now we should expect numOfEntriesToReplicate entries // replicated from each region server. This makes sure we didn't loose data // from any previous batch when we encounter EOF exception for empty file. - Assert.assertEquals("Replicated entries are not correct", numOfEntriesToReplicate * numRs, - replicatedEntries.size()); + assertEquals(numOfEntriesToReplicate * numRs, replicatedEntries.size(), + "Replicated entries are not correct"); // We expect just one batch of replication to be shipped which will // for non empty WAL - Assert.assertEquals("Replicated batches are not correct", 1, replicateCount.get()); + assertEquals(1, replicateCount.get(), "Replicated batches are not correct"); verifyNumberOfLogsInQueue(1, numRs); // we're now writing to the new wal // if everything works, the source should've stopped reading from the empty wal, and start @@ -310,12 +307,12 @@ public void testReplicationOfEmptyWALSurroundedNonEmptyWAL() throws Exception { // Now we should expect numOfEntriesToReplicate entries // replicated from each region server. This makes sure we didn't loose data // from any previous batch when we encounter EOF exception for empty file. - Assert.assertEquals("Replicated entries are not correct", numOfEntriesToReplicate * numRs * 2, - replicatedEntries.size()); + assertEquals(numOfEntriesToReplicate * numRs * 2, replicatedEntries.size(), + "Replicated entries are not correct"); // We expect two batch of replication to be shipped which will // for non empty WAL - Assert.assertEquals("Replicated batches are not correct", 2, replicateCount.get()); + assertEquals(2, replicateCount.get(), "Replicated batches are not correct"); verifyNumberOfLogsInQueue(1, numRs); // we're now writing to the new wal // if everything works, the source should've stopped reading from the empty wal, and start diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java index 1b87129cf5d5..f8f14c497ef2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java @@ -22,9 +22,6 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Tag; -/** - * - */ @Tag(ReplicationTests.TAG) @Tag(LargeTests.TAG) public class TestReplicationKillSlaveRS extends ReplicationKillSlaveRSTestBase { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationMetricsforUI.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationMetricsforUI.java index a5dc1490fc65..2a0f5e33f96a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationMetricsforUI.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationMetricsforUI.java @@ -17,9 +17,12 @@ */ package org.apache.hadoop.hbase.replication; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + import java.util.Map; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -28,16 +31,13 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationMetricsforUI extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationMetricsforUI.class); + private static final byte[] qualName = Bytes.toBytes("q"); @Test @@ -54,13 +54,13 @@ public void testReplicationMetrics() throws Exception { Thread.sleep(5000); HRegionServer rs = UTIL1.getRSForFirstRegionInTable(tableName); Map metrics = rs.getWalGroupsReplicationStatus(); - Assert.assertEquals("metric size ", 1, metrics.size()); + assertEquals(1, metrics.size(), "metric size"); long lastPosition = 0; for (Map.Entry metric : metrics.entrySet()) { - Assert.assertEquals("peerId", PEER_ID2, metric.getValue().getPeerId()); - Assert.assertEquals("queue length", 1, metric.getValue().getQueueSize()); - Assert.assertEquals("replication delay", 0, metric.getValue().getReplicationDelay()); - Assert.assertTrue("current position >= 0", metric.getValue().getCurrentPosition() >= 0); + assertEquals(PEER_ID2, metric.getValue().getPeerId(), "peerId"); + assertEquals(1, metric.getValue().getQueueSize(), "queue length"); + assertEquals(0, metric.getValue().getReplicationDelay(), "replication delay"); + assertTrue(metric.getValue().getCurrentPosition() >= 0, "current position >= 0"); lastPosition = metric.getValue().getCurrentPosition(); } for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { @@ -78,11 +78,11 @@ public void testReplicationMetrics() throws Exception { Path lastPath = null; for (Map.Entry metric : metrics.entrySet()) { lastPath = metric.getValue().getCurrentPath(); - Assert.assertEquals("peerId", PEER_ID2, metric.getValue().getPeerId()); - Assert.assertTrue("age of Last Shipped Op should be > 0 ", - metric.getValue().getAgeOfLastShippedOp() > 0); - Assert.assertTrue("current position should > last position", - metric.getValue().getCurrentPosition() - lastPosition > 0); + assertEquals(PEER_ID2, metric.getValue().getPeerId(), "peerId"); + assertTrue(metric.getValue().getAgeOfLastShippedOp() > 0, + "age of Last Shipped Op should be > 0"); + assertTrue(metric.getValue().getCurrentPosition() - lastPosition > 0, + "current position should > last position"); lastPosition = metric.getValue().getCurrentPosition(); } @@ -98,10 +98,10 @@ public void testReplicationMetrics() throws Exception { Thread.sleep(5000); metrics = rs.getWalGroupsReplicationStatus(); for (Map.Entry metric : metrics.entrySet()) { - Assert.assertEquals("replication delay", 0, metric.getValue().getReplicationDelay()); - Assert.assertTrue("current position should < last position", - metric.getValue().getCurrentPosition() < lastPosition); - Assert.assertNotEquals("current path", lastPath, metric.getValue().getCurrentPath()); + assertEquals(0, metric.getValue().getReplicationDelay(), "replication delay"); + assertTrue(metric.getValue().getCurrentPosition() < lastPosition, + "current position should < last position"); + assertNotEquals(lastPath, metric.getValue().getCurrentPath(), "current path"); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java index 769400159a66..1e421489c0cb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyString; @@ -28,7 +28,6 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.Admin; @@ -36,38 +35,34 @@ import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.mockito.invocation.InvocationOnMock; /** * All the modification method will fail once in the test and should finally succeed. */ -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationProcedureRetry { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationProcedureRetry.class); - private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - @BeforeClass + @BeforeAll public static void setUp() throws Exception { UTIL.getConfiguration().setClass(HConstants.MASTER_IMPL, MockHMaster.class, HMaster.class); UTIL.startMiniCluster(3); } - @AfterClass + @AfterAll public static void tearDown() throws Exception { UTIL.shutdownMiniCluster(); } - @After + @AfterEach public void tearDownAfterTest() throws IOException { for (ReplicationPeerDescription desc : UTIL.getAdmin().listReplicationPeers()) { UTIL.getAdmin().removeReplicationPeer(desc.getPeerId()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index 2757a7dc0ae0..9653f949b8a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -17,18 +17,19 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -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.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; +import java.util.stream.Stream; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; @@ -52,44 +53,37 @@ import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKeyImpl; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.params.provider.Arguments; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; - -@RunWith(Parameterized.class) -@Category({ ReplicationTests.class, LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: serialPeer={0}") public class TestReplicationSmallTests extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationSmallTests.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSmallTests.class); private static final String PEER_ID = "2"; - @Parameter - public boolean serialPeer; + private boolean serialPeer; + + public TestReplicationSmallTests(boolean serialPeer) { + this.serialPeer = serialPeer; + } @Override protected boolean isSerialPeer() { return serialPeer; } - @Parameters(name = "{index}: serialPeer={0}") - public static List parameters() { - return ImmutableList.of(true, false); + public static Stream parameters() { + return Stream.of(Arguments.of(true), Arguments.of(false)); } - @Before + @BeforeEach public void setUp() throws Exception { cleanUp(); } @@ -97,7 +91,7 @@ public void setUp() throws Exception { /** * Verify that version and column delete marker types are replicated correctly. */ - @Test + @TestTemplate public void testDeleteTypes() throws Exception { LOG.info("testDeleteTypes"); final byte[] v1 = Bytes.toBytes("v1"); @@ -183,7 +177,7 @@ public void testDeleteTypes() throws Exception { /** * Add a row, check it's replicated, delete it, check's gone */ - @Test + @TestTemplate public void testSimplePutDelete() throws Exception { LOG.info("testSimplePutDelete"); runSimplePutDeleteTest(); @@ -192,7 +186,7 @@ public void testSimplePutDelete() throws Exception { /** * Try a small batch upload using the write buffer, check it's replicated */ - @Test + @TestTemplate public void testSmallBatch() throws Exception { LOG.info("testSmallBatch"); runSmallBatchTest(); @@ -202,7 +196,7 @@ public void testSmallBatch() throws Exception { * Test disable/enable replication, trying to insert, make sure nothing's replicated, enable it, * the insert should be replicated */ - @Test + @TestTemplate public void testDisableEnable() throws Exception { // Test disabling replication hbaseAdmin.disableReplicationPeer(PEER_ID); @@ -242,7 +236,7 @@ public void testDisableEnable() throws Exception { /** * Integration test for TestReplicationAdmin, removes and re-add a peer cluster */ - @Test + @TestTemplate public void testAddAndRemoveClusters() throws Exception { LOG.info("testAddAndRemoveClusters"); hbaseAdmin.removeReplicationPeer(PEER_ID); @@ -295,7 +289,7 @@ public void testAddAndRemoveClusters() throws Exception { * Do a more intense version testSmallBatch, one that will trigger wal rolling and other * non-trivial code paths */ - @Test + @TestTemplate public void testLoading() throws Exception { LOG.info("Writing out rows to table1 in testLoading"); List puts = new ArrayList<>(NB_ROWS_IN_BIG_BATCH); @@ -357,7 +351,7 @@ public void testLoading() throws Exception { * ReplicationAdmin.listReplicated(). Finally verify the table:colfamilies. Note: * TestReplicationAdmin is a better place for this testing but it would need mocks. */ - @Test + @TestTemplate public void testVerifyListReplicatedTable() throws Exception { LOG.info("testVerifyListReplicatedTable"); @@ -390,7 +384,7 @@ public void testVerifyListReplicatedTable() throws Exception { // check the matching result for (int i = 0; i < match.length; i++) { - assertTrue("listReplicated() does not match table " + i, (match[i] == 1)); + assertEquals(1, match[i], "listReplicated() does not match table " + i); } // drop tables @@ -406,7 +400,7 @@ public void testVerifyListReplicatedTable() throws Exception { /** * Test for HBase-15259 WALEdits under replay will also be replicated */ - @Test + @TestTemplate public void testReplicationInReplay() throws Exception { final TableName tableName = htable1.getName(); @@ -447,7 +441,7 @@ public void testReplicationInReplay() throws Exception { /** * Test for HBASE-27448 Add an admin method to get replication enabled state */ - @Test + @TestTemplate public void testGetReplicationPeerState() throws Exception { // Test disable replication peer diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java index c232550d41d3..6a6fe955203d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java @@ -17,15 +17,14 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.EnumSet; import java.util.List; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerMetrics; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Waiter; @@ -37,19 +36,16 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationStatus extends TestReplicationBase { - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStatus.class); - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStatus.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStatus.class); static void insertRowsOnSource() throws IOException { final byte[] qualName = Bytes.toBytes("q"); @@ -91,21 +87,21 @@ public boolean evaluate() throws Exception { for (JVMClusterUtil.RegionServerThread thread : UTIL1.getHBaseCluster() .getRegionServerThreads()) { ServerName server = thread.getRegionServer().getServerName(); - assertTrue("" + server, metrics.getLiveServerMetrics().containsKey(server)); + assertTrue(metrics.getLiveServerMetrics().containsKey(server), "" + server); ServerMetrics sm = metrics.getLiveServerMetrics().get(server); List rLoadSourceList = sm.getReplicationLoadSourceList(); ReplicationLoadSink rLoadSink = sm.getReplicationLoadSink(); // check SourceList only has one entry, because only has one peer - assertEquals("Failed to get ReplicationLoadSourceList " + rLoadSourceList + ", " + server, 1, - rLoadSourceList.size()); + assertEquals(1, rLoadSourceList.size(), + "Failed to get ReplicationLoadSourceList " + rLoadSourceList + ", " + server); assertEquals(PEER_ID2, rLoadSourceList.get(0).getPeerID()); // check Sink exist only as it is difficult to verify the value on the fly - assertTrue("failed to get ReplicationLoadSink.AgeOfLastShippedOp ", - (rLoadSink.getAgeOfLastAppliedOp() >= 0)); - assertTrue("failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp ", - (rLoadSink.getTimestampsOfLastAppliedOp() >= 0)); + assertTrue(rLoadSink.getAgeOfLastAppliedOp() >= 0, + "failed to get ReplicationLoadSink.AgeOfLastShippedOp"); + assertTrue(rLoadSink.getTimestampsOfLastAppliedOp() >= 0, + "failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp"); } // Stop rs1, then the queue of rs1 will be transfered to rs0 @@ -122,7 +118,7 @@ public boolean evaluate() throws Exception { List rLoadSourceList = waitOnMetricsReport(1, server); // The remaining server should now have two queues -- the original and then the one that was // added because of failover. The original should still be PEER_ID2 though. - assertEquals("Failed ReplicationLoadSourceList " + rLoadSourceList, 2, rLoadSourceList.size()); + assertEquals(2, rLoadSourceList.size(), "Failed ReplicationLoadSourceList " + rLoadSourceList); assertEquals(PEER_ID2, rLoadSourceList.get(0).getPeerID()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusAfterLagging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusAfterLagging.java index d9a90b57c2a0..0731fc9a3f6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusAfterLagging.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusAfterLagging.java @@ -17,31 +17,26 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.EnumSet; import java.util.List; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationStatusAfterLagging extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStatusAfterLagging.class); - @Test public void testReplicationStatusAfterLagging() throws Exception { UTIL2.shutdownMiniHBaseCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusBothNormalAndRecoveryLagging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusBothNormalAndRecoveryLagging.java index 161e3c848f78..288dc368c102 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusBothNormalAndRecoveryLagging.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusBothNormalAndRecoveryLagging.java @@ -17,31 +17,26 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.EnumSet; import java.util.List; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationStatusBothNormalAndRecoveryLagging extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStatusBothNormalAndRecoveryLagging.class); - @Test public void testReplicationStatusBothNormalAndRecoveryLagging() throws Exception { UTIL2.shutdownMiniHBaseCluster(); @@ -79,7 +74,7 @@ public void testReplicationStatusBothNormalAndRecoveryLagging() throws Exception assertEquals(0, loadSource.getTimestampOfLastShippedOp()); assertTrue(loadSource.getReplicationLag() > 0); } - assertTrue("No normal queue found.", foundNormal); - assertTrue("No recovery queue found.", foundRecovery); + assertTrue(foundNormal, "No normal queue found."); + assertTrue(foundRecovery, "No recovery queue found."); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSink.java index 34dcf2329c46..c8bdb3eda31e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSink.java @@ -17,28 +17,25 @@ */ package org.apache.hadoop.hbase.replication; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; + import java.io.IOException; import java.util.EnumSet; import org.apache.hadoop.hbase.ClusterMetrics; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerMetrics; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationStatusSink extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStatusSink.class); - @Test public void testReplicationStatusSink() throws Exception { try (Admin admin = UTIL2.getConnection().getAdmin()) { @@ -46,7 +43,7 @@ public void testReplicationStatusSink() throws Exception { ReplicationLoadSink loadSink = getLatestSinkMetric(admin, server); // First checks if status of timestamp of last applied op is same as RS start, since no edits // were replicated yet - Assert.assertEquals(loadSink.getTimestampStarted(), loadSink.getTimestampsOfLastAppliedOp()); + assertEquals(loadSink.getTimestampStarted(), loadSink.getTimestampsOfLastAppliedOp()); // now insert some rows on source, so that it gets delivered to target TestReplicationStatus.insertRowsOnSource(); long wait = @@ -54,7 +51,7 @@ public void testReplicationStatusSink() throws Exception { ReplicationLoadSink loadSink1 = getLatestSinkMetric(admin, server); return loadSink1.getTimestampsOfLastAppliedOp() > loadSink1.getTimestampStarted(); }); - Assert.assertNotEquals(-1, wait); + assertNotEquals(-1, wait); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNewOp.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNewOp.java index 92688cb2575a..4d11e129391d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNewOp.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNewOp.java @@ -17,32 +17,27 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.EnumSet; import java.util.List; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationStatusSourceStartedTargetStoppedNewOp extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStatusSourceStartedTargetStoppedNewOp.class); - @Test public void testReplicationStatusSourceStartedTargetStoppedNewOp() throws Exception { UTIL2.shutdownMiniHBaseCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNoOps.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNoOps.java index 018bfb98c6e5..605c26215343 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNoOps.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNoOps.java @@ -17,29 +17,24 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import java.util.EnumSet; import java.util.List; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationStatusSourceStartedTargetStoppedNoOps extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStatusSourceStartedTargetStoppedNoOps.class); - @Test public void testReplicationStatusSourceStartedTargetStoppedNoOps() throws Exception { UTIL2.shutdownMiniHBaseCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedWithRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedWithRecovery.java index 3b097cff970f..be86277883c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedWithRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedWithRecovery.java @@ -17,33 +17,28 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.EnumSet; import java.util.List; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics.Option; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationStatusSourceStartedTargetStoppedWithRecovery extends TestReplicationBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStatusSourceStartedTargetStoppedWithRecovery.class); - @Test public void testReplicationStatusSourceStartedTargetStoppedWithRecovery() throws Exception { UTIL2.shutdownMiniHBaseCluster(); @@ -77,7 +72,7 @@ public void testReplicationStatusSourceStartedTargetStoppedWithRecovery() throws assertEquals(0, loadSource.getReplicationLag()); } } - assertTrue("No normal queue found.", foundNormal); - assertTrue("No recovery queue found.", foundRecovery); + assertTrue(foundNormal, "No normal queue found."); + assertTrue(foundRecovery, "No recovery queue found."); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStuckWithDeletedTableCFs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStuckWithDeletedTableCFs.java index 05e59c6110fa..dc25f46f6e92 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStuckWithDeletedTableCFs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStuckWithDeletedTableCFs.java @@ -18,12 +18,11 @@ package org.apache.hadoop.hbase.replication; import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_GLOBAL; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; import java.util.Arrays; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -37,13 +36,13 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,13 +50,10 @@ * Replication with dropped table will stuck as the default REPLICATION_DROP_ON_DELETED_TABLE_KEY is * false. */ -@Category({ LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestReplicationStuckWithDeletedTableCFs { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStuckWithDeletedTableCFs.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStuckWithDeletedTableCFs.class); @@ -81,7 +77,7 @@ public class TestReplicationStuckWithDeletedTableCFs { private static final long SLEEP_TIME = 1000; private static final int NB_RETRIES = 10; - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); conf1.setInt("replication.source.nb.capacity", 1); @@ -102,7 +98,7 @@ public static void setUpBeforeClass() throws Exception { admin2 = utility2.getAdmin(); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { utility2.shutdownMiniCluster(); utility1.shutdownMiniCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStuckWithDroppedTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStuckWithDroppedTable.java index b6c1ac417f75..a896b938e9e4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStuckWithDroppedTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStuckWithDroppedTable.java @@ -19,10 +19,9 @@ import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_GLOBAL; import static org.apache.hadoop.hbase.HConstants.ZOOKEEPER_ZNODE_PARENT; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; @@ -35,13 +34,13 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,13 +48,10 @@ * Replication with dropped table will stuck as the default REPLICATION_DROP_ON_DELETED_TABLE_KEY is * false. */ -@Category({ LargeTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestReplicationStuckWithDroppedTable { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationStuckWithDroppedTable.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStuckWithDroppedTable.class); @@ -79,7 +75,7 @@ public class TestReplicationStuckWithDroppedTable { private static final long SLEEP_TIME = 1000; private static final int NB_RETRIES = 10; - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { conf1.set(ZOOKEEPER_ZNODE_PARENT, "/1"); conf1.setInt("replication.source.nb.capacity", 1); @@ -100,7 +96,7 @@ public static void setUpBeforeClass() throws Exception { admin2 = utility2.getAdmin(); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { utility2.shutdownMiniCluster(); utility1.shutdownMiniCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index 762945d745ba..c348194902cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -17,8 +17,9 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -31,7 +32,6 @@ import java.util.TreeMap; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; @@ -45,20 +45,15 @@ import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKeyImpl; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -@Category({ ReplicationTests.class, SmallTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(SmallTests.TAG) public class TestReplicationWALEntryFilters { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationWALEntryFilters.class); - static byte[] a = new byte[] { 'a' }; static byte[] b = new byte[] { 'b' }; static byte[] c = new byte[] { 'c' }; @@ -88,19 +83,19 @@ public void testSystemTableWALEntryFilter() { new WALKeyImpl(new byte[0], TableName.valueOf("foo"), EnvironmentEdgeManager.currentTime()); Entry userEntry = new Entry(key3, null); - assertEquals(userEntry, filter.filter(userEntry)); + assertEntryEquals(userEntry, filter.filter(userEntry)); // hbase:acl should be allowed through the filter WALKeyImpl key4 = new WALKeyImpl(new byte[0], PermissionStorage.ACL_TABLE_NAME, System.currentTimeMillis()); Entry aclEntry = new Entry(key4, null); - assertEquals(aclEntry, filter.filter(aclEntry)); + assertEntryEquals(aclEntry, filter.filter(aclEntry)); // hbase:labels should be allowed through the filter WALKeyImpl key5 = new WALKeyImpl(new byte[0], VisibilityConstants.LABELS_TABLE_NAME, System.currentTimeMillis()); Entry labelsEntry = new Entry(key5, null); - assertEquals(labelsEntry, filter.filter(labelsEntry)); + assertEntryEquals(labelsEntry, filter.filter(labelsEntry)); } @Test @@ -133,39 +128,39 @@ public void testScopeWALEntryFilter() { scopes.put(c, HConstants.REPLICATION_SCOPE_GLOBAL); userEntry = createEntry(scopes, a, b); // all kvs should be filtered - assertEquals(userEntryEmpty, filter.filter(userEntry)); + assertEntryEquals(userEntryEmpty, filter.filter(userEntry)); // local scope scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL); userEntry = createEntry(scopes, a, b); - assertEquals(userEntryEmpty, filter.filter(userEntry)); + assertEntryEquals(userEntryEmpty, filter.filter(userEntry)); scopes.put(b, HConstants.REPLICATION_SCOPE_LOCAL); - assertEquals(userEntryEmpty, filter.filter(userEntry)); + assertEntryEquals(userEntryEmpty, filter.filter(userEntry)); // only scope a scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); scopes.put(a, HConstants.REPLICATION_SCOPE_GLOBAL); userEntry = createEntry(scopes, a, b); - assertEquals(userEntryA, filter.filter(userEntry)); + assertEntryEquals(userEntryA, filter.filter(userEntry)); scopes.put(b, HConstants.REPLICATION_SCOPE_LOCAL); - assertEquals(userEntryA, filter.filter(userEntry)); + assertEntryEquals(userEntryA, filter.filter(userEntry)); // only scope b scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL); userEntry = createEntry(scopes, a, b); - assertEquals(userEntryB, filter.filter(userEntry)); + assertEntryEquals(userEntryB, filter.filter(userEntry)); scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL); - assertEquals(userEntryB, filter.filter(userEntry)); + assertEntryEquals(userEntryB, filter.filter(userEntry)); // scope a and b scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL); userEntry = createEntry(scopes, a, b); - assertEquals(userEntryB, filter.filter(userEntry)); + assertEntryEquals(userEntryB, filter.filter(userEntry)); scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL); - assertEquals(userEntryB, filter.filter(userEntry)); + assertEntryEquals(userEntryB, filter.filter(userEntry)); } WALEntryFilter nullFilter = new WALEntryFilter() { @@ -187,41 +182,41 @@ public void testChainWALEntryFilter() { Entry userEntry = createEntry(null, a, b, c); ChainWALEntryFilter filter = new ChainWALEntryFilter(passFilter); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); filter = new ChainWALEntryFilter(passFilter, passFilter); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); filter = new ChainWALEntryFilter(passFilter, passFilter, passFilter); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); filter = new ChainWALEntryFilter(nullFilter); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); filter = new ChainWALEntryFilter(nullFilter, passFilter); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); filter = new ChainWALEntryFilter(passFilter, nullFilter); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); filter = new ChainWALEntryFilter(nullFilter, passFilter, nullFilter); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); filter = new ChainWALEntryFilter(nullFilter, nullFilter); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); // flatten filter = new ChainWALEntryFilter( new ChainWALEntryFilter(passFilter, new ChainWALEntryFilter(passFilter, passFilter), new ChainWALEntryFilter(passFilter), new ChainWALEntryFilter(passFilter)), new ChainWALEntryFilter(passFilter)); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); filter = new ChainWALEntryFilter( new ChainWALEntryFilter(passFilter, new ChainWALEntryFilter(passFilter, new ChainWALEntryFilter(nullFilter))), new ChainWALEntryFilter(passFilter)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); } @Test @@ -234,7 +229,7 @@ public void testNamespaceTableCfWALEntryFilter() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); Entry userEntry = createEntry(null, a, b, c); ChainWALEntryFilter filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); // 2. replicate_all flag is false, and only config table-cfs in peer // empty map @@ -243,7 +238,7 @@ public void testNamespaceTableCfWALEntryFilter() { peerConfigBuilder.setReplicateAllUserTables(false).setTableCFsMap(tableCfs); when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); // table bar userEntry = createEntry(null, a, b, c); @@ -252,7 +247,7 @@ public void testNamespaceTableCfWALEntryFilter() { peerConfigBuilder.setReplicateAllUserTables(false).setTableCFsMap(tableCfs); when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); // table foo:a userEntry = createEntry(null, a, b, c); @@ -261,7 +256,7 @@ public void testNamespaceTableCfWALEntryFilter() { peerConfigBuilder.setReplicateAllUserTables(false).setTableCFsMap(tableCfs); when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a), filter.filter(userEntry)); // table foo:a,c userEntry = createEntry(null, a, b, c, d); @@ -270,7 +265,7 @@ public void testNamespaceTableCfWALEntryFilter() { peerConfigBuilder.setReplicateAllUserTables(false).setTableCFsMap(tableCfs); when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, c), filter.filter(userEntry)); // 3. replicate_all flag is false, and only config namespaces in peer when(peer.getTableCFs()).thenReturn(null); @@ -281,7 +276,7 @@ public void testNamespaceTableCfWALEntryFilter() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); // namespace default namespaces.add("default"); @@ -289,7 +284,7 @@ public void testNamespaceTableCfWALEntryFilter() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); // namespace ns1 namespaces = new HashSet<>(); @@ -298,7 +293,7 @@ public void testNamespaceTableCfWALEntryFilter() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); // 4. replicate_all flag is false, and config namespaces and table-cfs both // Namespaces config should not confict with table-cfs config @@ -311,7 +306,7 @@ public void testNamespaceTableCfWALEntryFilter() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, c), filter.filter(userEntry)); namespaces = new HashSet<>(); tableCfs = new HashMap<>(); @@ -322,7 +317,7 @@ public void testNamespaceTableCfWALEntryFilter() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); namespaces = new HashSet<>(); tableCfs = new HashMap<>(); @@ -333,7 +328,7 @@ public void testNamespaceTableCfWALEntryFilter() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); } @Test @@ -348,7 +343,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); Entry userEntry = createEntry(null, a, b, c); ChainWALEntryFilter filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); // 2. replicate_all flag is true, and only config exclude namespaces // empty set @@ -357,7 +352,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); // exclude namespace default namespaces.add("default"); @@ -365,7 +360,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); // exclude namespace ns1 namespaces = new HashSet(); @@ -374,7 +369,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); // 3. replicate_all flag is true, and only config exclude table-cfs // empty table-cfs map @@ -383,7 +378,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); // exclude table bar tableCfs = new HashMap>(); @@ -392,7 +387,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, a, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, a, b, c), filter.filter(userEntry)); // exclude table foo:a tableCfs = new HashMap>(); @@ -401,7 +396,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, b, c), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, b, c), filter.filter(userEntry)); // 4. replicate_all flag is true, and config exclude namespaces and table-cfs both // exclude ns1 and table foo:a,c @@ -413,7 +408,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(createEntry(null, b), filter.filter(userEntry)); + assertEntryEquals(createEntry(null, b), filter.filter(userEntry)); // exclude namespace default and table ns1:bar namespaces = new HashSet(); @@ -424,7 +419,7 @@ public void testNamespaceTableCfWALEntryFilter2() { when(peer.getPeerConfig()).thenReturn(peerConfigBuilder.build()); userEntry = createEntry(null, a, b, c); filter = new ChainWALEntryFilter(new NamespaceTableCfWALEntryFilter(peer)); - assertEquals(null, filter.filter(userEntry)); + assertEntryEquals(null, filter.filter(userEntry)); } private Entry createEntry(TreeMap scopes, byte[]... kvs) { @@ -438,8 +433,8 @@ private Entry createEntry(TreeMap scopes, byte[]... kvs) { return new Entry(key1, edit1); } - private void assertEquals(Entry e1, Entry e2) { - Assert.assertEquals(e1 == null, e2 == null); + private void assertEntryEquals(Entry e1, Entry e2) { + assertEquals(e1 == null, e2 == null); if (e1 == null) { return; } @@ -447,13 +442,13 @@ private void assertEquals(Entry e1, Entry e2) { // do not compare WALKeys // compare kvs - Assert.assertEquals(e1.getEdit() == null, e2.getEdit() == null); + assertEquals(e1.getEdit() == null, e2.getEdit() == null); if (e1.getEdit() == null) { return; } List cells1 = e1.getEdit().getCells(); List cells2 = e2.getEdit().getCells(); - Assert.assertEquals(cells1.size(), cells2.size()); + assertEquals(cells1.size(), cells2.size()); for (int i = 0; i < cells1.size(); i++) { CellComparatorImpl.COMPARATOR.compare(cells1.get(i), cells2.get(i)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithFSPeerStorage.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithFSPeerStorage.java index 6f5c6c20d8d8..d0aa8e21c953 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithFSPeerStorage.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithFSPeerStorage.java @@ -17,33 +17,29 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, LargeTests.class }) -public class TestReplicationWithFSPeerStorage extends TestReplicationBase { +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) +public class TestReplicationWithFSPeerStorage extends TestReplicationBaseNoBeforeAll { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationWithFSPeerStorage.class); - - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { // enable file system based peer storage UTIL1.getConfiguration().set(ReplicationStorageFactory.REPLICATION_PEER_STORAGE_IMPL, ReplicationPeerStorageType.FILESYSTEM.name().toLowerCase()); UTIL2.getConfiguration().set(ReplicationStorageFactory.REPLICATION_PEER_STORAGE_IMPL, ReplicationPeerStorageType.FILESYSTEM.name().toLowerCase()); - TestReplicationBase.setUpBeforeClass(); + configureClusters(UTIL1, UTIL2); + startClusters(); } - @Before + @BeforeEach public void setUp() throws Exception { cleanUp(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java index f50c301a7c62..49c5dee0b508 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.ArrayList; @@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -60,21 +59,16 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ ReplicationTests.class, MediumTests.class }) +@org.junit.jupiter.api.Tag(ReplicationTests.TAG) +@org.junit.jupiter.api.Tag(MediumTests.TAG) public class TestReplicationWithTags { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationWithTags.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationWithTags.class); private static final byte TAG_TYPE = 1; @@ -98,7 +92,7 @@ public class TestReplicationWithTags { private static final byte[] FAMILY = Bytes.toBytes("f"); private static final byte[] ROW = Bytes.toBytes("row"); - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { conf1.setInt("hfile.format.version", 3); conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); @@ -162,7 +156,7 @@ public static void setUpBeforeClass() throws Exception { htable2 = utility2.getConnection().getTable(TABLE_NAME); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { utility2.shutdownMiniCluster(); utility1.shutdownMiniCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithWALExtendedAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithWALExtendedAttributes.java index 2f8fba919f42..baf0f86a00fc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithWALExtendedAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithWALExtendedAttributes.java @@ -17,15 +17,14 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Optional; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -57,11 +56,10 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,13 +68,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestReplicationWithWALExtendedAttributes { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReplicationWithWALExtendedAttributes.class); - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationWithWALExtendedAttributes.class); @@ -99,7 +94,7 @@ public class TestReplicationWithWALExtendedAttributes { private static final byte[] ROW = Bytes.toBytes("row"); private static final byte[] ROW2 = Bytes.toBytes("row2"); - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); conf1.setInt("replication.source.size.capacity", 10240); @@ -164,7 +159,7 @@ public static void setUpBeforeClass() throws Exception { htable2 = utility2.getConnection().getTable(TABLE_NAME); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { Closeables.close(replicationAdmin, true); Closeables.close(connection1, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java index 7ab82b60cec9..1e0f7fe54b7d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.HashMap; @@ -28,7 +28,6 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -44,19 +43,15 @@ import org.apache.hadoop.hbase.wal.NoEOFWALStreamReader; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALStreamReader; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestSerialReplication extends SerialReplicationTestBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestSerialReplication.class); - - @Before + @BeforeEach public void setUp() throws IOException, StreamLacksCapabilityException { setupWALWriter(); // add in disable state, so later when enabling it all sources will start push together. @@ -117,14 +112,14 @@ public void testRegionSplit() throws Exception { } String encodedName = Bytes.toString(entry.getKey().getEncodedRegionName()); Long seqId = regionsToSeqId.get(encodedName); - assertNotNull( - "Unexcepted entry " + entry + ", expected regions " + region + ", or " + regions, seqId); - assertTrue("Sequence id go backwards from " + seqId + " to " - + entry.getKey().getSequenceId() + " for " + encodedName, - entry.getKey().getSequenceId() >= seqId.longValue()); + assertNotNull(seqId, + "Unexcepted entry " + entry + ", expected regions " + region + ", or " + regions); + assertTrue(entry.getKey().getSequenceId() >= seqId.longValue(), + "Sequence id go backwards from " + seqId + " to " + entry.getKey().getSequenceId() + + " for " + encodedName); if (count < 100) { - assertEquals(encodedName + " is pushed before parent " + region.getEncodedName(), - region.getEncodedName(), encodedName); + assertEquals(region.getEncodedName(), encodedName, + encodedName + " is pushed before parent " + region.getEncodedName()); } else { assertNotEquals(region.getEncodedName(), encodedName); } @@ -137,7 +132,7 @@ public void testRegionSplit() throws Exception { @Test public void testRegionMerge() throws Exception { byte[] splitKey = Bytes.toBytes(50); - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = tableNameExt.getTableName(); UTIL.getAdmin().createTable( TableDescriptorBuilder.newBuilder(tableName) .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF) @@ -178,11 +173,11 @@ public void testRegionMerge() throws Exception { } String encodedName = Bytes.toString(entry.getKey().getEncodedRegionName()); Long seqId = regionsToSeqId.get(encodedName); - assertNotNull( - "Unexcepted entry " + entry + ", expected regions " + region + ", or " + regions, seqId); - assertTrue("Sequence id go backwards from " + seqId + " to " - + entry.getKey().getSequenceId() + " for " + encodedName, - entry.getKey().getSequenceId() >= seqId.longValue()); + assertNotNull(seqId, + "Unexcepted entry " + entry + ", expected regions " + region + ", or " + regions); + assertTrue(entry.getKey().getSequenceId() >= seqId.longValue(), + "Sequence id go backwards from " + seqId + " to " + entry.getKey().getSequenceId() + + " for " + encodedName); if (count < 100) { assertNotEquals( encodedName + " is pushed before parents " + regions.stream() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java index 1295ea14abcd..d4700052cac0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java @@ -18,31 +18,26 @@ package org.apache.hadoop.hbase.replication; import java.io.IOException; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; 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.Table; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(LargeTests.TAG) public class TestSerialReplicationFailover extends SerialReplicationTestBase { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestSerialReplicationFailover.class); - - @Before + @BeforeEach public void setUp() throws IOException, StreamLacksCapabilityException { setupWALWriter(); // add in disable state, so later when enabling it all sources will start push together. @@ -51,7 +46,7 @@ public void setUp() throws IOException, StreamLacksCapabilityException { @Test public void testKillRS() throws Exception { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = tableNameExt.getTableName(); UTIL.getAdmin().createTable( TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java index a39532c97b5b..b127e2b01882 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -25,7 +25,6 @@ import java.util.concurrent.TimeoutException; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; @@ -35,24 +34,20 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Confirm that the empty replication endpoint can work. */ -@Category({ ReplicationTests.class, MediumTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(MediumTests.TAG) public class TestVerifyCellsReplicationEndpoint { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestVerifyCellsReplicationEndpoint.class); - private static final Logger LOG = LoggerFactory.getLogger(TestVerifyCellsReplicationEndpoint.class); @@ -79,7 +74,7 @@ public boolean replicate(ReplicateContext replicateContext) { } } - @BeforeClass + @BeforeAll public static void setUp() throws Exception { UTIL.startMiniCluster(3); // notice that we do not need to set replication scope here, EmptyReplicationEndpoint take all @@ -90,7 +85,7 @@ public static void setUp() throws Exception { .setReplicationEndpointImpl(EndpointForTest.class.getName()).build()); } - @AfterClass + @AfterAll public static void tearDown() throws Exception { UTIL.shutdownMiniCluster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java index f016455e92ae..bab250ead6a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java @@ -17,18 +17,18 @@ */ package org.apache.hadoop.hbase.replication.master; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestExtension; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage; @@ -37,23 +37,18 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Category({ ReplicationTests.class, SmallTests.class }) +@Tag(ReplicationTests.TAG) +@Tag(SmallTests.TAG) public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestTableCFsUpdater.class); - private static final Logger LOG = LoggerFactory.getLogger(TestTableCFsUpdater.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -61,20 +56,20 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { private static Abortable abortable = null; private static ZKStorageUtil zkStorageUtil = null; + @RegisterExtension + private final TableNameTestExtension tableNameExt = new TableNameTestExtension(); + private static class ZKStorageUtil extends ZKReplicationPeerStorage { public ZKStorageUtil(ZKWatcher zookeeper, Configuration conf) { super(zookeeper, conf); } } - @Rule - public TestName name = new TestName(); - public TestTableCFsUpdater() throws IOException { super(zkw, TEST_UTIL.getConfiguration()); } - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniZKCluster(); Configuration conf = TEST_UTIL.getConfiguration(); @@ -93,7 +88,7 @@ public boolean isAborted() { zkStorageUtil = new ZKStorageUtil(zkw, conf); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniZKCluster(); } @@ -101,9 +96,9 @@ public static void tearDownAfterClass() throws Exception { @Test public void testUpgrade() throws Exception { String peerId = "1"; - final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1"); - final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2"); - final TableName tableName3 = TableName.valueOf(name.getMethodName() + "3"); + final TableName tableName1 = tableNameExt.getTableName("1"); + final TableName tableName2 = tableNameExt.getTableName("2"); + final TableName tableName3 = tableNameExt.getTableName("3"); ReplicationPeerConfig rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java index 18286bca91c1..d66d4cf8c46d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hbase.util; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import java.io.IOException; import java.util.ArrayList; @@ -115,8 +115,8 @@ public static void assertArchiveEqualToOriginal(FileStatus[] expected, FileStatu // check the backed up files versus the current (should match up, less the // backup time in the name) - assertEquals("Didn't expect any backup files, but got: " + backedup, hasTimedBackup, - backedup.size() > 0); + assertEquals(hasTimedBackup, backedup.size() > 0, + "Didn't expect any backup files, but got: " + backedup); String msg = null; if (hasTimedBackup) { assertArchiveEquality(original, backedup); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java index 9d4257e1aa7d..8d8924006d36 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.util; import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.Arrays; @@ -150,7 +150,7 @@ public static void verifyTags(Table table) throws IOException { } Tag t = tag.get(); byte[] tval = Tag.cloneValue(t); - assertArrayEquals(c.toString() + " has tag" + Bytes.toString(tval), r.getRow(), tval); + assertArrayEquals(r.getRow(), tval, c.toString() + " has tag" + Bytes.toString(tval)); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestServerHttpUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestServerHttpUtils.java index 9067f92dab86..179668bf986e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestServerHttpUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestServerHttpUtils.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.util; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.BufferedReader; import java.io.IOException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java index 41e98464bfde..7923b0e865fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.util.hbck; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.IOException; import java.util.ArrayList; 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 2f65e82aabb8..97c74c03f52e 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 @@ -22,17 +22,17 @@ 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 static org.junit.jupiter.api.Assertions.assertEquals; import java.io.IOException; -import java.util.Arrays; import java.util.HashSet; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -40,28 +40,21 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.provider.Arguments; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@RunWith(Parameterized.class) -@Category({ RegionServerTests.class, MediumTests.class }) +@Tag(RegionServerTests.TAG) +@Tag(MediumTests.TAG) +@HBaseParameterizedTestTemplate(name = "{index}: delegate-provider={0}") public class TestBoundedRegionGroupingStrategy { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestBoundedRegionGroupingStrategy.class); - private static final Logger LOG = LoggerFactory.getLogger(TestBoundedRegionGroupingStrategy.class); @@ -70,20 +63,22 @@ public class TestBoundedRegionGroupingStrategy { private static Configuration CONF; private static DistributedFileSystem FS; - @Parameter public String walProvider; - @Parameters(name = "{index}: delegate-provider={0}") - public static Iterable data() { - return Arrays.asList(new Object[] { "defaultProvider" }, new Object[] { "asyncfs" }); + public TestBoundedRegionGroupingStrategy(String walProvider) { + this.walProvider = walProvider; + } + + public static Stream parameters() { + return Stream.of(Arguments.of("defaultProvider"), Arguments.of("asyncfs")); } - @Before + @BeforeEach public void setUp() throws Exception { CONF.set(DELEGATE_PROVIDER, walProvider); } - @After + @AfterEach public void tearDown() throws Exception { FileStatus[] entries = FS.listStatus(new Path("/")); for (FileStatus dir : entries) { @@ -91,7 +86,7 @@ public void tearDown() throws Exception { } } - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { CONF = TEST_UTIL.getConfiguration(); // Make block sizes small. @@ -114,7 +109,7 @@ public static void setUpBeforeClass() throws Exception { FS = TEST_UTIL.getDFSCluster().getFileSystem(); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); } @@ -197,8 +192,9 @@ public void setMembershipDedups() throws IOException { count++; } } - assertEquals("received back a different number of WALs that are not equal() to each other " - + "than the bound we placed.", temp * 4, count); + assertEquals(temp * 4, count, + "received back a different number of WALs that are not equal() to each other " + + "than the bound we placed."); } finally { if (wals != null) { wals.close(); From 33417597b459666fe160cd7947ffd494292e220d Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 5 Jun 2026 19:21:54 +0800 Subject: [PATCH 2/2] fix --- .../hbase/wal/TestBoundedRegionGroupingStrategy.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 97c74c03f52e..a16506718a77 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 @@ -45,7 +45,7 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.params.provider.Arguments; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -117,7 +117,7 @@ public static void tearDownAfterClass() throws Exception { /** * Write to a log file with three concurrent threads and verifying all data is written. */ - @Test + @TestTemplate public void testConcurrentWrites() throws Exception { // Run the WPE tool with three threads writing 3000 edits each concurrently. // When done, verify that all edits were written. @@ -129,7 +129,7 @@ public void testConcurrentWrites() throws Exception { /** * Make sure we can successfully run with more regions then our bound. */ - @Test + @TestTemplate public void testMoreRegionsThanBound() throws Exception { final String parallelism = Integer.toString(DEFAULT_NUM_REGION_GROUPS * 2); int errCode = @@ -138,7 +138,7 @@ public void testMoreRegionsThanBound() throws Exception { assertEquals(0, errCode); } - @Test + @TestTemplate public void testBoundsGreaterThanDefault() throws Exception { final int temp = CONF.getInt(NUM_REGION_GROUPS, DEFAULT_NUM_REGION_GROUPS); try { @@ -153,7 +153,7 @@ public void testBoundsGreaterThanDefault() throws Exception { } } - @Test + @TestTemplate public void testMoreRegionsThanBoundWithBoundsGreaterThanDefault() throws Exception { final int temp = CONF.getInt(NUM_REGION_GROUPS, DEFAULT_NUM_REGION_GROUPS); try { @@ -171,7 +171,7 @@ public void testMoreRegionsThanBoundWithBoundsGreaterThanDefault() throws Except /** * Ensure that we can use Set.add to deduplicate WALs */ - @Test + @TestTemplate public void setMembershipDedups() throws IOException { final int temp = CONF.getInt(NUM_REGION_GROUPS, DEFAULT_NUM_REGION_GROUPS); WALFactory wals = null;