From 2bdf310df9b0ba472f967a8d1fbe69e240f5d5e9 Mon Sep 17 00:00:00 2001 From: fanshilun Date: Thu, 6 Feb 2025 08:04:45 +0800 Subject: [PATCH 1/7] HADOOP-19415. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-common Part4. --- .../hadoop/net/TestClusterTopology.java | 66 ++--- .../java/org/apache/hadoop/net/TestDNS.java | 19 +- .../hadoop/net/TestDNSDomainNameResolver.java | 6 +- .../net/TestMockDomainNameResolver.java | 10 +- .../org/apache/hadoop/net/TestNetUtils.java | 36 +-- .../net/TestNetworkTopologyWithNodeGroup.java | 18 +- .../hadoop/net/TestScriptBasedMapping.java | 18 +- .../TestScriptBasedMappingWithDependency.java | 20 +- .../hadoop/net/TestSocketIOWithTimeout.java | 4 +- .../apache/hadoop/net/TestStaticMapping.java | 34 +-- .../apache/hadoop/net/TestSwitchMapping.java | 34 +-- .../apache/hadoop/net/TestTableMapping.java | 8 +- .../hadoop/net/unix/TestDomainSocket.java | 108 ++++---- .../net/unix/TestDomainSocketWatcher.java | 31 ++- .../hadoop/oncrpc/TestFrameDecoder.java | 8 +- .../hadoop/oncrpc/TestRpcAcceptedReply.java | 10 +- .../org/apache/hadoop/oncrpc/TestRpcCall.java | 21 +- .../hadoop/oncrpc/TestRpcCallCache.java | 23 +- .../hadoop/oncrpc/TestRpcDeniedReply.java | 23 +- .../apache/hadoop/oncrpc/TestRpcMessage.java | 18 +- .../apache/hadoop/oncrpc/TestRpcReply.java | 20 +- .../org/apache/hadoop/oncrpc/TestXDR.java | 8 +- .../oncrpc/security/TestCredentialsSys.java | 4 +- .../oncrpc/security/TestRpcAuthInfo.java | 10 +- .../apache/hadoop/portmap/TestPortmap.java | 29 +- .../security/ManualTestKeytabLogins.java | 2 +- .../security/TestAuthenticationFilter.java | 4 +- .../security/TestCompositeGroupMapping.java | 4 +- .../hadoop/security/TestCredentials.java | 92 +++---- .../security/TestDoAsEffectiveUser.java | 37 +-- .../security/TestFixKerberosTicketOrder.java | 44 +-- .../hadoop/security/TestGroupFallback.java | 4 +- .../hadoop/security/TestGroupsCaching.java | 32 +-- .../TestHttpCrossOriginFilterInitializer.java | 12 +- .../TestIngressPortBasedResolver.java | 4 +- .../hadoop/security/TestJNIGroupsMapping.java | 8 +- .../org/apache/hadoop/security/TestKDiag.java | 22 +- .../hadoop/security/TestKDiagNoKDC.java | 14 +- .../security/TestLdapGroupsMapping.java | 46 ++-- .../security/TestLdapGroupsMappingBase.java | 6 +- ...stLdapGroupsMappingWithBindUserSwitch.java | 10 +- .../TestLdapGroupsMappingWithFailover.java | 4 +- .../TestLdapGroupsMappingWithOneQuery.java | 16 +- .../TestLdapGroupsMappingWithPosixGroup.java | 12 +- .../hadoop/security/TestNetgroupCache.java | 10 +- .../security/TestNullGroupsMapping.java | 20 +- .../hadoop/security/TestProxyUserFromEnv.java | 6 +- .../hadoop/security/TestRaceWhenRelogin.java | 10 +- .../TestRuleBasedLdapGroupsMapping.java | 10 +- .../hadoop/security/TestSecurityUtil.java | 14 +- .../security/TestShellBasedIdMapping.java | 6 +- .../TestShellBasedUnixGroupsMapping.java | 68 ++--- .../security/TestUGILoginFromKeytab.java | 152 +++++------ .../security/TestUGIWithExternalKdc.java | 16 +- .../hadoop/security/TestUGIWithMiniKdc.java | 10 +- .../hadoop/security/TestUserFromEnv.java | 6 +- .../security/TestUserGroupInformation.java | 197 ++++++++------ .../security/TestWhitelistBasedResolver.java | 4 +- .../hadoop/security/alias/TestCredShell.java | 40 +-- .../alias/TestCredentialProvider.java | 6 +- .../alias/TestCredentialProviderFactory.java | 32 +-- .../TestProxyUserAuthenticationFilter.java | 6 +- .../authorize/TestAccessControlList.java | 24 +- .../TestDefaultImpersonationProvider.java | 10 +- .../security/authorize/TestProxyServers.java | 6 +- .../security/authorize/TestProxyUsers.java | 82 +++--- .../authorize/TestServiceAuthorization.java | 6 +- .../security/http/TestCrossOriginFilter.java | 106 ++++---- .../http/TestRestCsrfPreventionFilter.java | 2 +- .../http/TestXFrameOptionsFilter.java | 31 ++- .../ssl/TestDelegatingSSLSocketFactory.java | 2 +- .../ssl/TestReloadingX509KeyManager.java | 48 ++-- .../ssl/TestReloadingX509TrustManager.java | 46 ++-- .../hadoop/security/ssl/TestSSLFactory.java | 104 ++++---- .../security/token/TestDtUtilShell.java | 178 ++++++------- .../hadoop/security/token/TestToken.java | 4 +- .../token/delegation/TestDelegationToken.java | 50 ++-- .../TestZKDelegationTokenSecretManager.java | 78 +++--- ...onTokenAuthenticationHandlerWithMocks.java | 88 +++--- .../web/TestDelegationTokenManager.java | 12 +- .../web/TestWebDelegationToken.java | 162 +++++------ .../apache/hadoop/service/ServiceAssert.java | 16 +- .../hadoop/service/TestCompositeService.java | 252 ++++++++++-------- .../TestGlobalStateChangeListener.java | 18 +- .../hadoop/service/TestServiceLifecycle.java | 10 +- .../hadoop/service/TestServiceOperations.java | 2 +- .../AbstractServiceLauncherTestBase.java | 18 +- .../service/launcher/TestServiceConf.java | 10 +- .../TestServiceInterruptHandling.java | 14 +- .../service/launcher/TestServiceLauncher.java | 2 +- .../TestServiceLauncherCreationFailures.java | 2 +- .../TestServiceLauncherInnerMethods.java | 6 +- .../InitInConstructorLaunchableService.java | 14 +- .../LaunchableRunningService.java | 4 +- .../hadoop/test/TestGenericTestUtils.java | 15 +- .../apache/hadoop/test/TestJUnitSetup.java | 6 +- .../hadoop/test/TestLambdaTestUtils.java | 18 +- .../test/TestMultithreadedTestUtil.java | 24 +- .../test/TestTimedOutTestsListener.java | 14 +- .../apache/hadoop/tools/TestCommandShell.java | 28 +- 100 files changed, 1614 insertions(+), 1470 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java index 57b620fde6c0e..d5dd4173dab01 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java @@ -26,10 +26,10 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.math3.stat.inference.ChiSquareTest; import org.apache.hadoop.conf.Configuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; -public class TestClusterTopology extends Assert { +public class TestClusterTopology extends Assertions { public static class NodeElement implements Node { private String location; @@ -96,34 +96,34 @@ public void testCountNumNodes() throws Exception { // create exclude list List excludedNodes = new ArrayList(); - assertEquals("4 nodes should be available", 4, - cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes)); + assertEquals(4 +, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "4 nodes should be available"); NodeElement deadNode = getNewNode("node5", "/d1/r2"); excludedNodes.add(deadNode); - assertEquals("4 nodes should be available with extra excluded Node", 4, - cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes)); + assertEquals(4 +, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "4 nodes should be available with extra excluded Node"); // add one existing node to exclude list excludedNodes.add(node4); - assertEquals("excluded nodes with ROOT scope should be considered", 3, - cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes)); - assertEquals("excluded nodes without ~ scope should be considered", 2, - cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), - excludedNodes)); - assertEquals("excluded nodes with rack scope should be considered", 1, - cluster.countNumOfAvailableNodes(deadNode.getNetworkLocation(), - excludedNodes)); + assertEquals(3 +, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "excluded nodes with ROOT scope should be considered"); + assertEquals(2 +, cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), + excludedNodes), "excluded nodes without ~ scope should be considered"); + assertEquals(1 +, cluster.countNumOfAvailableNodes(deadNode.getNetworkLocation(), + excludedNodes), "excluded nodes with rack scope should be considered"); // adding the node in excluded scope to excluded list excludedNodes.add(node2); - assertEquals("excluded nodes with ~ scope should be considered", 2, - cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), - excludedNodes)); + assertEquals(2 +, cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), + excludedNodes), "excluded nodes with ~ scope should be considered"); // getting count with non-exist scope. - assertEquals("No nodes should be considered for non-exist scope", 0, - cluster.countNumOfAvailableNodes("/non-exist", excludedNodes)); + assertEquals(0 +, cluster.countNumOfAvailableNodes("/non-exist", excludedNodes), "No nodes should be considered for non-exist scope"); // remove a node from the cluster cluster.remove(node1); - assertEquals("1 node should be available", 1, - cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes)); + assertEquals(1 +, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "1 node should be available"); } /** @@ -160,7 +160,7 @@ public void testChooseRandom() { } histogram.put(randomNode, histogram.get(randomNode) + 1); } - assertEquals("Random is not selecting all nodes", 4, histogram.size()); + assertEquals(4, histogram.size(), "Random is not selecting all nodes"); // Check with 99% confidence alpha=0.01 as confidence = 100 * (1 - alpha) ChiSquareTest chiSquareTest = new ChiSquareTest(); @@ -181,8 +181,8 @@ public void testChooseRandom() { } // Check that they have the proper distribution - assertFalse("Random not choosing nodes with proper distribution", - chiSquareTestRejectedCounter==3); + assertFalse( + chiSquareTestRejectedCounter==3, "Random not choosing nodes with proper distribution"); // Pick random nodes excluding the 2 nodes in /d1/r3 HashMap histogram = new HashMap(); @@ -193,8 +193,8 @@ public void testChooseRandom() { } histogram.put(randomNode, histogram.get(randomNode) + 1); } - assertEquals("Random is not selecting the nodes it should", - 2, histogram.size()); + assertEquals( + 2, histogram.size(), "Random is not selecting the nodes it should"); Node val = cluster.chooseRandom("/d1", "/d", Collections.emptyList()); assertNotNull(val); @@ -268,9 +268,9 @@ public void testWeights() { for (Pair test: new Pair[]{Pair.of(0, node1), Pair.of(2, node2), Pair.of(4, node3)}) { int expect = test.getLeft(); - assertEquals(test.toString(), expect, cluster.getWeight(node1, test.getRight())); - assertEquals(test.toString(), expect, - cluster.getWeightUsingNetworkLocation(node1, test.getRight())); + assertEquals(expect, cluster.getWeight(node1, test.getRight()), test.toString()); + assertEquals(expect +, cluster.getWeightUsingNetworkLocation(node1, test.getRight()), test.toString()); } // Reset so that we can have 2 levels cluster = NetworkTopology.getInstance(new Configuration()); @@ -281,9 +281,9 @@ public void testWeights() { for (Pair test: new Pair[]{Pair.of(0, node5), Pair.of(2, node6), Pair.of(4, node7), Pair.of(6, node8)}) { int expect = test.getLeft(); - assertEquals(test.toString(), expect, cluster.getWeight(node5, test.getRight())); - assertEquals(test.toString(), expect, - cluster.getWeightUsingNetworkLocation(node5, test.getRight())); + assertEquals(expect, cluster.getWeight(node5, test.getRight()), test.toString()); + assertEquals(expect +, cluster.getWeightUsingNetworkLocation(node5, test.getRight()), test.toString()); } } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java index d33545ab6fe0d..8a131ac2c4f08 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java @@ -31,12 +31,13 @@ import org.assertj.core.api.Assertions; import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; /** * Test host name and IP resolution and caching. @@ -78,8 +79,8 @@ public void testGetLocalHostIsFast() throws Exception { assertEquals(hostname2, hostname1); long interval = t2 - t1; assertTrue( - "Took too long to determine local host - caching is not working", - interval < 20000); + + interval < 20000, "Took too long to determine local host - caching is not working"); } /** @@ -154,7 +155,7 @@ public void testIPsOfUnknownInterface() throws Exception { @Test public void testGetIPWithDefault() throws Exception { String[] ips = DNS.getIPs(DEFAULT); - assertEquals("Should only return 1 default IP", 1, ips.length); + assertEquals(1, ips.length, "Should only return 1 default IP"); assertEquals(getLocalIPAddr().getHostAddress(), ips[0].toString()); String ip = DNS.getDefaultIP(DEFAULT); assertEquals(ip, ips[0].toString()); @@ -196,7 +197,8 @@ public void testRDNS() throws Exception { * * @throws Exception */ - @Test (timeout=60000) + @Test + @Timeout(value = 60) public void testLookupWithHostsFallback() throws Exception { assumeNotWindows(); final String oldHostname = DNS.getCachedHostname(); @@ -219,7 +221,8 @@ public void testLookupWithHostsFallback() throws Exception { * * @throws Exception */ - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testLookupWithoutHostsFallback() throws Exception { final String oldHostname = DNS.getCachedHostname(); try { @@ -249,7 +252,7 @@ private String getLoopbackInterface() throws SocketException { @Test public void testLocalhostResolves() throws Exception { InetAddress localhost = InetAddress.getByName("localhost"); - assertNotNull("localhost is null", localhost); + assertNotNull(localhost, "localhost is null"); LOG.info("Localhost IPAddr is " + localhost.toString()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java index 4729cee118818..b854524e498ea 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java @@ -17,14 +17,14 @@ */ package org.apache.hadoop.net; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Objects; -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 static org.junit.Assume.assumeFalse; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestMockDomainNameResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestMockDomainNameResolver.java index 21c6c7279fb81..70c968b784bde 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestMockDomainNameResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestMockDomainNameResolver.java @@ -19,15 +19,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; /** * This class mainly test the MockDomainNameResolver comes working as expected. @@ -36,7 +36,7 @@ public class TestMockDomainNameResolver { private Configuration conf; - @Before + @BeforeEach public void setup() { conf = new Configuration(); conf.set(CommonConfigurationKeys.HADOOP_DOMAINNAME_RESOLVER_IMPL, diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java index 4b18d74d9b73e..ff3218504fa2f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.net; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import java.io.EOFException; import java.io.IOException; @@ -48,9 +48,9 @@ import org.apache.hadoop.util.Shell; import org.junit.Assume; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -193,8 +193,8 @@ private void assertTimeSince(long startNanos, int expectedMillis) { long durationNano = System.nanoTime() - startNanos; long millis = TimeUnit.MILLISECONDS.convert( durationNano, TimeUnit.NANOSECONDS); - assertTrue("Expected " + expectedMillis + "ms, but took " + millis, - Math.abs(millis - expectedMillis) < TIME_FUDGE_MILLIS); + assertTrue( + Math.abs(millis - expectedMillis) < TIME_FUDGE_MILLIS, "Expected " + expectedMillis + "ms, but took " + millis); } /** @@ -209,10 +209,12 @@ public void testGetLocalInetAddress() throws Exception { assertNull(NetUtils.getLocalInetAddress(null)); } - @Test(expected=UnknownHostException.class) + @Test public void testVerifyHostnamesException() throws UnknownHostException { - String[] names = {"valid.host.com", "1.com", "invalid host here"}; - NetUtils.verifyHostnames(names); + assertThrows(UnknownHostException.class, ()->{ + String[] names = {"valid.host.com", "1.com", "invalid host here"}; + NetUtils.verifyHostnames(names); + }); } @Test @@ -440,7 +442,7 @@ private void assertInException(Exception e, String text) throws Throwable { } private String extractExceptionMessage(Exception e) throws Throwable { - assertNotNull("Null Exception", e); + assertNotNull(e, "Null Exception"); String message = e.getMessage(); if (message == null) { throw new AssertionError("Empty text in exception " + e) @@ -463,7 +465,7 @@ private void assertNotInException(Exception e, String text) private IOException verifyExceptionClass(IOException e, Class expectedClass) throws Throwable { - assertNotNull("Null Exception", e); + assertNotNull(e, "Null Exception"); IOException wrapped = NetUtils.wrapException("desthost", DEST_PORT, "localhost", LOCAL_PORT, e); LOG.info(wrapped.toString(), wrapped); @@ -478,12 +480,12 @@ private IOException verifyExceptionClass(IOException e, static NetUtilsTestResolver resolver; static Configuration config; - @BeforeClass + @BeforeAll public static void setupResolver() { resolver = NetUtilsTestResolver.install(); } - @Before + @BeforeEach public void resetResolver() { resolver.reset(); config = new Configuration(); @@ -739,13 +741,13 @@ public void testNormalizeHostName() { // when ipaddress is normalized, same address is expected in return assertEquals(summary, hosts.get(0), normalizedHosts.get(0)); // for normalizing a resolvable hostname, resolved ipaddress is expected in return - assertFalse("Element 1 equal "+ summary, - normalizedHosts.get(1).equals(hosts.get(1))); + assertFalse( + normalizedHosts.get(1).equals(hosts.get(1)), "Element 1 equal "+ summary); assertEquals(summary, hosts.get(0), normalizedHosts.get(1)); // this address HADOOP-8372: when normalizing a valid resolvable hostname start with numeric, // its ipaddress is expected to return - assertFalse("Element 2 equal " + summary, - normalizedHosts.get(2).equals(hosts.get(2))); + assertFalse( + normalizedHosts.get(2).equals(hosts.get(2)), "Element 2 equal " + summary); // return the same hostname after normalizing a irresolvable hostname. assertEquals(summary, hosts.get(3), normalizedHosts.get(3)); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java index c2c528a9c9f21..fed5898db94fa 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java @@ -17,15 +17,15 @@ */ package org.apache.hadoop.net; -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.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.HashMap; import java.util.Map; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestNetworkTopologyWithNodeGroup { private final static NetworkTopologyWithNodeGroup cluster = new @@ -182,13 +182,13 @@ public void testChooseRandomExcludedNode() { @Test public void testNodeGroup() throws Exception { String res = cluster.getNodeGroup(""); - assertTrue("NodeGroup should be NodeBase.ROOT for empty location", - res.equals(NodeBase.ROOT)); + assertTrue( + res.equals(NodeBase.ROOT), "NodeGroup should be NodeBase.ROOT for empty location"); try { cluster.getNodeGroup(null); } catch (IllegalArgumentException e) { - assertTrue("Null Network Location should throw exception!", - e.getMessage().contains("Network Location is null")); + assertTrue( + e.getMessage().contains("Network Location is null"), "Null Network Location should throw exception!"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java index 0d0d5b15cd181..40578cfb58476 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java @@ -19,8 +19,8 @@ import java.util.ArrayList; import java.util.List; -import org.junit.Test; -import static org.junit.Assert.*; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.*; import org.apache.hadoop.conf.Configuration; @@ -42,16 +42,16 @@ public void testNoArgsMeansNoResult() { names.add("some.machine.name"); names.add("other.machine.name"); List result = mapping.resolve(names); - assertNull("Expected an empty list", result); + assertNull(result, "Expected an empty list"); } @Test public void testNoFilenameMeansSingleSwitch() throws Throwable { Configuration conf = new Configuration(); ScriptBasedMapping mapping = createMapping(conf); - assertTrue("Expected to be single switch", mapping.isSingleSwitch()); - assertTrue("Expected to be single switch", - AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping)); + assertTrue(mapping.isSingleSwitch(), "Expected to be single switch"); + assertTrue( + AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), "Expected to be single switch"); } @Test @@ -59,15 +59,15 @@ public void testFilenameMeansMultiSwitch() throws Throwable { Configuration conf = new Configuration(); conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename"); ScriptBasedMapping mapping = createMapping(conf); - assertFalse("Expected to be multi switch", mapping.isSingleSwitch()); + assertFalse(mapping.isSingleSwitch(), "Expected to be multi switch"); mapping.setConf(new Configuration()); - assertTrue("Expected to be single switch", mapping.isSingleSwitch()); + assertTrue(mapping.isSingleSwitch(), "Expected to be single switch"); } @Test public void testNullConfig() throws Throwable { ScriptBasedMapping mapping = createMapping(null); - assertTrue("Expected to be single switch", mapping.isSingleSwitch()); + assertTrue(mapping.isSingleSwitch(), "Expected to be single switch"); } private ScriptBasedMapping createMapping(Configuration conf) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java index 8638591aa5c80..4233f7506448b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java @@ -19,8 +19,8 @@ import java.util.ArrayList; import java.util.List; -import org.junit.Test; -import static org.junit.Assert.*; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.*; import org.apache.hadoop.conf.Configuration; @@ -46,18 +46,18 @@ public void testNoArgsMeansNoResult() { names.add("some.machine.name"); names.add("other.machine.name"); List result = mapping.resolve(names); - assertNull("Expected an empty list for resolve", result); + assertNull(result, "Expected an empty list for resolve"); result = mapping.getDependency("some.machine.name"); - assertNull("Expected an empty list for getDependency", result); + assertNull(result, "Expected an empty list for getDependency"); } @Test public void testNoFilenameMeansSingleSwitch() throws Throwable { Configuration conf = new Configuration(); ScriptBasedMapping mapping = createMapping(conf); - assertTrue("Expected to be single switch", mapping.isSingleSwitch()); - assertTrue("Expected to be single switch", - AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping)); + assertTrue(mapping.isSingleSwitch(), "Expected to be single switch"); + assertTrue( + AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), "Expected to be single switch"); } @Test @@ -65,15 +65,15 @@ public void testFilenameMeansMultiSwitch() throws Throwable { Configuration conf = new Configuration(); conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename"); ScriptBasedMapping mapping = createMapping(conf); - assertFalse("Expected to be multi switch", mapping.isSingleSwitch()); + assertFalse(mapping.isSingleSwitch(), "Expected to be multi switch"); mapping.setConf(new Configuration()); - assertTrue("Expected to be single switch", mapping.isSingleSwitch()); + assertTrue(mapping.isSingleSwitch(), "Expected to be single switch"); } @Test public void testNullConfig() throws Throwable { ScriptBasedMapping mapping = createMapping(null); - assertTrue("Expected to be single switch", mapping.isSingleSwitch()); + assertTrue(mapping.isSingleSwitch(), "Expected to be single switch"); } private ScriptBasedMappingWithDependency createMapping(Configuration conf) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java index 008d842937158..682c7cfb304e5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java @@ -38,11 +38,11 @@ import org.apache.hadoop.util.Shell; import org.apache.hadoop.io.nativeio.NativeIO; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; /** * This tests timeout out from SocketInputStream and diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java index a906c4aa85615..d019b0b21445f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java @@ -20,8 +20,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,7 +33,7 @@ * Test the static mapping class. * Because the map is actually static, this map needs to be reset for every test */ -public class TestStaticMapping extends Assert { +public class TestStaticMapping extends Assertions { private static final Logger LOG = LoggerFactory.getLogger(TestStaticMapping.class); @@ -78,23 +78,23 @@ private Configuration createConf(String script) { } private void assertSingleSwitch(DNSToSwitchMapping mapping) { - assertEquals("Expected a single switch mapping " - + mapping, - true, - AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping)); + assertEquals( + true +, AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), "Expected a single switch mapping " + + mapping); } private void assertMultiSwitch(DNSToSwitchMapping mapping) { - assertEquals("Expected a multi switch mapping " - + mapping, - false, - AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping)); + assertEquals( + false +, AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), "Expected a multi switch mapping " + + mapping); } protected void assertMapSize(AbstractDNSToSwitchMapping switchMapping, int expectedSize) { assertEquals( - "Expected two entries in the map " + switchMapping.dumpTopology(), - expectedSize, switchMapping.getSwitchMap().size()); + + expectedSize, switchMapping.getSwitchMap().size(), "Expected two entries in the map " + switchMapping.dumpTopology()); } private List createQueryList() { @@ -130,7 +130,7 @@ public void testAddResolveNodes() throws Throwable { Map switchMap = mapping.getSwitchMap(); String topology = mapping.dumpTopology(); LOG.info(topology); - assertEquals(topology, 1, switchMap.size()); + assertEquals(1, switchMap.size(), topology); assertEquals(topology, "/r1", switchMap.get("n1")); } @@ -160,9 +160,9 @@ public void testReadNodesFromConfig() throws Throwable { Map switchMap = mapping.getSwitchMap(); String topology = mapping.dumpTopology(); LOG.info(topology); - assertEquals(topology, 2, switchMap.size()); - assertEquals(topology, "/r1", switchMap.get("n1")); - assertNull(topology, switchMap.get("unknown")); + assertEquals(2, switchMap.size(), topology); + assertEquals("/r1", switchMap.get("n1"), topology); + assertNull(switchMap.get("unknown"), topology); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java index b5de661caca41..2c4b7609beff7 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java @@ -20,15 +20,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.List; /** * Test some other details of the switch mapping */ -public class TestSwitchMapping extends Assert { +public class TestSwitchMapping extends Assertions { /** @@ -40,8 +40,8 @@ public class TestSwitchMapping extends Assert { @Test public void testStandaloneClassesAssumedMultiswitch() throws Throwable { DNSToSwitchMapping mapping = new StandaloneSwitchMapping(); - assertFalse("Expected to be multi switch " + mapping, - AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping)); + assertFalse(AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), + "Expected to be multi switch " + mapping); } @@ -55,8 +55,8 @@ public void testStandaloneClassesAssumedMultiswitch() throws Throwable { public void testCachingRelays() throws Throwable { CachedDNSToSwitchMapping mapping = new CachedDNSToSwitchMapping(new StandaloneSwitchMapping()); - assertFalse("Expected to be multi switch " + mapping, - mapping.isSingleSwitch()); + assertFalse(mapping.isSingleSwitch(), + "Expected to be multi switch " + mapping); } @@ -73,12 +73,12 @@ public void testCachingRelaysStringOperations() throws Throwable { conf.set(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, scriptname); ScriptBasedMapping scriptMapping = new ScriptBasedMapping(conf); - assertTrue("Did not find " + scriptname + " in " + scriptMapping, - scriptMapping.toString().contains(scriptname)); + assertTrue(scriptMapping.toString().contains(scriptname), + "Did not find " + scriptname + " in " + scriptMapping); CachedDNSToSwitchMapping mapping = new CachedDNSToSwitchMapping(scriptMapping); - assertTrue("Did not find " + scriptname + " in " + mapping, - mapping.toString().contains(scriptname)); + assertTrue(mapping.toString().contains(scriptname), + "Did not find " + scriptname + " in " + mapping); } /** @@ -91,14 +91,14 @@ public void testCachingRelaysStringOperations() throws Throwable { public void testCachingRelaysStringOperationsToNullScript() throws Throwable { Configuration conf = new Configuration(); ScriptBasedMapping scriptMapping = new ScriptBasedMapping(conf); - assertTrue("Did not find " + ScriptBasedMapping.NO_SCRIPT - + " in " + scriptMapping, - scriptMapping.toString().contains(ScriptBasedMapping.NO_SCRIPT)); + assertTrue( + scriptMapping.toString().contains(ScriptBasedMapping.NO_SCRIPT), "Did not find " + ScriptBasedMapping.NO_SCRIPT + + " in " + scriptMapping); CachedDNSToSwitchMapping mapping = new CachedDNSToSwitchMapping(scriptMapping); - assertTrue("Did not find " + ScriptBasedMapping.NO_SCRIPT - + " in " + mapping, - mapping.toString().contains(ScriptBasedMapping.NO_SCRIPT)); + assertTrue( + mapping.toString().contains(ScriptBasedMapping.NO_SCRIPT), "Did not find " + ScriptBasedMapping.NO_SCRIPT + + " in " + mapping); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java index 50fe0c098f478..6b571b19b27dc 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java @@ -19,7 +19,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.NET_TOPOLOGY_TABLE_MAPPING_FILE_KEY; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import org.apache.hadoop.thirdparty.com.google.common.io.Files; @@ -31,7 +31,8 @@ import org.apache.hadoop.conf.Configuration; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestTableMapping { private String hostName1 = "1.2.3.4"; @@ -162,7 +163,8 @@ public void testClearingCachedMappings() throws IOException { } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testBadFile() throws IOException { File mapFile = File.createTempFile(getClass().getSimpleName() + ".testBadFile", ".txt"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java index 952f2b35e4314..fdd9ef51387ad 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java @@ -37,12 +37,13 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.junit.AfterClass; -import org.junit.Assert; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; import org.junit.Assume; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.net.unix.DomainSocket.DomainChannel; @@ -54,18 +55,18 @@ public class TestDomainSocket { private static TemporarySocketDirectory sockDir; - @BeforeClass + @BeforeAll public static void init() { sockDir = new TemporarySocketDirectory(); DomainSocket.disableBindPathValidation(); } - @AfterClass + @AfterAll public static void shutdown() throws IOException { sockDir.close(); } - @Before + @BeforeEach public void before() { Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null); } @@ -76,7 +77,8 @@ public void before() { * * @throws IOException */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testSocketCreateAndClose() throws IOException { DomainSocket serv = DomainSocket.bindAndListen( new File(sockDir.getDir(), "test_sock_create_and_close"). @@ -89,9 +91,10 @@ public void testSocketCreateAndClose() throws IOException { * * @throws IOException */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testSocketPathSetGet() throws IOException { - Assert.assertEquals("/var/run/hdfs/sock.100", + Assertions.assertEquals("/var/run/hdfs/sock.100", DomainSocket.getEffectivePath("/var/run/hdfs/sock._PORT", 100)); } @@ -100,7 +103,8 @@ public void testSocketPathSetGet() throws IOException { * * @throws IOException */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testSocketReadEof() throws Exception { final String TEST_PATH = new File(sockDir.getDir(), "testSocketReadEof").getAbsolutePath(); @@ -119,7 +123,7 @@ public Void call(){ buf[i] = 0; } try { - Assert.assertEquals(-1, conn.getInputStream().read()); + Assertions.assertEquals(-1, conn.getInputStream().read()); } catch (IOException e) { throw new RuntimeException("unexpected IOException", e); } @@ -140,7 +144,8 @@ public Void call(){ * * @throws IOException */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testSocketAcceptAndClose() throws Exception { final String TEST_PATH = new File(sockDir.getDir(), "test_sock_accept_and_close").getAbsolutePath(); @@ -245,12 +250,14 @@ public Void call(){ serverFuture.get(2, TimeUnit.MINUTES); } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testAsyncCloseDuringWrite() throws Exception { testAsyncCloseDuringIO(true); } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testAsyncCloseDuringRead() throws Exception { testAsyncCloseDuringIO(false); } @@ -260,7 +267,8 @@ public void testAsyncCloseDuringRead() throws Exception { * * @throws IOException */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testInvalidOperations() throws IOException { try { DomainSocket.connect( @@ -276,7 +284,8 @@ public void testInvalidOperations() throws IOException { * * @throws IOException */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testServerOptions() throws Exception { final String TEST_PATH = new File(sockDir.getDir(), "test_sock_server_options").getAbsolutePath(); @@ -286,19 +295,19 @@ public void testServerOptions() throws Exception { int newBufSize = bufSize / 2; serv.setAttribute(DomainSocket.RECEIVE_BUFFER_SIZE, newBufSize); int nextBufSize = serv.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE); - Assert.assertEquals(newBufSize, nextBufSize); + Assertions.assertEquals(newBufSize, nextBufSize); // Let's set a server timeout int newTimeout = 1000; serv.setAttribute(DomainSocket.RECEIVE_TIMEOUT, newTimeout); int nextTimeout = serv.getAttribute(DomainSocket.RECEIVE_TIMEOUT); - Assert.assertEquals(newTimeout, nextTimeout); + Assertions.assertEquals(newTimeout, nextTimeout); ExecutorService exeServ = Executors.newSingleThreadExecutor(); Callable callable = new Callable() { public Void call() { try { serv.accept(); - Assert.fail("expected the accept() to time out and fail"); + Assertions.fail("expected the accept() to time out and fail"); } catch (SocketTimeoutException e) { GenericTestUtils.assertExceptionContains("accept(2) error: ", e); } catch (AsynchronousCloseException e) { @@ -313,7 +322,7 @@ public Void call() { Thread.sleep(500); serv.close(true); future.get(); - Assert.assertFalse(serv.isOpen()); + Assertions.assertFalse(serv.isOpen()); } /** @@ -454,17 +463,17 @@ public void run(){ ReadStrategy reader = readStrategyClass.newInstance(); reader.init(conn); reader.readFully(in1, 0, in1.length); - Assert.assertTrue(Arrays.equals(clientMsg1, in1)); + Assertions.assertTrue(Arrays.equals(clientMsg1, in1)); WriteStrategy writer = writeStrategyClass.newInstance(); writer.init(conn); writer.write(serverMsg1); InputStream connInputStream = conn.getInputStream(); int in2 = connInputStream.read(); - Assert.assertEquals((int)clientMsg2, in2); + Assertions.assertEquals((int)clientMsg2, in2); conn.close(); } catch (Throwable e) { threadResults.add(e); - Assert.fail(e.getMessage()); + Assertions.fail(e.getMessage()); } threadResults.add(new Success()); } @@ -483,7 +492,7 @@ public void run(){ reader.init(client); byte in1[] = new byte[serverMsg1.length]; reader.readFully(in1, 0, in1.length); - Assert.assertTrue(Arrays.equals(serverMsg1, in1)); + Assertions.assertTrue(Arrays.equals(serverMsg1, in1)); OutputStream clientOutputStream = client.getOutputStream(); clientOutputStream.write(clientMsg2); client.close(); @@ -498,7 +507,7 @@ public void run(){ for (int i = 0; i < 2; i++) { Throwable t = threadResults.take(); if (!(t instanceof Success)) { - Assert.fail(t.getMessage() + ExceptionUtils.getStackTrace(t)); + Assertions.fail(t.getMessage() + ExceptionUtils.getStackTrace(t)); } } serverThread.join(120000); @@ -508,37 +517,43 @@ public void run(){ } } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testClientServerOutStreamInStream() throws Exception { testClientServer1(OutputStreamWriteStrategy.class, InputStreamReadStrategy.class, null); } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testClientServerOutStreamInStreamWithSocketpair() throws Exception { testClientServer1(OutputStreamWriteStrategy.class, InputStreamReadStrategy.class, DomainSocket.socketpair()); } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testClientServerOutStreamInDbb() throws Exception { testClientServer1(OutputStreamWriteStrategy.class, DirectByteBufferReadStrategy.class, null); } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testClientServerOutStreamInDbbWithSocketpair() throws Exception { testClientServer1(OutputStreamWriteStrategy.class, DirectByteBufferReadStrategy.class, DomainSocket.socketpair()); } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testClientServerOutStreamInAbb() throws Exception { testClientServer1(OutputStreamWriteStrategy.class, ArrayBackedByteBufferReadStrategy.class, null); } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testClientServerOutStreamInAbbWithSocketpair() throws Exception { testClientServer1(OutputStreamWriteStrategy.class, ArrayBackedByteBufferReadStrategy.class, DomainSocket.socketpair()); @@ -589,7 +604,8 @@ protected void finalize() { * * @throws IOException */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testFdPassing() throws Exception { final String TEST_PATH = new File(sockDir.getDir(), "test_sock").getAbsolutePath(); @@ -614,14 +630,14 @@ public void run(){ byte in1[] = new byte[clientMsg1.length]; InputStream connInputStream = conn.getInputStream(); IOUtils.readFully(connInputStream, in1, 0, in1.length); - Assert.assertTrue(Arrays.equals(clientMsg1, in1)); + Assertions.assertTrue(Arrays.equals(clientMsg1, in1)); DomainSocket domainConn = (DomainSocket)conn; domainConn.sendFileDescriptors(passedFds, serverMsg1, 0, serverMsg1.length); conn.close(); } catch (Throwable e) { threadResults.add(e); - Assert.fail(e.getMessage()); + Assertions.fail(e.getMessage()); } threadResults.add(new Success()); } @@ -640,11 +656,11 @@ public void run(){ FileInputStream recvFis[] = new FileInputStream[passedFds.length]; int r = domainConn. recvFileInputStreams(recvFis, in1, 0, in1.length - 1); - Assert.assertTrue(r > 0); + Assertions.assertTrue(r > 0); IOUtils.readFully(clientInputStream, in1, r, in1.length - r); - Assert.assertTrue(Arrays.equals(serverMsg1, in1)); + Assertions.assertTrue(Arrays.equals(serverMsg1, in1)); for (int i = 0; i < passedFds.length; i++) { - Assert.assertNotNull(recvFis[i]); + Assertions.assertNotNull(recvFis[i]); passedFiles[i].checkInputStream(recvFis[i]); } for (FileInputStream fis : recvFis) { @@ -662,7 +678,7 @@ public void run(){ for (int i = 0; i < 2; i++) { Throwable t = threadResults.take(); if (!(t instanceof Success)) { - Assert.fail(t.getMessage() + ExceptionUtils.getStackTrace(t)); + Assertions.fail(t.getMessage() + ExceptionUtils.getStackTrace(t)); } } serverThread.join(120000); @@ -700,7 +716,8 @@ private static void testValidateSocketPath(String str, String prefix) * * @throws IOException */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testFdPassingPathSecurity() throws Exception { TemporarySocketDirectory tmp = new TemporarySocketDirectory(); try { @@ -739,7 +756,8 @@ public void testFdPassingPathSecurity() throws Exception { } } - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testShutdown() throws Exception { final AtomicInteger bytesRead = new AtomicInteger(0); final AtomicBoolean failed = new AtomicBoolean(false); @@ -765,11 +783,11 @@ public void run() { socks[0].getOutputStream().write(1); socks[0].getOutputStream().write(2); socks[0].getOutputStream().write(3); - Assert.assertTrue(readerThread.isAlive()); + Assertions.assertTrue(readerThread.isAlive()); socks[0].shutdown(); readerThread.join(); - Assert.assertFalse(failed.get()); - Assert.assertEquals(3, bytesRead.get()); + Assertions.assertFalse(failed.get()); + Assertions.assertEquals(3, bytesRead.get()); IOUtils.cleanupWithLogger(null, socks); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocketWatcher.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocketWatcher.java index ca801dac2c247..6e9f3073b27f0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocketWatcher.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocketWatcher.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.net.unix; -import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.assertFalse; import java.util.ArrayList; import java.util.Random; @@ -26,10 +26,11 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; -import org.junit.After; +import org.junit.jupiter.api.AfterEach; import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Uninterruptibles; import org.slf4j.Logger; @@ -41,12 +42,12 @@ public class TestDomainSocketWatcher { private Throwable trappedException = null; - @Before + @BeforeEach public void before() { Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null); } - @After + @AfterEach public void after() { if (trappedException != null) { throw new IllegalStateException( @@ -58,7 +59,8 @@ public void after() { /** * Test that we can create a DomainSocketWatcher and then shut it down. */ - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testCreateShutdown() throws Exception { DomainSocketWatcher watcher = newDomainSocketWatcher(10000000); watcher.close(); @@ -67,7 +69,8 @@ public void testCreateShutdown() throws Exception { /** * Test that we can get notifications out a DomainSocketWatcher. */ - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testDeliverNotifications() throws Exception { DomainSocketWatcher watcher = newDomainSocketWatcher(10000000); DomainSocket pair[] = DomainSocket.socketpair(); @@ -87,7 +90,8 @@ public boolean handle(DomainSocket sock) { /** * Test that a java interruption can stop the watcher thread */ - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testInterruption() throws Exception { final DomainSocketWatcher watcher = newDomainSocketWatcher(10); watcher.watcherThread.interrupt(); @@ -98,7 +102,8 @@ public void testInterruption() throws Exception { /** * Test that domain sockets are closed when the watcher is closed. */ - @Test(timeout=300000) + @Test + @Timeout(value = 300) public void testCloseSocketOnWatcherClose() throws Exception { final DomainSocketWatcher watcher = newDomainSocketWatcher(10000000); DomainSocket pair[] = DomainSocket.socketpair(); @@ -113,7 +118,8 @@ public boolean handle(DomainSocket sock) { assertFalse(pair[1].isOpen()); } - @Test(timeout=300000) + @Test + @Timeout(value = 300) public void testStress() throws Exception { final int SOCKET_NUM = 250; final ReentrantLock lock = new ReentrantLock(); @@ -183,7 +189,8 @@ public void run() { watcher.close(); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testStressInterruption() throws Exception { final int SOCKET_NUM = 250; final ReentrantLock lock = new ReentrantLock(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java index a8141d762d151..4bd3bcc4ff302 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java @@ -18,9 +18,9 @@ package org.apache.hadoop.oncrpc; -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.ArrayList; import java.util.List; @@ -35,7 +35,7 @@ import org.apache.hadoop.oncrpc.security.CredentialsNone; import org.apache.hadoop.oncrpc.security.VerifierNone; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.slf4j.event.Level; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java index 44c1ee2e986c8..99863943c436c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java @@ -17,13 +17,14 @@ */ package org.apache.hadoop.oncrpc; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import org.apache.hadoop.oncrpc.RpcAcceptedReply.AcceptState; import org.apache.hadoop.oncrpc.RpcReply.ReplyState; import org.apache.hadoop.oncrpc.security.Verifier; import org.apache.hadoop.oncrpc.security.VerifierNone; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Test for {@link RpcAcceptedReply} @@ -39,9 +40,10 @@ public void testAcceptState() { assertEquals(AcceptState.SYSTEM_ERR, AcceptState.fromValue(5)); } - @Test(expected = IndexOutOfBoundsException.class) + @Test public void testAcceptStateFromInvalidValue() { - AcceptState.fromValue(6); + assertThrows(IndexOutOfBoundsException.class, ()-> + AcceptState.fromValue(6)); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java index 2a5705a99b679..0219094607a4f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java @@ -17,13 +17,14 @@ */ package org.apache.hadoop.oncrpc; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import org.apache.hadoop.oncrpc.security.CredentialsNone; import org.apache.hadoop.oncrpc.security.Credentials; import org.apache.hadoop.oncrpc.security.Verifier; import org.apache.hadoop.oncrpc.security.VerifierNone; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Tests for {@link RpcCall} @@ -50,15 +51,19 @@ public void testConstructor() { assertEquals(verifier, call.getVerifier()); } - @Test(expected=IllegalArgumentException.class) + @Test public void testInvalidRpcVersion() { - int invalidRpcVersion = 3; - new RpcCall(0, RpcMessage.Type.RPC_CALL, invalidRpcVersion, 2, 3, 4, null, null); + assertThrows(IllegalArgumentException.class, () -> { + int invalidRpcVersion = 3; + new RpcCall(0, RpcMessage.Type.RPC_CALL, invalidRpcVersion, 2, 3, 4, null, null); + }); } - @Test(expected=IllegalArgumentException.class) + @Test public void testInvalidRpcMessageType() { - RpcMessage.Type invalidMessageType = RpcMessage.Type.RPC_REPLY; // Message typ is not RpcMessage.RPC_CALL - new RpcCall(0, invalidMessageType, RpcCall.RPC_VERSION, 2, 3, 4, null, null); + assertThrows(IllegalArgumentException.class, () -> { + RpcMessage.Type invalidMessageType = RpcMessage.Type.RPC_REPLY; // Message typ is not RpcMessage.RPC_CALL + new RpcCall(0, invalidMessageType, RpcCall.RPC_VERSION, 2, 3, 4, null, null); + }); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCallCache.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCallCache.java index 5e5cdc010a514..0e1df10bf5449 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCallCache.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCallCache.java @@ -17,12 +17,6 @@ */ package org.apache.hadoop.oncrpc; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Iterator; @@ -30,8 +24,9 @@ import org.apache.hadoop.oncrpc.RpcCallCache.CacheEntry; import org.apache.hadoop.oncrpc.RpcCallCache.ClientRequest; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.*; /** @@ -39,14 +34,16 @@ */ public class TestRpcCallCache { - @Test(expected=IllegalArgumentException.class) - public void testRpcCallCacheConstructorIllegalArgument0(){ - new RpcCallCache("test", 0); + @Test + public void testRpcCallCacheConstructorIllegalArgument0() { + assertThrows(IllegalArgumentException.class, () -> + new RpcCallCache("test", 0)); } - @Test(expected=IllegalArgumentException.class) - public void testRpcCallCacheConstructorIllegalArgumentNegative(){ - new RpcCallCache("test", -1); + @Test + public void testRpcCallCacheConstructorIllegalArgumentNegative() { + assertThrows(IllegalArgumentException.class, () -> + new RpcCallCache("test", -1)); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java index 31e5f98e66ee7..a2880f76a80d6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java @@ -20,8 +20,10 @@ import org.apache.hadoop.oncrpc.RpcDeniedReply.RejectState; import org.apache.hadoop.oncrpc.RpcReply.ReplyState; import org.apache.hadoop.oncrpc.security.VerifierNone; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Test for {@link RpcDeniedReply} @@ -29,22 +31,23 @@ public class TestRpcDeniedReply { @Test public void testRejectStateFromValue() { - Assert.assertEquals(RejectState.RPC_MISMATCH, RejectState.fromValue(0)); - Assert.assertEquals(RejectState.AUTH_ERROR, RejectState.fromValue(1)); + Assertions.assertEquals(RejectState.RPC_MISMATCH, RejectState.fromValue(0)); + Assertions.assertEquals(RejectState.AUTH_ERROR, RejectState.fromValue(1)); } - @Test(expected=IndexOutOfBoundsException.class) + @Test public void testRejectStateFromInvalidValue1() { - RejectState.fromValue(2); + assertThrows(IndexOutOfBoundsException.class, () -> + RejectState.fromValue(2)); } @Test public void testConstructor() { RpcDeniedReply reply = new RpcDeniedReply(0, ReplyState.MSG_ACCEPTED, RejectState.AUTH_ERROR, new VerifierNone()); - Assert.assertEquals(0, reply.getXid()); - Assert.assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType()); - Assert.assertEquals(ReplyState.MSG_ACCEPTED, reply.getState()); - Assert.assertEquals(RejectState.AUTH_ERROR, reply.getRejectState()); + Assertions.assertEquals(0, reply.getXid()); + Assertions.assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType()); + Assertions.assertEquals(ReplyState.MSG_ACCEPTED, reply.getState()); + Assertions.assertEquals(RejectState.AUTH_ERROR, reply.getRejectState()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java index 435e30bbc4463..62729b0bab423 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java @@ -17,8 +17,10 @@ */ package org.apache.hadoop.oncrpc; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Test for {@link RpcMessage} @@ -36,8 +38,8 @@ public XDR write(XDR xdr) { @Test public void testRpcMessage() { RpcMessage msg = getRpcMessage(0, RpcMessage.Type.RPC_CALL); - Assert.assertEquals(0, msg.getXid()); - Assert.assertEquals(RpcMessage.Type.RPC_CALL, msg.getMessageType()); + Assertions.assertEquals(0, msg.getXid()); + Assertions.assertEquals(RpcMessage.Type.RPC_CALL, msg.getMessageType()); } @Test @@ -46,9 +48,11 @@ public void testValidateMessage() { msg.validateMessageType(RpcMessage.Type.RPC_CALL); } - @Test(expected = IllegalArgumentException.class) + @Test public void testValidateMessageException() { - RpcMessage msg = getRpcMessage(0, RpcMessage.Type.RPC_CALL); - msg.validateMessageType(RpcMessage.Type.RPC_REPLY); + assertThrows(IllegalArgumentException.class, () -> { + RpcMessage msg = getRpcMessage(0, RpcMessage.Type.RPC_CALL); + msg.validateMessageType(RpcMessage.Type.RPC_REPLY); + }); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java index 1483a7901cdc1..16663572b3cbc 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java @@ -20,8 +20,10 @@ import org.apache.hadoop.oncrpc.RpcReply.ReplyState; import org.apache.hadoop.oncrpc.security.VerifierNone; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Test for {@link RpcReply} @@ -29,13 +31,13 @@ public class TestRpcReply { @Test public void testReplyStateFromValue() { - Assert.assertEquals(ReplyState.MSG_ACCEPTED, ReplyState.fromValue(0)); - Assert.assertEquals(ReplyState.MSG_DENIED, ReplyState.fromValue(1)); + Assertions.assertEquals(ReplyState.MSG_ACCEPTED, ReplyState.fromValue(0)); + Assertions.assertEquals(ReplyState.MSG_DENIED, ReplyState.fromValue(1)); } - @Test(expected=IndexOutOfBoundsException.class) + @Test public void testReplyStateFromInvalidValue1() { - ReplyState.fromValue(2); + assertThrows(IndexOutOfBoundsException.class, () -> ReplyState.fromValue(2)); } @Test @@ -47,8 +49,8 @@ public XDR write(XDR xdr) { return null; } }; - Assert.assertEquals(0, reply.getXid()); - Assert.assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType()); - Assert.assertEquals(ReplyState.MSG_ACCEPTED, reply.getState()); + Assertions.assertEquals(0, reply.getXid()); + Assertions.assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType()); + Assertions.assertEquals(ReplyState.MSG_ACCEPTED, reply.getState()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java index 4c6c735c5cefd..7c64f6d81acc6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.oncrpc; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestXDR { static final int WRITE_VALUE=23; @@ -29,7 +29,7 @@ private void serializeInt(int times) { XDR r = w.asReadOnlyWrap(); for (int i = 0; i < times; ++i) - Assert.assertEquals( + Assertions.assertEquals( WRITE_VALUE,r.readInt()); } @@ -40,7 +40,7 @@ private void serializeLong(int times) { XDR r = w.asReadOnlyWrap(); for (int i = 0; i < times; ++i) - Assert.assertEquals(WRITE_VALUE, r.readHyper()); + Assertions.assertEquals(WRITE_VALUE, r.readHyper()); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java index 147eed774dacf..5c595b1e46058 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.oncrpc.security; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import org.apache.hadoop.oncrpc.XDR; import org.apache.hadoop.oncrpc.security.CredentialsSys; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Test for {@link CredentialsSys} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java index b458dd0c235d2..ec069f5c407d7 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java @@ -17,11 +17,12 @@ */ package org.apache.hadoop.oncrpc.security; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import org.apache.hadoop.oncrpc.security.RpcAuthInfo; import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Tests for {@link RpcAuthInfo} @@ -36,8 +37,9 @@ public void testAuthFlavor() { assertEquals(AuthFlavor.RPCSEC_GSS, AuthFlavor.fromValue(6)); } - @Test(expected=IllegalArgumentException.class) + @Test public void testInvalidAuthFlavor() { - assertEquals(AuthFlavor.AUTH_NONE, AuthFlavor.fromValue(4)); + assertThrows(IllegalArgumentException.class, ()-> + assertEquals(AuthFlavor.AUTH_NONE, AuthFlavor.fromValue(4))); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/portmap/TestPortmap.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/portmap/TestPortmap.java index 35ab5cdc3da67..90bd2a1e314fc 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/portmap/TestPortmap.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/portmap/TestPortmap.java @@ -27,17 +27,18 @@ import java.util.Map; import org.apache.hadoop.oncrpc.RpcReply; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.apache.hadoop.oncrpc.RpcCall; import org.apache.hadoop.oncrpc.XDR; import org.apache.hadoop.oncrpc.security.CredentialsNone; import org.apache.hadoop.oncrpc.security.VerifierNone; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; public class TestPortmap { private static Portmap pm = new Portmap(); @@ -45,18 +46,19 @@ public class TestPortmap { private static final int RETRY_TIMES = 5; private int xid; - @BeforeClass + @BeforeAll public static void setup() throws InterruptedException { pm.start(SHORT_TIMEOUT_MILLISECONDS, new InetSocketAddress("localhost", 0), new InetSocketAddress("localhost", 0)); } - @AfterClass + @AfterAll public static void tearDown() { pm.shutdown(); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testIdle() throws InterruptedException, IOException { Socket s = new Socket(); try { @@ -68,17 +70,18 @@ public void testIdle() throws InterruptedException, IOException { Thread.sleep(SHORT_TIMEOUT_MILLISECONDS); } - Assert.assertTrue("Failed to connect to the server", s.isConnected() - && i < RETRY_TIMES); + Assertions.assertTrue(s.isConnected() + && i < RETRY_TIMES, "Failed to connect to the server"); int b = s.getInputStream().read(); - Assert.assertTrue("The server failed to disconnect", b == -1); + Assertions.assertTrue(b == -1, "The server failed to disconnect"); } finally { s.close(); } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testRegistration() throws IOException, InterruptedException, IllegalAccessException { XDR req = new XDR(); RpcCall.getInstance(++xid, RpcProgramPortmap.PROGRAM, @@ -125,6 +128,6 @@ public void testRegistration() throws IOException, InterruptedException, Illegal break; } } - Assert.assertTrue("Registration failed", found); + Assertions.assertTrue(found, "Registration failed"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ManualTestKeytabLogins.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ManualTestKeytabLogins.java index 632ceaf1ff109..18faea29ad1fd 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ManualTestKeytabLogins.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ManualTestKeytabLogins.java @@ -18,7 +18,7 @@ package org.apache.hadoop.security; import org.apache.hadoop.security.UserGroupInformation; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Regression test for HADOOP-6947 which can be run manually in diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java index e0b3e515a0f9d..b8b56c522894c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java @@ -17,14 +17,14 @@ package org.apache.hadoop.security; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.ArgumentMatchers.any; import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.security.authentication.server.AuthenticationFilter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.http.FilterContainer; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCompositeGroupMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCompositeGroupMapping.java index 1803fb1a05806..6a4b46c6f57f1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCompositeGroupMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCompositeGroupMapping.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.security; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.ArrayList; @@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java index 02ba1539d4190..58859f08d4b39 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java @@ -42,21 +42,21 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import static org.junit.Assert.*; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.*; public class TestCredentials { private static final String DEFAULT_HMAC_ALGORITHM = "HmacSHA1"; private static final File tmpDir = GenericTestUtils.getTestDir("mapred"); - @Before + @BeforeEach public void setUp() { tmpDir.mkdir(); } - @After + @AfterEach public void tearDown() { tmpDir.delete(); } @@ -111,29 +111,29 @@ public void testReadWriteStorage() // get the tokens and compare the services Map> tokenMap = ts.getTokenMap(); - assertEquals("getTokenMap should return collection of size 2", 2, - tokenMap.size()); - assertTrue("Token for alias " + alias1 + " must be present", - tokenMap.containsKey(alias1)); - assertTrue("Token for alias " + alias2 + " must be present", - tokenMap.containsKey(alias2)); - assertEquals("Token for service " + service1 + " must be present", service1, - tokenMap.get(alias1).getService()); - assertEquals("Token for service " + service2 + " must be present", service2, - tokenMap.get(alias2).getService()); + assertEquals(2 +, tokenMap.size(), "getTokenMap should return collection of size 2"); + assertTrue( + tokenMap.containsKey(alias1), "Token for alias " + alias1 + " must be present"); + assertTrue( + tokenMap.containsKey(alias2), "Token for alias " + alias2 + " must be present"); + assertEquals(service1 +, tokenMap.get(alias1).getService(), "Token for service " + service1 + " must be present"); + assertEquals(service2 +, tokenMap.get(alias2).getService(), "Token for service " + service2 + " must be present"); // compare secret keys Map secretKeyMap = ts.getSecretKeyMap(); - assertEquals("wrong number of keys in the Storage", m.size(), - ts.numberOfSecretKeys()); + assertEquals(m.size() +, ts.numberOfSecretKeys(), "wrong number of keys in the Storage"); for (Map.Entry entry : m.entrySet()) { byte[] key = secretKeyMap.get(entry.getKey()); - assertNotNull("Secret key for alias " + entry.getKey() + " not found", - key); - assertTrue("Keys don't match for alias " + entry.getKey(), - Arrays.equals(key, entry.getValue())); + assertNotNull( + key, "Secret key for alias " + entry.getKey() + " not found"); + assertTrue( + Arrays.equals(key, entry.getValue()), "Keys don't match for alias " + entry.getKey()); } tmpFileName.delete(); @@ -146,8 +146,8 @@ public void testBasicReadWriteProtoEmpty() Credentials ts = new Credentials(); writeCredentialsProto(ts, testname); Credentials ts2 = readCredentialsProto(testname); - assertEquals("test empty tokens", 0, ts2.numberOfTokens()); - assertEquals("test empty keys", 0, ts2.numberOfSecretKeys()); + assertEquals(0, ts2.numberOfTokens(), "test empty tokens"); + assertEquals(0, ts2.numberOfSecretKeys(), "test empty keys"); } @Test @@ -171,8 +171,8 @@ public void testBasicReadWriteStreamEmpty() Credentials ts = new Credentials(); writeCredentialsStream(ts, testname); Credentials ts2 = readCredentialsStream(testname); - assertEquals("test empty tokens", 0, ts2.numberOfTokens()); - assertEquals("test empty keys", 0, ts2.numberOfSecretKeys()); + assertEquals(0, ts2.numberOfTokens(), "test empty tokens"); + assertEquals(0, ts2.numberOfSecretKeys(), "test empty keys"); } @Test @@ -201,10 +201,10 @@ public void testWritablePropertiesEmpty() Credentials ts2 = new Credentials(); writeCredentialsProtos(ts, ts2, testname); List clist = readCredentialsProtos(testname); - assertEquals("test empty tokens 0", 0, clist.get(0).numberOfTokens()); - assertEquals("test empty keys 0", 0, clist.get(0).numberOfSecretKeys()); - assertEquals("test empty tokens 1", 0, clist.get(1).numberOfTokens()); - assertEquals("test empty keys 1", 0, clist.get(1).numberOfSecretKeys()); + assertEquals(0, clist.get(0).numberOfTokens(), "test empty tokens 0"); + assertEquals(0, clist.get(0).numberOfSecretKeys(), "test empty keys 0"); + assertEquals(0, clist.get(1).numberOfTokens(), "test empty tokens 1"); + assertEquals(0, clist.get(1).numberOfSecretKeys(), "test empty keys 1"); } @Test @@ -245,22 +245,22 @@ private Credentials generateCredentials(Text t1, Text t2, Text t3) private void assertCredentials(String tag, Text alias, Text keykey, Credentials a, Credentials b) { - assertEquals(tag + ": test token count", a.numberOfTokens(), - b.numberOfTokens()); - assertEquals(tag + ": test service", a.getToken(alias).getService(), - b.getToken(alias).getService()); - assertEquals(tag + ": test kind", a.getToken(alias).getKind(), - b.getToken(alias).getKind()); - assertTrue(tag + ": test password", - Arrays.equals(a.getToken(alias).getPassword(), - b.getToken(alias).getPassword())); - assertTrue(tag + ": test identifier", - Arrays.equals(a.getToken(alias).getIdentifier(), - b.getToken(alias).getIdentifier())); - assertEquals(tag + ": test number of keys", a.numberOfSecretKeys(), - b.numberOfSecretKeys()); - assertTrue(tag + ":test key values", Arrays.equals(a.getSecretKey(keykey), - b.getSecretKey(keykey))); + assertEquals(a.numberOfTokens() +, b.numberOfTokens(), tag + ": test token count"); + assertEquals(a.getToken(alias).getService() +, b.getToken(alias).getService(), tag + ": test service"); + assertEquals(a.getToken(alias).getKind() +, b.getToken(alias).getKind(), tag + ": test kind"); + assertTrue( + Arrays.equals(a.getToken(alias).getPassword(), + b.getToken(alias).getPassword()), tag + ": test password"); + assertTrue( + Arrays.equals(a.getToken(alias).getIdentifier(), + b.getToken(alias).getIdentifier()), tag + ": test identifier"); + assertEquals(a.numberOfSecretKeys() +, b.numberOfSecretKeys(), tag + ": test number of keys"); + assertTrue(Arrays.equals(a.getSecretKey(keykey), + b.getSecretKey(keykey)), tag + ":test key values"); } private void writeCredentialsStream(Credentials creds, String filename) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java index edd537011c4a8..5d58fc5277b1b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java @@ -29,9 +29,10 @@ import org.apache.hadoop.security.authorize.DefaultImpersonationProvider; import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.security.token.Token; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,7 +70,7 @@ public class TestDoAsEffectiveUser extends TestRpcBase { + "DEFAULT"); } - @Before + @BeforeEach public void setMasterConf() throws IOException { UserGroupInformation.setConfiguration(masterConf); refreshConf(masterConf); @@ -120,7 +121,7 @@ public UserGroupInformation run() throws IOException { return UserGroupInformation.getCurrentUser(); } }); - Assert.assertEquals( + Assertions.assertEquals( PROXY_USER_NAME + " (auth:PROXY) via " + REAL_USER_NAME + " (auth:SIMPLE)", curUGI.toString()); } @@ -136,14 +137,15 @@ public Void run() throws ServiceException { String serverRemoteUser = client.getServerRemoteUser(null, newEmptyRequest()).getUser(); - Assert.assertEquals(ugi.toString(), currentUser); - Assert.assertEquals(ugi.toString(), serverRemoteUser); + Assertions.assertEquals(ugi.toString(), currentUser); + Assertions.assertEquals(ugi.toString(), serverRemoteUser); return null; } }); } - @Test(timeout=4000) + @Test + @Timeout(value = 4) public void testRealUserSetup() throws IOException { final Configuration conf = new Configuration(); conf.setStrings(DefaultImpersonationProvider.getTestProvider(). @@ -167,13 +169,14 @@ public void testRealUserSetup() throws IOException { checkRemoteUgi(proxyUserUgi, conf); } catch (Exception e) { e.printStackTrace(); - Assert.fail(); + Assertions.fail(); } finally { stop(server, client); } } - @Test(timeout=4000) + @Test + @Timeout(value = 4) public void testRealUserAuthorizationSuccess() throws IOException { final Configuration conf = new Configuration(); configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME); @@ -196,7 +199,7 @@ public void testRealUserAuthorizationSuccess() throws IOException { checkRemoteUgi(proxyUserUgi, conf); } catch (Exception e) { e.printStackTrace(); - Assert.fail(); + Assertions.fail(); } finally { stop(server, client); } @@ -237,7 +240,7 @@ public String run() throws ServiceException { } }); - Assert.fail("The RPC must have failed " + retVal); + Assertions.fail("The RPC must have failed " + retVal); } catch (Exception e) { e.printStackTrace(); } finally { @@ -273,7 +276,7 @@ public String run() throws ServiceException { } }); - Assert.fail("The RPC must have failed " + retVal); + Assertions.fail("The RPC must have failed " + retVal); } catch (Exception e) { e.printStackTrace(); } finally { @@ -306,7 +309,7 @@ public String run() throws ServiceException { } }); - Assert.fail("The RPC must have failed " + retVal); + Assertions.fail("The RPC must have failed " + retVal); } catch (Exception e) { e.printStackTrace(); } finally { @@ -344,7 +347,7 @@ public String run() throws ServiceException { } }); - Assert.fail("The RPC must have failed " + retVal); + Assertions.fail("The RPC must have failed " + retVal); } catch (Exception e) { e.printStackTrace(); } finally { @@ -397,7 +400,7 @@ public String run() throws Exception { } }); //The user returned by server must be the one in the token. - Assert.assertEquals(REAL_USER_NAME + " (auth:TOKEN) via SomeSuperUser (auth:SIMPLE)", retVal); + Assertions.assertEquals(REAL_USER_NAME + " (auth:TOKEN) via SomeSuperUser (auth:SIMPLE)", retVal); } /* @@ -441,7 +444,7 @@ public String run() throws Exception { } }); String expected = REAL_USER_NAME + " (auth:TOKEN) via SomeSuperUser (auth:SIMPLE)"; - Assert.assertEquals(retVal + "!=" + expected, expected, retVal); + Assertions.assertEquals(expected, retVal, retVal + "!=" + expected); } // diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java index cbea393d93164..234037bd54ce2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java @@ -18,8 +18,8 @@ package org.apache.hadoop.security; import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; import java.security.PrivilegedExceptionAction; @@ -37,8 +37,8 @@ import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Testcase for HADOOP-13433 that verifies the logic of fixKerberosTicketOrder. @@ -63,7 +63,7 @@ public class TestFixKerberosTicketOrder extends KerberosSecurityTestcase { private Map props; - @Before + @BeforeEach public void setUp() throws Exception { keytabFile = new File(getWorkDir(), "keytab"); getKdc().createPrincipal(keytabFile, clientPrincipal, server1Principal, @@ -106,13 +106,13 @@ public Void run() throws Exception { } // make sure the first ticket is not tgt assertFalse( - "The first ticket is still tgt, " - + "the implementation in jdk may have been changed, " - + "please reconsider the problem in HADOOP-13433", - subject.getPrivateCredentials().stream() + + subject.getPrivateCredentials().stream() .filter(c -> c instanceof KerberosTicket) .map(c -> ((KerberosTicket) c).getServer().getName()).findFirst() - .get().startsWith("krbtgt")); + .get().startsWith("krbtgt"), "The first ticket is still tgt, " + + "the implementation in jdk may have been changed, " + + "please reconsider the problem in HADOOP-13433"); // should fail as we send a service ticket instead of tgt to KDC. intercept(SaslException.class, () -> ugi.doAs(new PrivilegedExceptionAction() { @@ -131,11 +131,11 @@ public Void run() throws Exception { ugi.fixKerberosTicketOrder(); // check if TGT is the first ticket after the fix. - assertTrue("The first ticket is not tgt", - subject.getPrivateCredentials().stream() + assertTrue( + subject.getPrivateCredentials().stream() .filter(c -> c instanceof KerberosTicket) .map(c -> ((KerberosTicket) c).getServer().getName()).findFirst() - .get().startsWith("krbtgt")); + .get().startsWith("krbtgt"), "The first ticket is not tgt"); // make sure we can still get new service ticket after the fix. ugi.doAs(new PrivilegedExceptionAction() { @@ -150,10 +150,10 @@ public Void run() throws Exception { return null; } }); - assertTrue("No service ticket for " + server2Protocol + " found", - subject.getPrivateCredentials(KerberosTicket.class).stream() + assertTrue( + subject.getPrivateCredentials(KerberosTicket.class).stream() .filter(t -> t.getServer().getName().startsWith(server2Protocol)) - .findAny().isPresent()); + .findAny().isPresent(), "No service ticket for " + server2Protocol + " found"); } @Test @@ -188,11 +188,11 @@ public Void run() throws Exception { ugi.fixKerberosTicketOrder(); // verify that after fixing, the tgt ticket should be removed - assertFalse("The first ticket is not tgt", - subject.getPrivateCredentials().stream() + assertFalse( + subject.getPrivateCredentials().stream() .filter(c -> c instanceof KerberosTicket) .map(c -> ((KerberosTicket) c).getServer().getName()).findFirst() - .isPresent()); + .isPresent(), "The first ticket is not tgt"); // should fail as we send a service ticket instead of tgt to KDC. @@ -227,9 +227,9 @@ public Void run() throws Exception { } }); - assertTrue("No service ticket for " + server2Protocol + " found", - subject.getPrivateCredentials(KerberosTicket.class).stream() + assertTrue( + subject.getPrivateCredentials(KerberosTicket.class).stream() .filter(t -> t.getServer().getName().startsWith(server2Protocol)) - .findAny().isPresent()); + .findAny().isPresent(), "No service ticket for " + server2Protocol + " found"); } } \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupFallback.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupFallback.java index 3ef3698495173..fdafc03d81260 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupFallback.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupFallback.java @@ -17,14 +17,14 @@ */ package org.apache.hadoop.security; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java index 87788691f6d1b..be6c136df7f22 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java @@ -31,15 +31,15 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.FakeTimer; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.function.Supplier; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.fail; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; @@ -54,7 +54,7 @@ public class TestGroupsCaching { private static String[] myGroups = {"grp1", "grp2"}; private Configuration conf; - @Before + @BeforeEach public void setup() throws IOException { FakeGroupMapping.clearAll(); ExceptionalGroupMapping.resetRequestCount(); @@ -271,25 +271,25 @@ public void testGroupLookupForStaticUsers() throws Exception { conf.set(CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES, "me=;user1=group1;user2=group1,group2"); Groups groups = new Groups(conf); List userGroups = groups.getGroups("me"); - assertTrue("non-empty groups for static user", userGroups.isEmpty()); - assertFalse("group lookup done for static user", - FakeunPrivilegedGroupMapping.invoked); + assertTrue(userGroups.isEmpty(), "non-empty groups for static user"); + assertFalse( + FakeunPrivilegedGroupMapping.invoked, "group lookup done for static user"); List expected = new ArrayList(); expected.add("group1"); FakeunPrivilegedGroupMapping.invoked = false; userGroups = groups.getGroups("user1"); - assertTrue("groups not correct", expected.equals(userGroups)); - assertFalse("group lookup done for unprivileged user", - FakeunPrivilegedGroupMapping.invoked); + assertTrue(expected.equals(userGroups), "groups not correct"); + assertFalse( + FakeunPrivilegedGroupMapping.invoked, "group lookup done for unprivileged user"); expected.add("group2"); FakeunPrivilegedGroupMapping.invoked = false; userGroups = groups.getGroups("user2"); - assertTrue("groups not correct", expected.equals(userGroups)); - assertFalse("group lookup done for unprivileged user", - FakeunPrivilegedGroupMapping.invoked); + assertTrue(expected.equals(userGroups), "groups not correct"); + assertFalse( + FakeunPrivilegedGroupMapping.invoked, "group lookup done for unprivileged user"); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestHttpCrossOriginFilterInitializer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestHttpCrossOriginFilterInitializer.java index 3db6ef1b1e811..e05b0acd6682a 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestHttpCrossOriginFilterInitializer.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestHttpCrossOriginFilterInitializer.java @@ -22,8 +22,8 @@ import org.apache.hadoop.conf.Configuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestHttpCrossOriginFilterInitializer { @@ -48,11 +48,11 @@ public void testGetFilterParameters() { String outofscopeparam = filterParameters.get("outofscopeparam"); // verify expected values are in place - Assert.assertEquals("Could not find filter parameter", "rootvalue", + Assertions.assertEquals("Could not find filter parameter", "rootvalue", rootvalue); - Assert.assertEquals("Could not find filter parameter", "nestedvalue", + Assertions.assertEquals("Could not find filter parameter", "nestedvalue", nestedvalue); - Assert.assertNull("Found unexpected value in filter parameters", - outofscopeparam); + Assertions.assertNull( + outofscopeparam, "Found unexpected value in filter parameters"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestIngressPortBasedResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestIngressPortBasedResolver.java index 96c80af15f34b..1a67a69b84ce6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestIngressPortBasedResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestIngressPortBasedResolver.java @@ -19,9 +19,9 @@ import javax.security.sasl.Sasl; import org.apache.hadoop.conf.Configuration; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestJNIGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestJNIGroupsMapping.java index 99c5c2a83f28c..8e20aa2e2b128 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestJNIGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestJNIGroupsMapping.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.security; import static org.junit.Assume.assumeTrue; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import java.util.Arrays; import java.util.List; @@ -26,14 +26,14 @@ import org.apache.hadoop.security.ShellBasedUnixGroupsMapping; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.NativeCodeLoader; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestJNIGroupsMapping { - @Before + @BeforeEach public void isNativeCodeLoaded() { assumeTrue(NativeCodeLoader.isNativeCodeLoaded()); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java index 79dcd1afc5313..fdee97ba794a0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java @@ -22,12 +22,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.minikdc.MiniKdc; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.TestName; import org.junit.rules.Timeout; import org.slf4j.Logger; @@ -43,7 +43,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; import static org.apache.hadoop.security.KDiag.*; -public class TestKDiag extends Assert { +public class TestKDiag extends Assertions { private static final Logger LOG = LoggerFactory.getLogger(TestKDiag.class); public static final String KEYLEN = "128"; @@ -56,7 +56,7 @@ public class TestKDiag extends Assert { @Rule public Timeout testTimeout = new Timeout(30000, TimeUnit.MILLISECONDS); - @BeforeClass + @BeforeAll public static void nameThread() { Thread.currentThread().setName("JUnit"); } @@ -67,7 +67,7 @@ public static void nameThread() { private static Properties securityProperties; private static Configuration conf; - @BeforeClass + @BeforeAll public static void startMiniKdc() throws Exception { workDir = GenericTestUtils.getTestDir(TestKDiag.class.getSimpleName()); securityProperties = MiniKdc.createConf(); @@ -78,7 +78,7 @@ public static void startMiniKdc() throws Exception { conf.set(HADOOP_SECURITY_AUTHENTICATION, "KERBEROS"); } - @AfterClass + @AfterAll public static synchronized void stopMiniKdc() { if (kdc != null) { kdc.stop(); @@ -86,7 +86,7 @@ public static synchronized void stopMiniKdc() { } } - @Before + @BeforeEach public void reset() { UserGroupInformation.reset(); } @@ -155,7 +155,7 @@ public void testKeytabNoPrincipal() throws Throwable { @Test public void testConfIsSecure() throws Throwable { - Assert.assertFalse(SecurityUtil.getAuthenticationMethod(conf) + Assertions.assertFalse(SecurityUtil.getAuthenticationMethod(conf) .equals(UserGroupInformation.AuthenticationMethod.SIMPLE)); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java index 03d953b5f3cc3..b28c8292df7b8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java @@ -20,11 +20,11 @@ import org.apache.hadoop.conf.Configuration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.TestName; import org.junit.rules.Timeout; import org.slf4j.Logger; @@ -41,7 +41,7 @@ import static org.apache.hadoop.security.KDiag.KerberosDiagsFailure; import static org.apache.hadoop.security.KDiag.exec; -public class TestKDiagNoKDC extends Assert { +public class TestKDiagNoKDC extends Assertions { private static final Logger LOG = LoggerFactory.getLogger(TestKDiagNoKDC.class); public static final String KEYLEN = "128"; @@ -52,7 +52,7 @@ public class TestKDiagNoKDC extends Assert { @Rule public Timeout testTimeout = new Timeout(30000, TimeUnit.MILLISECONDS); - @BeforeClass + @BeforeAll public static void nameThread() { Thread.currentThread().setName("JUnit"); } @@ -60,7 +60,7 @@ public static void nameThread() { private static Configuration conf = new Configuration(); - @Before + @BeforeEach public void reset() { UserGroupInformation.reset(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java index 82e80fd9fa504..32d76a774b92c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java @@ -22,10 +22,10 @@ import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_NUM_ATTEMPTS_KEY; import static org.apache.hadoop.security.LdapGroupsMapping.READ_TIMEOUT; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; @@ -59,9 +59,10 @@ import org.apache.hadoop.security.alias.JavaKeyStoreProvider; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import org.slf4j.Logger; @@ -90,7 +91,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase { private static final String TEST_LDAP_URL = "ldap://test"; - @Before + @BeforeEach public void setupMocks() { when(getUserSearchResult().getNameInNamespace()). thenReturn(userDN); @@ -165,7 +166,7 @@ public void testGetGroupsWithDynamicGroupFilter() throws Exception { // Check the group filter got resolved and get the desired values. List groups = groupsMapping.getGroups(userName); - Assert.assertEquals(Arrays.asList(getTestGroups()), groups); + Assertions.assertEquals(Arrays.asList(getTestGroups()), groups); } /** @@ -190,7 +191,7 @@ private void doTestGetGroupsWithBaseDN(Configuration conf, String userBaseDN, .thenReturn(getUserNames(), getGroupNames()); List groups = groupsMapping.getGroups(userName); - Assert.assertEquals(Arrays.asList(getTestGroups()), groups); + Assertions.assertEquals(Arrays.asList(getTestGroups()), groups); // We should have searched for the username and groups with default base dn verify(getContext(), times(1)).search(userBaseDN, @@ -258,7 +259,7 @@ private void doTestGetGroups(List expectedGroups, int searchTimes) // regardless of input List groups = groupsMapping.getGroups("some_user"); - Assert.assertEquals(expectedGroups, groups); + Assertions.assertEquals(expectedGroups, groups); // We should have searched for a user, and then two groups verify(getContext(), times(searchTimes)).search(anyString(), @@ -278,7 +279,7 @@ private void doTestGetGroupsWithParent(List expectedGroups, List groups = groupsMapping.getGroups("some_user"); // compare lists, ignoring the order - Assert.assertEquals(new HashSet<>(expectedGroups), new HashSet<>(groups)); + Assertions.assertEquals(new HashSet<>(expectedGroups), new HashSet<>(groups)); // We should have searched for a user, and group verify(getContext(), times(searchTimesGroup)).search(anyString(), @@ -298,7 +299,7 @@ public void testExtractPassword() throws IOException { writer.close(); LdapGroupsMapping mapping = new LdapGroupsMapping(); - Assert.assertEquals("hadoop", + Assertions.assertEquals("hadoop", mapping.extractPassword(secretFile.getPath())); } @@ -344,15 +345,15 @@ public void testConfGetPassword() throws Exception { LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY).getCredential()); LdapGroupsMapping mapping = new LdapGroupsMapping(); - Assert.assertEquals("bindpass", + Assertions.assertEquals("bindpass", mapping.getPassword(conf, LdapGroupsMapping.BIND_PASSWORD_KEY, "")); - Assert.assertEquals("storepass", + Assertions.assertEquals("storepass", mapping.getPassword(conf, LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY, "")); // let's make sure that a password that doesn't exist returns an // empty string as currently expected and used to trigger a call to // extract password - Assert.assertEquals("", mapping.getPassword(conf,"invalid-alias", "")); + Assertions.assertEquals("", mapping.getPassword(conf,"invalid-alias", "")); } @Test @@ -387,11 +388,11 @@ public void testConfGetPasswordUsingAlias() throws Exception { bindpassAlias).getCredential()); LdapGroupsMapping mapping = new LdapGroupsMapping(); - Assert.assertEquals("bindpass", + Assertions.assertEquals("bindpass", mapping.getPasswordFromCredentialProviders(conf, bindpassAlias, "")); // Empty for an invalid alias - Assert.assertEquals("", mapping.getPasswordFromCredentialProviders( + Assertions.assertEquals("", mapping.getPasswordFromCredentialProviders( conf, "invalid-alias", "")); } @@ -402,7 +403,8 @@ public void testConfGetPasswordUsingAlias() throws Exception { * @throws IOException * @throws InterruptedException */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testLdapConnectionTimeout() throws IOException, InterruptedException { final int connectionTimeoutMs = 3 * 1000; // 3s @@ -456,7 +458,8 @@ public void run() { * @throws IOException * @throws InterruptedException */ - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testLdapReadTimeout() throws IOException, InterruptedException { final int readTimeoutMs = 4 * 1000; // 4s try (ServerSocket serverSock = new ServerSocket(0)) { @@ -511,7 +514,8 @@ public void run() { * * @throws Exception */ - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testSetConf() throws Exception { Configuration conf = getBaseConf(TEST_LDAP_URL); Configuration mockConf = Mockito.spy(conf); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java index bd396ddb828c3..39de02d078bf2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java @@ -21,7 +21,7 @@ import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_CTX_FACTORY_CLASS_DEFAULT; import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_CTX_FACTORY_CLASS_KEY; import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_URL_KEY; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -39,7 +39,7 @@ import javax.naming.spi.InitialContextFactory; import org.apache.hadoop.conf.Configuration; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -67,7 +67,7 @@ public class TestLdapGroupsMappingBase { private String[] testParentGroups = new String[] {"group1", "group2", "group1_1"}; - @Before + @BeforeEach public void setupMocksBase() throws NamingException { DummyLdapCtxFactory.reset(); MockitoAnnotations.initMocks(this); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithBindUserSwitch.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithBindUserSwitch.java index 3b4c77d9e4ab9..852d7745a6055 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithBindUserSwitch.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithBindUserSwitch.java @@ -25,7 +25,7 @@ import org.apache.hadoop.security.alias.CredentialProviderFactory; import org.apache.hadoop.security.alias.JavaKeyStoreProvider; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import javax.naming.AuthenticationException; import javax.naming.NamingException; @@ -44,10 +44,10 @@ import static org.apache.hadoop.security.LdapGroupsMapping.BIND_USERS_KEY; import static org.apache.hadoop.security.LdapGroupsMapping.BIND_USER_SUFFIX; import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_NUM_ATTEMPTS_KEY; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -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.assertNull; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.times; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithFailover.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithFailover.java index 2f14c735ab6ba..44378d2b993ce 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithFailover.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithFailover.java @@ -19,7 +19,7 @@ package org.apache.hadoop.security; import org.apache.hadoop.conf.Configuration; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -33,7 +33,7 @@ import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_KEY; import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_NUM_ATTEMPTS_KEY; import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_URL_KEY; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.times; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java index 8686d5c6e3b46..e7fe020341a82 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java @@ -31,8 +31,8 @@ import javax.naming.directory.SearchResult; import org.apache.hadoop.conf.Configuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.mockito.stubbing.Stubber; import static org.mockito.ArgumentMatchers.any; @@ -114,9 +114,9 @@ private void doTestGetGroups(List expectedGroups) // regardless of input List groups = groupsMapping.getGroups("some_user"); - Assert.assertEquals(expectedGroups, groups); - Assert.assertFalse("Second LDAP query should NOT have been called.", - groupsMapping.isSecondaryQueryCalled()); + Assertions.assertEquals(expectedGroups, groups); + Assertions.assertFalse( + groupsMapping.isSecondaryQueryCalled(), "Second LDAP query should NOT have been called."); // We should have only made one query because single-query lookup is enabled verify(getContext(), times(1)).search(anyString(), anyString(), @@ -144,11 +144,11 @@ private void doTestGetGroupsWithFallback() List groups = groupsMapping.getGroups("some_user"); // expected to be empty due to invalid memberOf - Assert.assertEquals(0, groups.size()); + Assertions.assertEquals(0, groups.size()); // expect secondary query to be called: getGroups() - Assert.assertTrue("Second LDAP query should have been called.", - groupsMapping.isSecondaryQueryCalled()); + Assertions.assertTrue( + groupsMapping.isSecondaryQueryCalled(), "Second LDAP query should have been called."); // We should have fallen back to the second query because first threw // NamingException expected count is 3 since testGetGroups calls diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java index 7ecc636984ca0..345f7df08b414 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java @@ -37,15 +37,15 @@ import javax.naming.directory.SearchControls; import org.apache.hadoop.conf.Configuration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; @SuppressWarnings("unchecked") public class TestLdapGroupsMappingWithPosixGroup extends TestLdapGroupsMappingBase { - @Before + @BeforeEach public void setupMocks() throws NamingException { Attribute uidNumberAttr = mock(Attribute.class); Attribute gidNumberAttr = mock(Attribute.class); @@ -90,11 +90,11 @@ private void doTestGetGroups(List expectedGroups, int searchTimes) // regardless of input List groups = groupsMapping.getGroups("some_user"); - Assert.assertEquals(expectedGroups, groups); + Assertions.assertEquals(expectedGroups, groups); groupsMapping.getConf().set(LdapGroupsMapping.POSIX_UID_ATTR_KEY, "uid"); - Assert.assertEquals(expectedGroups, groups); + Assertions.assertEquals(expectedGroups, groups); // We should have searched for a user, and then two groups verify(getContext(), times(searchTimes)).search(anyString(), diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNetgroupCache.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNetgroupCache.java index bd95422e651d9..1d6c4bb8b0912 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNetgroupCache.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNetgroupCache.java @@ -16,14 +16,14 @@ */ package org.apache.hadoop.security; -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.ArrayList; import java.util.List; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; public class TestNetgroupCache { @@ -33,7 +33,7 @@ public class TestNetgroupCache { private static final String GROUP1 = "group1"; private static final String GROUP2 = "group2"; - @After + @AfterEach public void teardown() { NetgroupCache.clear(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNullGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNullGroupsMapping.java index 4b94e51ee1da8..a03ff36a032f1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNullGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNullGroupsMapping.java @@ -18,9 +18,9 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import org.junit.Before; -import org.junit.Test; -import static org.junit.Assert.*; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.*; /** * Test that the {@link NullGroupsMapping} really does nothing. @@ -28,7 +28,7 @@ public class TestNullGroupsMapping { private NullGroupsMapping ngm; - @Before + @BeforeEach public void setUp() { this.ngm = new NullGroupsMapping(); } @@ -42,19 +42,19 @@ public void testGetGroups() { List expResult = Collections.emptyList(); List result = ngm.getGroups(user); - assertEquals("No groups should be returned", - expResult, result); + assertEquals( + expResult, result, "No groups should be returned"); ngm.cacheGroupsAdd(Arrays.asList(new String[] {"group1", "group2"})); result = ngm.getGroups(user); - assertEquals("No groups should be returned", - expResult, result); + assertEquals( + expResult, result, "No groups should be returned"); ngm.cacheGroupsRefresh(); result = ngm.getGroups(user); - assertEquals("No groups should be returned", - expResult, result); + assertEquals( + expResult, result, "No groups should be returned"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestProxyUserFromEnv.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestProxyUserFromEnv.java index 65756fcdd3944..67e2868a7146f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestProxyUserFromEnv.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestProxyUserFromEnv.java @@ -17,14 +17,14 @@ package org.apache.hadoop.security; -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.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestProxyUserFromEnv { /** Test HADOOP_PROXY_USER for impersonation */ diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRaceWhenRelogin.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRaceWhenRelogin.java index 4f9946c3e27ba..473afcc6c44a7 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRaceWhenRelogin.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRaceWhenRelogin.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.security; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; import java.io.IOException; @@ -37,8 +37,8 @@ import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Testcase for HADOOP-13433 that confirms that tgt will always be the first @@ -68,7 +68,7 @@ public class TestRaceWhenRelogin extends KerberosSecurityTestcase { private UserGroupInformation ugi; - @Before + @BeforeEach public void setUp() throws Exception { keytabFile = new File(getWorkDir(), "keytab"); serverProtocols = new String[numThreads]; @@ -157,6 +157,6 @@ public void test() throws InterruptedException, IOException { for (Thread getServiceTicketThread : getServiceTicketThreads) { getServiceTicketThread.join(); } - assertTrue("tgt is not the first ticket after relogin", pass.get()); + assertTrue(pass.get(), "tgt is not the first ticket after relogin"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java index 8862fd7b60984..23c8197bac528 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java @@ -18,8 +18,8 @@ package org.apache.hadoop.security; import org.apache.hadoop.conf.Configuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import javax.naming.NamingException; @@ -56,7 +56,7 @@ public void testGetGroupsToUpper() throws NamingException { List groupsUpper = new ArrayList<>(); groupsUpper.add("GROUP1"); groupsUpper.add("GROUP2"); - Assert.assertEquals(groupsUpper, groupsMapping.getGroups("admin")); + Assertions.assertEquals(groupsUpper, groupsMapping.getGroups("admin")); } @Test @@ -77,7 +77,7 @@ public void testGetGroupsToLower() throws NamingException { List groupsLower = new ArrayList<>(); groupsLower.add("group1"); groupsLower.add("group2"); - Assert.assertEquals(groupsLower, groupsMapping.getGroups("admin")); + Assertions.assertEquals(groupsLower, groupsMapping.getGroups("admin")); } @Test @@ -95,7 +95,7 @@ public void testGetGroupsInvalidRule() throws NamingException { conf.set(CONVERSION_RULE_KEY, "none"); groupsMapping.setConf(conf); - Assert.assertEquals(groups, groupsMapping.getGroupsSet("admin")); + Assertions.assertEquals(groups, groupsMapping.getGroupsSet("admin")); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java index b6b9684445342..3b43459786dca 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java @@ -19,7 +19,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.*; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import java.io.File; import java.io.IOException; @@ -42,8 +42,8 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.ZKUtil.ZKAuthInfo; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.apache.hadoop.thirdparty.com.google.common.io.Files; @@ -52,7 +52,7 @@ public class TestSecurityUtil { private static final String ZK_AUTH_VALUE = "a_scheme:a_password"; - @BeforeClass + @BeforeAll public static void unsetKerberosRealm() { // prevent failures if kinit-ed or on os x with no realm System.setProperty("java.security.krb5.kdc", ""); @@ -145,7 +145,7 @@ public void testStartsWithIncorrectSettings() throws IOException { // expected gotException=true; } - assertTrue("Exception for empty keytabfile name was expected", gotException); + assertTrue(gotException, "Exception for empty keytabfile name was expected"); } @Test @@ -238,7 +238,7 @@ void runBadPortPermutes(String arg, boolean validIfPosPort) { } catch (IllegalArgumentException e) { bad = true; } finally { - assertTrue("should be bad: '"+arg+"'", bad); + assertTrue(bad, "should be bad: '"+arg+"'"); } for (int port : ports) { if (validIfPosPort && port > 0) continue; @@ -249,7 +249,7 @@ void runBadPortPermutes(String arg, boolean validIfPosPort) { } catch (IllegalArgumentException e) { bad = true; } finally { - assertTrue("should be bad: '"+arg+"' (default port:"+port+")", bad); + assertTrue(bad, "should be bad: '"+arg+"' (default port:"+port+")"); } } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java index 939209d267b50..cbd1f4495b8d1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java @@ -18,8 +18,8 @@ package org.apache.hadoop.security; import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows; -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.File; import java.io.FileOutputStream; @@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.ShellBasedIdMapping.PassThroughMap; import org.apache.hadoop.security.ShellBasedIdMapping.StaticMapping; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.thirdparty.com.google.common.collect.BiMap; import org.apache.hadoop.thirdparty.com.google.common.collect.HashBiMap; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java index 8c1339d38d58e..360bfece5fbac 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java @@ -27,11 +27,12 @@ import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell.ExitCodeException; import org.apache.hadoop.util.Shell.ShellCommandExecutor; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -190,18 +191,18 @@ public void testShellTimeOutConf() { // Test a 1 second max-runtime timeout assertEquals( - "Expected the group names executor to carry the configured timeout", - 1000L, getTimeoutInterval("1s")); + + 1000L, getTimeoutInterval("1s"), "Expected the group names executor to carry the configured timeout"); // Test a 1 minute max-runtime timeout assertEquals( - "Expected the group names executor to carry the configured timeout", - 60000L, getTimeoutInterval("1m")); + + 60000L, getTimeoutInterval("1m"), "Expected the group names executor to carry the configured timeout"); // Test a 1 millisecond max-runtime timeout assertEquals( - "Expected the group names executor to carry the configured timeout", - 1L, getTimeoutInterval("1")); + + 1L, getTimeoutInterval("1"), "Expected the group names executor to carry the configured timeout"); } private class TestGroupResolvable @@ -274,7 +275,8 @@ protected String[] getGroupsIDForUserCommand(String userName) { } } - @Test(timeout=4000) + @Test + @Timeout(value = 4) public void testFiniteGroupResolutionTime() throws Exception { Configuration conf = new Configuration(); String userName = "foobarnonexistinguser"; @@ -293,25 +295,25 @@ public void testFiniteGroupResolutionTime() throws Exception { ShellCommandExecutor executor = mapping.createGroupExecutor(userName); assertEquals( - "Expected the group names executor to carry the configured timeout", - testTimeout, - executor.getTimeoutInterval()); + + testTimeout +, executor.getTimeoutInterval(), "Expected the group names executor to carry the configured timeout"); executor = mapping.createGroupIDExecutor(userName); assertEquals( - "Expected the group ID executor to carry the configured timeout", - testTimeout, - executor.getTimeoutInterval()); + + testTimeout +, executor.getTimeoutInterval(), "Expected the group ID executor to carry the configured timeout"); assertEquals( - "Expected no groups to be returned given a shell command timeout", - 0, - mapping.getGroups(userName).size()); + + 0 +, mapping.getGroups(userName).size(), "Expected no groups to be returned given a shell command timeout"); assertTrue( - "Expected the logs to carry " + + + shellMappingLog.getOutput().contains(commandTimeoutMessage), "Expected the logs to carry " + "a message about command timeout but was: " + - shellMappingLog.getOutput(), - shellMappingLog.getOutput().contains(commandTimeoutMessage)); + shellMappingLog.getOutput()); shellMappingLog.clearOutput(); // Test also the parent Groups framework for expected behaviour @@ -326,10 +328,10 @@ public void testFiniteGroupResolutionTime() throws Exception { "have failed with a command timeout"); } catch (IOException e) { assertTrue( - "Expected the logs to carry " + + + shellMappingLog.getOutput().contains(commandTimeoutMessage), "Expected the logs to carry " + "a message about command timeout but was: " + - shellMappingLog.getOutput(), - shellMappingLog.getOutput().contains(commandTimeoutMessage)); + shellMappingLog.getOutput()); } shellMappingLog.clearOutput(); @@ -344,21 +346,21 @@ public void testFiniteGroupResolutionTime() throws Exception { executor = mapping.createGroupExecutor(userName); assertEquals( - "Expected the group names executor to carry the default timeout", - defaultTimeout, - executor.getTimeoutInterval()); + + defaultTimeout +, executor.getTimeoutInterval(), "Expected the group names executor to carry the default timeout"); executor = mapping.createGroupIDExecutor(userName); assertEquals( - "Expected the group ID executor to carry the default timeout", - defaultTimeout, - executor.getTimeoutInterval()); + + defaultTimeout +, executor.getTimeoutInterval(), "Expected the group ID executor to carry the default timeout"); mapping.getGroups(userName); assertFalse( - "Didn't expect a timeout of command in execution but logs carry it: " + - shellMappingLog.getOutput(), - shellMappingLog.getOutput().contains(commandTimeoutMessage)); + + shellMappingLog.getOutput().contains(commandTimeoutMessage), "Didn't expect a timeout of command in execution but logs carry it: " + + shellMappingLog.getOutput()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java index db0095f2171e2..12fcfc5118d93 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java @@ -24,11 +24,12 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Time; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.rules.TemporaryFolder; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -37,10 +38,10 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN; -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.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.io.File; import java.io.IOException; @@ -76,7 +77,7 @@ public class TestUGILoginFromKeytab { @Rule public final TemporaryFolder folder = new TemporaryFolder(); - @Before + @BeforeEach public void startMiniKdc() throws Exception { // This setting below is required. If not enabled, UGI will abort // any attempt to loginUserFromKeytab. @@ -92,7 +93,7 @@ public void startMiniKdc() throws Exception { } - @After + @AfterEach public void stopMiniKdc() { if (kdc != null) { kdc.stop(); @@ -114,15 +115,15 @@ public void testUGILoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assert.assertTrue("UGI should be configured to login from keytab", - ugi.isFromKeytab()); + Assertions.assertTrue( + ugi.isFromKeytab(), "UGI should be configured to login from keytab"); User user = getUser(ugi.getSubject()); - Assert.assertNotNull(user.getLogin()); + Assertions.assertNotNull(user.getLogin()); - Assert.assertTrue("User login time is less than before login time, " - + "beforeLoginTime:" + beforeLogin + " userLoginTime:" + user.getLastLogin(), - user.getLastLogin() > beforeLogin); + Assertions.assertTrue( + user.getLastLogin() > beforeLogin, "User login time is less than before login time, " + + "beforeLoginTime:" + beforeLogin + " userLoginTime:" + user.getLastLogin()); } /** @@ -137,14 +138,14 @@ public void testUGIReLoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assert.assertTrue("UGI should be configured to login from keytab", - ugi.isFromKeytab()); + Assertions.assertTrue( + ugi.isFromKeytab(), "UGI should be configured to login from keytab"); // Verify relogin from keytab. User user = getUser(ugi.getSubject()); final long firstLogin = user.getLastLogin(); final LoginContext login1 = user.getLogin(); - Assert.assertNotNull(login1); + Assertions.assertNotNull(login1); // Sleep for 2 secs to have a difference between first and second login Thread.sleep(2000); @@ -152,10 +153,10 @@ public void testUGIReLoginFromKeytab() throws Exception { ugi.reloginFromKeytab(); final long secondLogin = user.getLastLogin(); final LoginContext login2 = user.getLogin(); - Assert.assertTrue("User should have been able to relogin from keytab", - secondLogin > firstLogin); - Assert.assertNotNull(login2); - Assert.assertNotSame(login1, login2); + Assertions.assertTrue( + secondLogin > firstLogin, "User should have been able to relogin from keytab"); + Assertions.assertNotNull(login2); + Assertions.assertNotSame(login1, login2); } /** @@ -172,14 +173,14 @@ public void testUGIForceReLoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assert.assertTrue("UGI should be configured to login from keytab", - ugi.isFromKeytab()); + Assertions.assertTrue( + ugi.isFromKeytab(), "UGI should be configured to login from keytab"); // Verify relogin from keytab. User user = getUser(ugi.getSubject()); final long firstLogin = user.getLastLogin(); final LoginContext login1 = user.getLogin(); - Assert.assertNotNull(login1); + Assertions.assertNotNull(login1); // Sleep for 2 secs to have a difference between first and second login Thread.sleep(2000); @@ -188,10 +189,10 @@ public void testUGIForceReLoginFromKeytab() throws Exception { ugi.forceReloginFromKeytab(); final long secondLogin = user.getLastLogin(); final LoginContext login2 = user.getLogin(); - Assert.assertTrue("User should have been able to relogin from keytab", - secondLogin > firstLogin); - Assert.assertNotNull(login2); - Assert.assertNotSame(login1, login2); + Assertions.assertTrue( + secondLogin > firstLogin, "User should have been able to relogin from keytab"); + Assertions.assertNotNull(login2); + Assertions.assertNotSame(login1, login2); } @Test @@ -205,14 +206,14 @@ public void testGetUGIFromKnownSubject() throws Exception { principal.getName(), keytab.getPath()); Subject subject = ugi1.getSubject(); User user = getUser(subject); - Assert.assertNotNull(user); + Assertions.assertNotNull(user); LoginContext login = user.getLogin(); - Assert.assertNotNull(login); + Assertions.assertNotNull(login); // User instance and/or login context should not change. UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject); - Assert.assertSame(user, getUser(ugi2.getSubject())); - Assert.assertSame(login, user.getLogin()); + Assertions.assertSame(user, getUser(ugi2.getSubject())); + Assertions.assertSame(login, user.getLogin()); } @Test @@ -230,17 +231,17 @@ public void testGetUGIFromExternalSubject() throws Exception { // first call to get the ugi should add the User instance w/o a login // context. UserGroupInformation ugi1 = UserGroupInformation.getUGIFromSubject(subject); - Assert.assertSame(subject, ugi1.getSubject()); + Assertions.assertSame(subject, ugi1.getSubject()); User user = getUser(subject); - Assert.assertNotNull(user); - Assert.assertEquals(principal.getName(), user.getName()); - Assert.assertNull(user.getLogin()); + Assertions.assertNotNull(user); + Assertions.assertEquals(principal.getName(), user.getName()); + Assertions.assertNull(user.getLogin()); // subsequent call should not change the existing User instance. UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject); - Assert.assertSame(subject, ugi2.getSubject()); - Assert.assertSame(user, getUser(ugi2.getSubject())); - Assert.assertNull(user.getLogin()); + Assertions.assertSame(subject, ugi2.getSubject()); + Assertions.assertSame(user, getUser(ugi2.getSubject())); + Assertions.assertNull(user.getLogin()); } @Test @@ -261,9 +262,9 @@ public void testGetUGIFromExternalSubjectWithLogin() throws Exception { // nothing should change. UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject); - Assert.assertSame(subject, ugi2.getSubject()); - Assert.assertSame(user, getUser(ugi2.getSubject())); - Assert.assertSame(dummyLogin, user.getLogin()); + Assertions.assertSame(subject, ugi2.getSubject()); + Assertions.assertSame(user, getUser(ugi2.getSubject())); + Assertions.assertSame(dummyLogin, user.getLogin()); } @Test @@ -282,12 +283,12 @@ public void testUGIRefreshFromKeytab() throws Exception { UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assert.assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, + Assertions.assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); - Assert.assertTrue(ugi.isFromKeytab()); - Assert.assertTrue( + Assertions.assertTrue(ugi.isFromKeytab()); + Assertions.assertTrue( UserGroupInformation.isKerberosKeyTabLoginRenewalEnabled()); - Assert.assertTrue( + Assertions.assertTrue( UserGroupInformation.getKerberosLoginRenewalExecutor() .isPresent()); } @@ -309,12 +310,12 @@ public void testUGIRefreshFromKeytabDisabled() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assert.assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, + Assertions.assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); - Assert.assertTrue(ugi.isFromKeytab()); - Assert.assertFalse( + Assertions.assertTrue(ugi.isFromKeytab()); + Assertions.assertFalse( UserGroupInformation.isKerberosKeyTabLoginRenewalEnabled()); - Assert.assertFalse( + Assertions.assertFalse( UserGroupInformation.getKerberosLoginRenewalExecutor() .isPresent()); } @@ -329,13 +330,13 @@ private static KerberosTicket getTicket(UserGroupInformation ugi) { // the expected principal. private static KerberosTicket checkTicketAndKeytab(UserGroupInformation ugi, KerberosPrincipal principal, boolean expectIsKeytab) { - Assert.assertEquals("wrong principal", - principal.getName(), ugi.getUserName()); - Assert.assertEquals("is not keytab", - expectIsKeytab, ugi.isFromKeytab()); + Assertions.assertEquals( + principal.getName(), ugi.getUserName(), "wrong principal"); + Assertions.assertEquals( + expectIsKeytab, ugi.isFromKeytab(), "is not keytab"); KerberosTicket ticket = getTicket(ugi); - Assert.assertNotNull("no ticket", ticket); - Assert.assertEquals("wrong principal", principal, ticket.getClient()); + Assertions.assertNotNull(ticket, "no ticket"); + Assertions.assertEquals(principal, ticket.getClient(), "wrong principal"); return ticket; } @@ -377,16 +378,16 @@ public Void run() throws IOException { loginUser.reloginFromKeytab(); KerberosTicket newLoginTicket = checkTicketAndKeytab(loginUser, principal1, true); - Assert.assertNotEquals(loginTicket.getAuthTime(), + Assertions.assertNotEquals(loginTicket.getAuthTime(), newLoginTicket.getAuthTime()); // verify an "external" subject ticket does not change. extSubjectUser.reloginFromKeytab(); - Assert.assertSame(ticket, + Assertions.assertSame(ticket, checkTicketAndKeytab(extSubjectUser, principal2, false)); // verify subject ugi relogin did not affect the login user. - Assert.assertSame(newLoginTicket, + Assertions.assertSame(newLoginTicket, checkTicketAndKeytab(loginUser, principal1, true)); return null; @@ -409,7 +410,7 @@ public void testReloginForLoginFromSubject() throws Exception { principal1.getName(), keytab1.getPath()); final UserGroupInformation originalLoginUser = UserGroupInformation.getLoginUser(); - Assert.assertNotNull(getUser(originalLoginUser.getSubject()).getLogin()); + Assertions.assertNotNull(getUser(originalLoginUser.getSubject()).getLogin()); originalLoginUser.doAs(new PrivilegedExceptionAction() { @Override @@ -426,7 +427,7 @@ public Void run() throws IOException { // verify the new login user is external. UserGroupInformation.loginUserFromSubject(subject); - Assert.assertNull(getUser(subject).getLogin()); + Assertions.assertNull(getUser(subject).getLogin()); UserGroupInformation extLoginUser = UserGroupInformation.getLoginUser(); KerberosTicket extLoginUserTicket = @@ -435,17 +436,17 @@ public Void run() throws IOException { // verify subject-based login user does not get a new ticket, and // original login user not affected. extLoginUser.reloginFromKeytab(); - Assert.assertSame(extLoginUserTicket, + Assertions.assertSame(extLoginUserTicket, checkTicketAndKeytab(extLoginUser, principal2, false)); - Assert.assertSame(originalLoginUserTicket, + Assertions.assertSame(originalLoginUserTicket, checkTicketAndKeytab(originalLoginUser, principal1, true)); // verify original login user gets a new ticket, new login user // not affected. originalLoginUser.reloginFromKeytab(); - Assert.assertNotSame(originalLoginUserTicket, + Assertions.assertNotSame(originalLoginUserTicket, checkTicketAndKeytab(originalLoginUser, principal1, true)); - Assert.assertSame(extLoginUserTicket, + Assertions.assertSame(extLoginUserTicket, checkTicketAndKeytab(extLoginUser, principal2, false)); return null; } @@ -465,27 +466,28 @@ public void testReloginAfterFailedRelogin() throws Exception { checkTicketAndKeytab(loginUser, principal, true); // move the keytab to induce a relogin failure. - Assert.assertTrue(keytab.renameTo(keytabBackup)); + Assertions.assertTrue(keytab.renameTo(keytabBackup)); try { loginUser.reloginFromKeytab(); - Assert.fail("relogin should fail"); + Assertions.fail("relogin should fail"); } catch (KerberosAuthException kae) { // expected. } // even though no KeyTab object, ugi should know it's keytab based. - Assert.assertTrue(loginUser.isFromKeytab()); - Assert.assertNull(getTicket(loginUser)); + Assertions.assertTrue(loginUser.isFromKeytab()); + Assertions.assertNull(getTicket(loginUser)); // move keytab back to enable relogin to succeed. - Assert.assertTrue(keytabBackup.renameTo(keytab)); + Assertions.assertTrue(keytabBackup.renameTo(keytab)); loginUser.reloginFromKeytab(); checkTicketAndKeytab(loginUser, principal, true); } // verify getting concurrent relogins blocks to avoid indeterminate // credentials corruption, but getting a ugi for the subject does not block. - @Test(timeout=180000) + @Test + @Timeout(value = 180) public void testConcurrentRelogin() throws Exception { final CyclicBarrier barrier = new CyclicBarrier(2); final CountDownLatch latch = new CountDownLatch(1); @@ -537,8 +539,8 @@ public Void call() throws Exception { }); // wait for the thread to block on the barrier in the logout of the // relogin. - assertTrue("first relogin didn't block", - latch.await(2, TimeUnit.SECONDS)); + assertTrue( + latch.await(2, TimeUnit.SECONDS), "first relogin didn't block"); // although the logout removed the keytab instance, verify the ugi // knows from its login params that it is supposed to be from a keytab. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java index 896643a038077..e32ef16e8f375 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java @@ -18,15 +18,15 @@ import java.io.IOException; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import static org.apache.hadoop.security.SecurityUtilTestHelper.isExternalKdcRunning; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Tests kerberos keytab login using a user-specified external KDC @@ -39,7 +39,7 @@ */ public class TestUGIWithExternalKdc { - @Before + @BeforeEach public void testExternalKdcRunning() { Assume.assumeTrue(isExternalKdcRunning()); } @@ -48,8 +48,8 @@ public void testExternalKdcRunning() { public void testLogin() throws IOException { String userPrincipal = System.getProperty("user.principal"); String userKeyTab = System.getProperty("user.keytab"); - Assert.assertNotNull("User principal was not specified", userPrincipal); - Assert.assertNotNull("User keytab was not specified", userKeyTab); + Assertions.assertNotNull(userPrincipal, "User principal was not specified"); + Assertions.assertNotNull(userKeyTab, "User keytab was not specified"); Configuration conf = new Configuration(); conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, @@ -59,13 +59,13 @@ public void testLogin() throws IOException { UserGroupInformation ugi = UserGroupInformation .loginUserFromKeytabAndReturnUGI(userPrincipal, userKeyTab); - Assert.assertEquals(AuthenticationMethod.KERBEROS, + Assertions.assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); try { UserGroupInformation .loginUserFromKeytabAndReturnUGI("bogus@EXAMPLE.COM", userKeyTab); - Assert.fail("Login should have failed"); + Assertions.fail("Login should have failed"); } catch (Exception ex) { ex.printStackTrace(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java index f04fbe1e08926..5b62da2545b43 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java @@ -22,8 +22,9 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import javax.security.auth.kerberos.KerberosPrincipal; @@ -43,7 +44,7 @@ public class TestUGIWithMiniKdc { private static MiniKdc kdc; - @After + @AfterEach public void teardown() { UserGroupInformation.reset(); if (kdc != null) { @@ -63,7 +64,8 @@ private void setupKdc() throws Exception { kdc.start(); } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testAutoRenewalThreadRetryWithKdc() throws Exception { GenericTestUtils.setLogLevel(UserGroupInformation.LOG, Level.DEBUG); final Configuration conf = new Configuration(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserFromEnv.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserFromEnv.java index e436f26a07c84..b173c97f0e863 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserFromEnv.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserFromEnv.java @@ -18,15 +18,15 @@ package org.apache.hadoop.security; import java.io.IOException; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestUserFromEnv { @Test public void testUserFromEnvironment() throws IOException { System.setProperty(UserGroupInformation.HADOOP_USER_NAME, "randomUser"); - Assert.assertEquals("randomUser", UserGroupInformation.getLoginUser() + Assertions.assertEquals("randomUser", UserGroupInformation.getLoginUser() .getUserName()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java index 1327561cf4356..a285de8a851e9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java @@ -33,11 +33,12 @@ import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -81,16 +82,16 @@ import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges; import static org.apache.hadoop.test.MetricsAsserts.getDoubleGauge; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.junit.Assert.assertArrayEquals; -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.assertNotSame; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -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.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -134,7 +135,7 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) { } /** configure ugi */ - @BeforeClass + @BeforeAll public static void setup() { javax.security.auth.login.Configuration.setConfiguration( new DummyLoginConfiguration()); @@ -145,29 +146,32 @@ public static void setup() { System.setProperty("hadoop.home.dir", (home != null ? home : ".")); } - @Before + @BeforeEach public void setupUgi() { conf = new Configuration(); UserGroupInformation.reset(); UserGroupInformation.setConfiguration(conf); } - @After + @AfterEach public void resetUgi() { UserGroupInformation.setLoginUser(null); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testSimpleLogin() throws IOException { tryLoginAuthenticationMethod(AuthenticationMethod.SIMPLE, true); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testTokenLogin() throws IOException { tryLoginAuthenticationMethod(AuthenticationMethod.TOKEN, false); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testProxyLogin() throws IOException { tryLoginAuthenticationMethod(AuthenticationMethod.PROXY, false); } @@ -196,7 +200,8 @@ private void tryLoginAuthenticationMethod(AuthenticationMethod method, } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testGetRealAuthenticationMethod() { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user1"); ugi.setAuthenticationMethod(AuthenticationMethod.SIMPLE); @@ -207,7 +212,8 @@ public void testGetRealAuthenticationMethod() { assertEquals(AuthenticationMethod.SIMPLE, ugi.getRealAuthenticationMethod()); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testCreateRemoteUser() { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user1"); assertEquals(AuthenticationMethod.SIMPLE, ugi.getAuthenticationMethod()); @@ -219,7 +225,8 @@ public void testCreateRemoteUser() { } /** Test login method */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testLogin() throws Exception { conf.set(HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS, String.valueOf(PERCENTILES_INTERVAL)); @@ -250,7 +257,8 @@ public UserGroupInformation run() throws IOException { * given user name - get all the groups. * Needs to happen before creating the test users */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testGetServerSideGroups() throws IOException, InterruptedException { // get the user name @@ -311,7 +319,8 @@ public Object run() throws IOException { } /** test constructor */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testConstructor() throws Exception { // security off, so default should just return simple name testConstructorSuccess("user1", "user1"); @@ -325,7 +334,8 @@ public void testConstructor() throws Exception { } /** test constructor */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testConstructorWithRules() throws Exception { // security off, but use rules if explicitly set conf.set(HADOOP_SECURITY_AUTH_TO_LOCAL, @@ -357,7 +367,8 @@ public void testConstructorWithRules() throws Exception { } /** test constructor */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testConstructorWithKerberos() throws Exception { // security on, default is remove default realm conf.set(HADOOP_SECURITY_AUTH_TO_LOCAL_MECHANISM, "hadoop"); @@ -387,7 +398,8 @@ public void testConstructorWithKerberos() throws Exception { } /** test constructor */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testConstructorWithKerberosRules() throws Exception { // security on, explicit rules SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf); @@ -423,12 +435,13 @@ private void testConstructorFailures(String userName) { String expect = (userName == null || userName.isEmpty()) ? "Null user" : "Illegal principal name "+userName; String expect2 = "Malformed Kerberos name: "+userName; - assertTrue("Did not find "+ expect + " or " + expect2 + " in " + e, - e.toString().contains(expect) || e.toString().contains(expect2)); + assertTrue( + e.toString().contains(expect) || e.toString().contains(expect2), "Did not find "+ expect + " or " + expect2 + " in " + e); } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testSetConfigWithRules() { String[] rules = { "RULE:[1:TEST1]", "RULE:[1:TEST2]", "RULE:[1:TEST3]" }; @@ -458,7 +471,8 @@ public void testSetConfigWithRules() { assertEquals(rules[2], KerberosName.getRules()); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testEnsureInitWithRules() throws IOException { String rules = "RULE:[1:RULE1]"; @@ -477,7 +491,8 @@ public void testEnsureInitWithRules() throws IOException { assertEquals(rules, KerberosName.getRules()); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testEquals() throws Exception { UserGroupInformation uugi = UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES); @@ -495,7 +510,8 @@ public void testEquals() throws Exception { assertEquals(uugi.hashCode(), ugi3.hashCode()); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testEqualsWithRealUser() throws Exception { UserGroupInformation realUgi1 = UserGroupInformation.createUserForTesting( "RealUser", GROUP_NAMES); @@ -508,7 +524,8 @@ public void testEqualsWithRealUser() throws Exception { assertFalse(remoteUgi.equals(proxyUgi1)); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testGettingGroups() throws Exception { UserGroupInformation uugi = UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES); @@ -519,8 +536,8 @@ public void testGettingGroups() throws Exception { assertEquals(GROUP1_NAME, uugi.getPrimaryGroupName()); } - @SuppressWarnings("unchecked") // from Mockito mocks - @Test (timeout = 30000) + @SuppressWarnings("unchecked")@Test + @Timeout(value = 30) public void testAddToken() throws Exception { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("someone"); @@ -557,8 +574,8 @@ public void testAddToken() throws Exception { checkTokens(ugi, t1, t2, t3); } - @SuppressWarnings("unchecked") // from Mockito mocks - @Test (timeout = 30000) + @SuppressWarnings("unchecked")@Test + @Timeout(value = 30) public void testGetCreds() throws Exception { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("someone"); @@ -583,8 +600,8 @@ public void testGetCreds() throws Exception { checkTokens(ugi, t1, t2); } - @SuppressWarnings("unchecked") // from Mockito mocks - @Test (timeout = 30000) + @SuppressWarnings("unchecked")@Test + @Timeout(value = 30) public void testAddCreds() throws Exception { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("someone"); @@ -609,7 +626,8 @@ public void testAddCreds() throws Exception { assertSame(secret, ugi.getCredentials().getSecretKey(secretKey)); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testGetCredsNotSame() throws Exception { UserGroupInformation ugi = @@ -636,8 +654,8 @@ private void checkTokens(UserGroupInformation ugi, Token ... tokens) { assertEquals(tokens.length, ugiCreds.numberOfTokens()); } - @SuppressWarnings("unchecked") // from Mockito mocks - @Test (timeout = 30000) + @SuppressWarnings("unchecked")@Test + @Timeout(value = 30) public void testAddNamedToken() throws Exception { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("someone"); @@ -657,8 +675,8 @@ public void testAddNamedToken() throws Exception { assertSame(t1, ugi.getCredentials().getToken(service2)); } - @SuppressWarnings("unchecked") // from Mockito mocks - @Test (timeout = 30000) + @SuppressWarnings("unchecked")@Test + @Timeout(value = 30) public void testUGITokens() throws Exception { UserGroupInformation ugi = UserGroupInformation.createUserForTesting("TheDoctor", @@ -704,7 +722,8 @@ public Collection> run() throws IOException { assertTrue(otherSet.contains(t2)); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testTokenIdentifiers() throws Exception { UserGroupInformation ugi = UserGroupInformation.createUserForTesting( "TheDoctor", new String[] { "TheTARDIS" }); @@ -732,7 +751,8 @@ public Collection run() throws IOException { assertEquals(2, otherSet.size()); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testTestAuthMethod() throws Exception { UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); // verify the reverse mappings works @@ -744,40 +764,42 @@ public void testTestAuthMethod() throws Exception { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUGIAuthMethod() throws Exception { final UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); final AuthenticationMethod am = AuthenticationMethod.KERBEROS; ugi.setAuthenticationMethod(am); - Assert.assertEquals(am, ugi.getAuthenticationMethod()); + Assertions.assertEquals(am, ugi.getAuthenticationMethod()); ugi.doAs(new PrivilegedExceptionAction() { @Override public Object run() throws IOException { - Assert.assertEquals(am, UserGroupInformation.getCurrentUser() + Assertions.assertEquals(am, UserGroupInformation.getCurrentUser() .getAuthenticationMethod()); return null; } }); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUGIAuthMethodInRealUser() throws Exception { final UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); UserGroupInformation proxyUgi = UserGroupInformation.createProxyUser( "proxy", ugi); final AuthenticationMethod am = AuthenticationMethod.KERBEROS; ugi.setAuthenticationMethod(am); - Assert.assertEquals(am, ugi.getAuthenticationMethod()); - Assert.assertEquals(AuthenticationMethod.PROXY, + Assertions.assertEquals(am, ugi.getAuthenticationMethod()); + Assertions.assertEquals(AuthenticationMethod.PROXY, proxyUgi.getAuthenticationMethod()); - Assert.assertEquals(am, UserGroupInformation + Assertions.assertEquals(am, UserGroupInformation .getRealAuthenticationMethod(proxyUgi)); proxyUgi.doAs(new PrivilegedExceptionAction() { @Override public Object run() throws IOException { - Assert.assertEquals(AuthenticationMethod.PROXY, UserGroupInformation + Assertions.assertEquals(AuthenticationMethod.PROXY, UserGroupInformation .getCurrentUser().getAuthenticationMethod()); - Assert.assertEquals(am, UserGroupInformation.getCurrentUser() + Assertions.assertEquals(am, UserGroupInformation.getCurrentUser() .getRealUser().getAuthenticationMethod()); return null; } @@ -785,17 +807,18 @@ public Object run() throws IOException { UserGroupInformation proxyUgi2 = new UserGroupInformation(proxyUgi.getSubject()); proxyUgi2.setAuthenticationMethod(AuthenticationMethod.PROXY); - Assert.assertEquals(proxyUgi, proxyUgi2); + Assertions.assertEquals(proxyUgi, proxyUgi2); // Equality should work if authMethod is null UserGroupInformation realugi = UserGroupInformation.getCurrentUser(); UserGroupInformation proxyUgi3 = UserGroupInformation.createProxyUser( "proxyAnother", realugi); UserGroupInformation proxyUgi4 = new UserGroupInformation(proxyUgi3.getSubject()); - Assert.assertEquals(proxyUgi3, proxyUgi4); + Assertions.assertEquals(proxyUgi3, proxyUgi4); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testLoginObjectInSubject() throws Exception { UserGroupInformation loginUgi = UserGroupInformation.getLoginUser(); UserGroupInformation anotherUgi = new UserGroupInformation(loginUgi @@ -805,10 +828,11 @@ public void testLoginObjectInSubject() throws Exception { LoginContext login2 = anotherUgi.getSubject().getPrincipals(User.class) .iterator().next().getLogin(); //login1 and login2 must be same instances - Assert.assertTrue(login1 == login2); + Assertions.assertTrue(login1 == login2); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testLoginModuleCommit() throws Exception { UserGroupInformation loginUgi = UserGroupInformation.getLoginUser(); User user1 = loginUgi.getSubject().getPrincipals(User.class).iterator() @@ -819,7 +843,7 @@ public void testLoginModuleCommit() throws Exception { User user2 = loginUgi.getSubject().getPrincipals(User.class).iterator() .next(); // user1 and user2 must be same instances. - Assert.assertTrue(user1 == user2); + Assertions.assertTrue(user1 == user2); } public static void verifyLoginMetrics(long success, int failure) @@ -857,7 +881,8 @@ private static void verifyGroupMetrics( * with it, but that Subject was not created by Hadoop (ie it has no * associated User principal) */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUGIUnderNonHadoopContext() throws Exception { Subject nonHadoopSubject = new Subject(); Subject.doAs(nonHadoopSubject, new PrivilegedExceptionAction() { @@ -870,7 +895,8 @@ public Void run() throws IOException { }); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testGetUGIFromSubject() throws Exception { KerberosPrincipal p = new KerberosPrincipal("guest"); Subject subject = new Subject(); @@ -881,7 +907,8 @@ public void testGetUGIFromSubject() throws Exception { } /** Test hasSufficientTimeElapsed method */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testHasSufficientTimeElapsed() throws Exception { // Make hasSufficientTimeElapsed public Method method = UserGroupInformation.class @@ -915,7 +942,8 @@ public void testHasSufficientTimeElapsed() throws Exception { method.setAccessible(false); } - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testSetLoginUser() throws IOException { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test-user"); UserGroupInformation.setLoginUser(ugi); @@ -982,8 +1010,8 @@ public Void run() throws Exception { Token t = mock(Token.class); when(t.getService()).thenReturn(new Text("t" + i)); UserGroupInformation.getCurrentUser().addToken(t); - assertNull("ConcurrentModificationException encountered", - thread.cme); + assertNull( + thread.cme, "ConcurrentModificationException encountered"); } } catch (ConcurrentModificationException cme) { cme.printStackTrace(); @@ -1139,7 +1167,7 @@ public void testGetNextRetryTime() throws Exception { String str = "5th retry, now:" + currentTime + ", retry:" + lastRetry; LOG.info(str); - assertEquals(str, endTime - reloginIntervalMs, lastRetry); + assertEquals(endTime - reloginIntervalMs, lastRetry, str); // make sure no more retries after (tgt endTime - login interval). UserGroupInformation.metrics.getRenewalFailures().incr(); @@ -1147,7 +1175,7 @@ public void testGetNextRetryTime() throws Exception { UserGroupInformation.getNextTgtRenewalTime(endTime, currentTime, rp); str = "overflow retry, now:" + currentTime + ", retry:" + lastRetry; LOG.info(str); - assertEquals(str, endTime - reloginIntervalMs, lastRetry); + assertEquals(endTime - reloginIntervalMs, lastRetry, str); } private void assertWithinBounds(final int numFailures, final long lastRetry, @@ -1160,12 +1188,13 @@ private void assertWithinBounds(final int numFailures, final long lastRetry, + ", lower bound:" + lower + ", upper bound:" + upper + ", retry:" + lastRetry); LOG.info(str); - assertTrue(str, lower <= lastRetry && lastRetry < upper); + assertTrue(lower <= lastRetry && lastRetry < upper, str); } // verify that getCurrentUser on the same and different subjects can be // concurrent. Ie. no synchronization. - @Test(timeout=8000) + @Test + @Timeout(value = 8) public void testConcurrentGetCurrentUser() throws Exception { final CyclicBarrier barrier = new CyclicBarrier(2); final CountDownLatch latch = new CountDownLatch(1); @@ -1306,8 +1335,8 @@ public void testImportTokensFromConfig() throws IOException { // Check if the tokens were loaded UserGroupInformation ugi = UserGroupInformation.getLoginUser(); Credentials outCred = ugi.getCredentials(); - assertEquals("Tokens: " + outCred.getAllTokens(), - 2, outCred.getAllTokens().size()); + assertEquals( + 2, outCred.getAllTokens().size(), "Tokens: " + outCred.getAllTokens()); boolean found0 = false; boolean found1 = false; for (Token token : outCred.getAllTokens()) { @@ -1321,10 +1350,10 @@ public void testImportTokensFromConfig() throws IOException { found1 = true; } } - assertTrue("Expected token testTokenService0 not found: " + outCred, - found0); - assertTrue("Expected token testTokenService1 not found: " + outCred, - found1); + assertTrue( + found0, "Expected token testTokenService0 not found: " + outCred); + assertTrue( + found1, "Expected token testTokenService1 not found: " + outCred); // Try to add the same token through configuration and file Credentials cred1 = new Credentials(); @@ -1336,8 +1365,8 @@ public void testImportTokensFromConfig() throws IOException { UserGroupInformation ugi1 = UserGroupInformation.getLoginUser(); Credentials outCred1 = ugi1.getCredentials(); - assertEquals("Tokens: " + outCred1.getAllTokens(), - 1, outCred1.getAllTokens().size()); + assertEquals( + 1, outCred1.getAllTokens().size(), "Tokens: " + outCred1.getAllTokens()); } @Test @@ -1356,8 +1385,8 @@ public void testImportTokensFromProperty() throws IOException { UserGroupInformation.reset(); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); Credentials creds = ugi.getCredentials(); - assertEquals("Tokens: " + creds.getAllTokens(), - 1, creds.getAllTokens().size()); + assertEquals( + 1, creds.getAllTokens().size(), "Tokens: " + creds.getAllTokens()); assertArrayEquals(creds.getToken(service).getIdentifier(), identity); // Cleanup diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java index 81abc42a023f3..a55322a849e02 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java @@ -21,8 +21,8 @@ import java.net.InetAddress; import java.util.Map; -import org.junit.Test; -import static org.junit.Assert.*; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.*; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.TestFileBasedIPList; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java index 7061589302939..2a7b84d2112ed 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.security.alias; -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.ByteArrayOutputStream; import java.io.File; @@ -33,8 +33,8 @@ import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.test.GenericTestUtils; import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestCredShell { private final ByteArrayOutputStream outContent = new ByteArrayOutputStream(); @@ -49,7 +49,7 @@ private void assertOutputContains(String expected) { .contains(expected); } - @Before + @BeforeEach public void setup() throws Exception { System.setOut(new PrintStream(outContent)); System.setErr(new PrintStream(errContent)); @@ -67,7 +67,7 @@ public void testCredentialSuccessfulLifecycle() throws Exception { CredentialShell cs = new CredentialShell(); cs.setConf(new Configuration()); rc = cs.run(args1); - assertEquals(outContent.toString(), 0, rc); + assertEquals(0, rc, outContent.toString()); assertTrue(outContent.toString().contains("credential1 has been successfully " + "created.")); assertTrue(outContent.toString() @@ -97,7 +97,7 @@ public void testCredentialSuccessfulLifecycle() throws Exception { jceksProvider}; rc = cs.run(args5); assertEquals(0, rc); - assertFalse(outContent.toString(), outContent.toString().contains("credential1")); + assertFalse(outContent.toString().contains("credential1"), outContent.toString()); } @Test @@ -123,13 +123,13 @@ public void testTransientProviderWarning() throws Exception { CredentialShell cs = new CredentialShell(); cs.setConf(new Configuration()); rc = cs.run(args1); - assertEquals(outContent.toString(), 0, rc); + assertEquals(0, rc, outContent.toString()); assertTrue(outContent.toString().contains("WARNING: you are modifying a " + "transient provider.")); String[] args2 = {"delete", "credential1", "-f", "-provider", "user:///"}; rc = cs.run(args2); - assertEquals(outContent.toString(), 0, rc); + assertEquals(0, rc, outContent.toString()); assertTrue(outContent.toString().contains("credential1 has been successfully " + "deleted.")); } @@ -161,7 +161,7 @@ public void testPromptForCredentialWithEmptyPasswd() throws Exception { shell.setConf(new Configuration()); shell.setPasswordReader(new MockPasswordReader(passwords)); rc = shell.run(args1); - assertEquals(outContent.toString(), 1, rc); + assertEquals(1, rc, outContent.toString()); assertTrue(outContent.toString().contains("Passwords don't match")); } @@ -249,8 +249,8 @@ public void testCommandHelpExitsNormally() throws Exception { for (String cmd : Arrays.asList("create", "list", "delete")) { CredentialShell shell = new CredentialShell(); shell.setConf(new Configuration()); - assertEquals("Expected help argument on " + cmd + " to return 0", - 0, shell.init(new String[] {cmd, "-help"})); + assertEquals( + 0, shell.init(new String[] {cmd, "-help"}), "Expected help argument on " + cmd + " to return 0"); } } @@ -258,13 +258,13 @@ public void testCommandHelpExitsNormally() throws Exception { public void testEmptyArgForCommands() throws Exception { CredentialShell shell = new CredentialShell(); String[] command = { "list", "-provider" }; - assertEquals("Expected empty argument on " + command + " to return 1", 1, - shell.init(command)); + assertEquals(1 +, shell.init(command), "Expected empty argument on " + command + " to return 1"); for (String cmd : Arrays.asList("create", "delete")) { shell.setConf(new Configuration()); - assertEquals("Expected empty argument on " + cmd + " to return 1", 1, - shell.init(new String[] { cmd })); + assertEquals(1 +, shell.init(new String[] { cmd }), "Expected empty argument on " + cmd + " to return 1"); } } @@ -277,7 +277,7 @@ public void testStrict() throws Exception { CredentialShell cs = new CredentialShell(); cs.setConf(new Configuration()); rc = cs.run(args1); - assertEquals(outContent.toString(), 1, rc); + assertEquals(1, rc, outContent.toString()); assertFalse(outContent.toString().contains("credential1 has been " + "successfully created.")); assertTrue(outContent.toString() @@ -294,7 +294,7 @@ public void testHelp() throws Exception { CredentialShell cs = new CredentialShell(); cs.setConf(new Configuration()); rc = cs.run(args1); - assertEquals(outContent.toString(), 0, rc); + assertEquals(0, rc, outContent.toString()); assertTrue(outContent.toString().contains("Usage")); } @@ -306,7 +306,7 @@ public void testHelpCreate() throws Exception { CredentialShell cs = new CredentialShell(); cs.setConf(new Configuration()); rc = cs.run(args1); - assertEquals(outContent.toString(), 0, rc); + assertEquals(0, rc, outContent.toString()); assertTrue(outContent.toString().contains("Usage")); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProvider.java index 0d83974c4693e..a5ddc22409f1a 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProvider.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProvider.java @@ -19,12 +19,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.ProviderUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.net.URI; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; public class TestCredentialProvider { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java index 37da798e804fd..d5d762e6c167e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java @@ -33,17 +33,17 @@ import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestCredentialProviderFactory { public static final Logger LOG = @@ -52,7 +52,7 @@ public class TestCredentialProviderFactory { @Rule public final TestName test = new TestName(); - @Before + @BeforeEach public void announce() { LOG.info("Running test " + test.getMethodName()); } @@ -90,7 +90,7 @@ public void testFactoryErrors() throws Exception { try { List providers = CredentialProviderFactory.getProviders(conf); - assertTrue("should throw!", false); + assertTrue(false, "should throw!"); } catch (IOException e) { assertEquals("No CredentialProviderFactory for unknown:/// in " + CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, @@ -105,7 +105,7 @@ public void testUriErrors() throws Exception { try { List providers = CredentialProviderFactory.getProviders(conf); - assertTrue("should throw!", false); + assertTrue(false, "should throw!"); } catch (IOException e) { assertEquals("Bad configuration of " + CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH + @@ -143,14 +143,14 @@ static void checkSpecificProvider(Configuration conf, // try recreating pass try { provider.createCredentialEntry("pass", passwd); - assertTrue("should throw", false); + assertTrue(false, "should throw"); } catch (IOException e) { assertEquals("Credential pass already exists in " + ourUrl, e.getMessage()); } provider.deleteCredentialEntry("pass"); try { provider.deleteCredentialEntry("pass"); - assertTrue("should throw", false); + assertTrue(false, "should throw"); } catch (IOException e) { assertEquals("Credential pass does not exist in " + ourUrl, e.getMessage()); } @@ -183,9 +183,9 @@ static void checkSpecificProvider(Configuration conf, assertArrayEquals(passwd, provider.getCredentialEntry("pass").getCredential()); List creds = provider.getAliases(); - assertTrue("Credentials should have been returned.", creds.size() == 2); - assertTrue("Returned Credentials should have included pass.", creds.contains("pass")); - assertTrue("Returned Credentials should have included pass2.", creds.contains("pass2")); + assertTrue(creds.size() == 2, "Credentials should have been returned."); + assertTrue(creds.contains("pass"), "Returned Credentials should have included pass."); + assertTrue(creds.contains("pass2"), "Returned Credentials should have included pass2."); } @Test @@ -216,7 +216,7 @@ public void testJksProvider() throws Exception { FileSystem fs = path.getFileSystem(conf); FileStatus s = fs.getFileStatus(path); assertEquals("rw-------", s.getPermission().toString()); - assertTrue(file + " should exist", file.isFile()); + assertTrue(file.isFile(), file + " should exist"); // check permission retention after explicit change fs.setPermission(path, new FsPermission("777")); @@ -239,7 +239,7 @@ public void testLocalJksProvider() throws Exception { FileStatus s = fs.getFileStatus(path); assertEquals("Unexpected permissions: " + s.getPermission().toString(), "rw-------", s.getPermission().toString()); - assertTrue(file + " should exist", file.isFile()); + assertTrue(file.isFile(), file + " should exist"); // check permission retention after explicit change fs.setPermission(path, new FsPermission("777")); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authentication/server/TestProxyUserAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authentication/server/TestProxyUserAuthenticationFilter.java index 978c15d8f2a0f..f67b72c7eadf2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authentication/server/TestProxyUserAuthenticationFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authentication/server/TestProxyUserAuthenticationFilter.java @@ -39,7 +39,8 @@ import javax.servlet.http.HttpServletResponse; import static org.assertj.core.api.Assertions.assertThat; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; @@ -271,7 +272,8 @@ public Locale getLocale() { } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testFilter() throws Exception { Map params = new HashMap(); params.put("proxyuser.knox.users", "testuser"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java index 53ab275b664fb..5b94c6d8a045b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.security.authorize; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Collection; import java.util.Iterator; @@ -31,7 +31,7 @@ import org.apache.hadoop.security.Groups; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.NativeCodeLoader; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -461,15 +461,15 @@ public void testIsUserAllowed() { private void assertUserAllowed(UserGroupInformation ugi, AccessControlList acl) { - assertTrue("User " + ugi + " is not granted the access-control!!", - acl.isUserAllowed(ugi)); + assertTrue( + acl.isUserAllowed(ugi), "User " + ugi + " is not granted the access-control!!"); } private void assertUserNotAllowed(UserGroupInformation ugi, AccessControlList acl) { - assertFalse("User " + ugi - + " is incorrectly granted the access-control!!", - acl.isUserAllowed(ugi)); + assertFalse( + acl.isUserAllowed(ugi), "User " + ugi + + " is incorrectly granted the access-control!!"); } @Test @@ -481,12 +481,12 @@ public void testUseRealUserAclsForProxiedUser() { UserGroupInformation user1 = UserGroupInformation.createProxyUserForTesting("regularJane", realUserUgi, new String [] {"group1"}); - assertFalse("User " + user1 + " should not have been granted access.", - acl.isUserAllowed(user1)); + assertFalse( + acl.isUserAllowed(user1), "User " + user1 + " should not have been granted access."); acl = new AccessControlList(AccessControlList.USE_REAL_ACLS + realUser); - assertTrue("User " + user1 + " should have access but was denied.", - acl.isUserAllowed(user1)); + assertTrue( + acl.isUserAllowed(user1), "User " + user1 + " should have access but was denied."); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java index 9c9618ce5b3cf..47d25044e6772 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java @@ -22,10 +22,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.LambdaTestUtils; -import org.junit.After; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.Timeout; import org.mockito.Mockito; @@ -47,7 +47,7 @@ public class TestDefaultImpersonationProvider { @Rule public Timeout globalTimeout = new Timeout(10000, TimeUnit.MILLISECONDS); - @Before + @BeforeEach public void setup() { conf = new Configuration(); provider = new DefaultImpersonationProvider(); @@ -91,7 +91,7 @@ public void testAuthorizationFailure() throws Exception { provider.authorize(userGroupInformation, "2.2.2.2")); } - @After + @AfterEach public void clear() { provider = null; conf = null; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyServers.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyServers.java index 858fb7b1a8b2a..bc89b7ffd99f3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyServers.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyServers.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.security.authorize; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.hadoop.conf.Configuration; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestProxyServers { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java index ab9de2d308ac0..45fe04514c0fd 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java @@ -17,9 +17,6 @@ */ package org.apache.hadoop.security.authorize; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; @@ -33,10 +30,12 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.NativeCodeLoader; import org.apache.hadoop.util.StringUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.junit.jupiter.api.Assertions.*; + public class TestProxyUsers { private static final Logger LOG = @@ -336,43 +335,39 @@ public void testIPRange() { assertNotAuthorized(proxyUserUgi, "10.221.0.0"); } - @Test(expected = IllegalArgumentException.class) + @Test public void testNullUser() throws Exception { - Configuration conf = new Configuration(); - conf.set( - DefaultImpersonationProvider.getTestProvider(). - getProxySuperuserGroupConfKey(REAL_USER_NAME), - "*"); - conf.set( - DefaultImpersonationProvider.getTestProvider(). - getProxySuperuserIpConfKey(REAL_USER_NAME), - PROXY_IP_RANGE); - ProxyUsers.refreshSuperUserGroupsConfiguration(conf); - // user is null - ProxyUsers.authorize(null, "10.222.0.0"); + assertThrows(IllegalArgumentException.class, () -> { + Configuration conf = new Configuration(); + conf.set(DefaultImpersonationProvider.getTestProvider(). + getProxySuperuserGroupConfKey(REAL_USER_NAME), "*"); + conf.set(DefaultImpersonationProvider.getTestProvider(). + getProxySuperuserIpConfKey(REAL_USER_NAME), PROXY_IP_RANGE); + ProxyUsers.refreshSuperUserGroupsConfiguration(conf); + // user is null + ProxyUsers.authorize(null, "10.222.0.0"); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void testNullIpAddress() throws Exception { - Configuration conf = new Configuration(); - conf.set( - DefaultImpersonationProvider.getTestProvider(). - getProxySuperuserGroupConfKey(REAL_USER_NAME), - "*"); - conf.set( - DefaultImpersonationProvider.getTestProvider(). - getProxySuperuserIpConfKey(REAL_USER_NAME), - PROXY_IP_RANGE); - ProxyUsers.refreshSuperUserGroupsConfiguration(conf); - - // First try proxying a group that's allowed - UserGroupInformation realUserUgi = UserGroupInformation - .createRemoteUser(REAL_USER_NAME); - UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting( - PROXY_USER_NAME, realUserUgi, GROUP_NAMES); - - // remote address is null - ProxyUsers.authorize(proxyUserUgi, (InetAddress) null); + assertThrows(IllegalArgumentException.class, () -> { + Configuration conf = new Configuration(); + conf.set(DefaultImpersonationProvider.getTestProvider(). + getProxySuperuserGroupConfKey(REAL_USER_NAME), "*"); + conf.set(DefaultImpersonationProvider.getTestProvider(). + getProxySuperuserIpConfKey(REAL_USER_NAME), PROXY_IP_RANGE); + ProxyUsers.refreshSuperUserGroupsConfiguration(conf); + + // First try proxying a group that's allowed + UserGroupInformation realUserUgi = UserGroupInformation + .createRemoteUser(REAL_USER_NAME); + UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting( + PROXY_USER_NAME, realUserUgi, GROUP_NAMES); + + // remote address is null + ProxyUsers.authorize(proxyUserUgi, (InetAddress) null); + }); } @Test @@ -476,16 +471,17 @@ public void testWithProxyGroupsAndUsersWithSpaces() throws Exception { assertEquals (GROUP_NAMES.length, groupsToBeProxied.size()); } - @Test(expected = IllegalArgumentException.class) + @Test public void testProxyUsersWithNullPrefix() throws Exception { - ProxyUsers.refreshSuperUserGroupsConfiguration(new Configuration(false), - null); + assertThrows(IllegalArgumentException.class, + ()-> ProxyUsers.refreshSuperUserGroupsConfiguration(new Configuration(false), null)); } - @Test(expected = IllegalArgumentException.class) + @Test public void testProxyUsersWithEmptyPrefix() throws Exception { - ProxyUsers.refreshSuperUserGroupsConfiguration(new Configuration(false), - ""); + assertThrows(IllegalArgumentException.class,() -> { + ProxyUsers.refreshSuperUserGroupsConfiguration(new Configuration(false), ""); + }); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestServiceAuthorization.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestServiceAuthorization.java index d02fe604d79e3..c14972056cf6b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestServiceAuthorization.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestServiceAuthorization.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.security.authorize; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.lang.annotation.Annotation; import java.net.InetAddress; @@ -32,7 +32,7 @@ import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.TokenInfo; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestServiceAuthorization { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java index dc587bce61724..6d5c36ee96bf5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java @@ -32,8 +32,8 @@ import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.security.http.CrossOriginFilter; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import static org.apache.hadoop.test.MockitoUtil.verifyZeroInteractions; @@ -76,28 +76,28 @@ public void testAllowAllOrigins() throws ServletException, IOException { // Object under test CrossOriginFilter filter = new CrossOriginFilter(); filter.init(filterConfig); - Assert.assertTrue(filter.areOriginsAllowed("example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("example.com")); } @Test public void testEncodeHeaders() { String validOrigin = "http://localhost:12345"; String encodedValidOrigin = CrossOriginFilter.encodeHeader(validOrigin); - Assert.assertEquals("Valid origin encoding should match exactly", - validOrigin, encodedValidOrigin); + Assertions.assertEquals( + validOrigin, encodedValidOrigin, "Valid origin encoding should match exactly"); String httpResponseSplitOrigin = validOrigin + " \nSecondHeader: value"; String encodedResponseSplitOrigin = CrossOriginFilter.encodeHeader(httpResponseSplitOrigin); - Assert.assertEquals("Http response split origin should be protected against", - validOrigin, encodedResponseSplitOrigin); + Assertions.assertEquals( + validOrigin, encodedResponseSplitOrigin, "Http response split origin should be protected against"); // Test Origin List String validOriginList = "http://foo.example.com:12345 http://bar.example.com:12345"; String encodedValidOriginList = CrossOriginFilter .encodeHeader(validOriginList); - Assert.assertEquals("Valid origin list encoding should match exactly", - validOriginList, encodedValidOriginList); + Assertions.assertEquals( + validOriginList, encodedValidOriginList, "Valid origin list encoding should match exactly"); } @Test @@ -113,17 +113,17 @@ public void testPatternMatchingOrigins() throws ServletException, IOException { filter.init(filterConfig); // match multiple sub-domains - Assert.assertFalse(filter.areOriginsAllowed("example.com")); - Assert.assertFalse(filter.areOriginsAllowed("foo:example.com")); - Assert.assertTrue(filter.areOriginsAllowed("foo.example.com")); - Assert.assertTrue(filter.areOriginsAllowed("foo.bar.example.com")); + Assertions.assertFalse(filter.areOriginsAllowed("example.com")); + Assertions.assertFalse(filter.areOriginsAllowed("foo:example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.bar.example.com")); // First origin is allowed - Assert.assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com")); // Second origin is allowed - Assert.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com")); // No origin in list is allowed - Assert.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); + Assertions.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); } @Test @@ -139,17 +139,17 @@ public void testRegexPatternMatchingOrigins() throws ServletException, IOExcepti filter.init(filterConfig); // match multiple sub-domains - Assert.assertFalse(filter.areOriginsAllowed("example.com")); - Assert.assertFalse(filter.areOriginsAllowed("foo:example.com")); - Assert.assertTrue(filter.areOriginsAllowed("foo.example.com")); - Assert.assertTrue(filter.areOriginsAllowed("foo.bar.example.com")); + Assertions.assertFalse(filter.areOriginsAllowed("example.com")); + Assertions.assertFalse(filter.areOriginsAllowed("foo:example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.bar.example.com")); // First origin is allowed - Assert.assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com")); // Second origin is allowed - Assert.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com")); // No origin in list is allowed - Assert.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); + Assertions.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); } @Test @@ -164,13 +164,13 @@ public void testComplexRegexPatternMatchingOrigins() throws ServletException, IO CrossOriginFilter filter = new CrossOriginFilter(); filter.init(filterConfig); - Assert.assertTrue(filter.areOriginsAllowed("http://sub1.example.com")); - Assert.assertTrue(filter.areOriginsAllowed("https://sub1.example.com")); - Assert.assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234")); - Assert.assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080")); + Assertions.assertTrue(filter.areOriginsAllowed("http://sub1.example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("https://sub1.example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234")); + Assertions.assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080")); // No origin in list is allowed - Assert.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); + Assertions.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); } @Test @@ -186,23 +186,23 @@ public void testMixedRegexPatternMatchingOrigins() throws ServletException, IOEx CrossOriginFilter filter = new CrossOriginFilter(); filter.init(filterConfig); - Assert.assertTrue(filter.areOriginsAllowed("http://sub1.example.com")); - Assert.assertTrue(filter.areOriginsAllowed("https://sub1.example.com")); - Assert.assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234")); - Assert.assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080")); + Assertions.assertTrue(filter.areOriginsAllowed("http://sub1.example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("https://sub1.example.com")); + Assertions.assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234")); + Assertions.assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080")); // match multiple sub-domains - Assert.assertFalse(filter.areOriginsAllowed("example2.com")); - Assert.assertFalse(filter.areOriginsAllowed("foo:example2.com")); - Assert.assertTrue(filter.areOriginsAllowed("foo.example2.com")); - Assert.assertTrue(filter.areOriginsAllowed("foo.bar.example2.com")); + Assertions.assertFalse(filter.areOriginsAllowed("example2.com")); + Assertions.assertFalse(filter.areOriginsAllowed("foo:example2.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.example2.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.bar.example2.com")); // First origin is allowed - Assert.assertTrue(filter.areOriginsAllowed("foo.example2.com foo.nomatch.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.example2.com foo.nomatch.com")); // Second origin is allowed - Assert.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example2.com")); + Assertions.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example2.com")); // No origin in list is allowed - Assert.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); + Assertions.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); } @Test @@ -343,13 +343,13 @@ public void testCrossOriginFilterAfterRestart() throws ServletException { filter.init(filterConfig); //verify filter values - Assert.assertTrue("Allowed headers do not match", - filter.getAllowedHeadersHeader() - .compareTo("X-Requested-With,Accept") == 0); - Assert.assertTrue("Allowed methods do not match", - filter.getAllowedMethodsHeader() - .compareTo("GET,POST") == 0); - Assert.assertTrue(filter.areOriginsAllowed("example.com")); + Assertions.assertTrue( + filter.getAllowedHeadersHeader() + .compareTo("X-Requested-With,Accept") == 0, "Allowed headers do not match"); + Assertions.assertTrue( + filter.getAllowedMethodsHeader() + .compareTo("GET,POST") == 0, "Allowed methods do not match"); + Assertions.assertTrue(filter.areOriginsAllowed("example.com")); //destroy filter values and clear conf filter.destroy(); @@ -365,13 +365,13 @@ public void testCrossOriginFilterAfterRestart() throws ServletException { filter.init(filterConfig); //verify filter values - Assert.assertTrue("Allowed headers do not match", - filter.getAllowedHeadersHeader() - .compareTo("Content-Type,Origin") == 0); - Assert.assertTrue("Allowed methods do not match", - filter.getAllowedMethodsHeader() - .compareTo("GET,HEAD") == 0); - Assert.assertTrue(filter.areOriginsAllowed("newexample.com")); + Assertions.assertTrue( + filter.getAllowedHeadersHeader() + .compareTo("Content-Type,Origin") == 0, "Allowed headers do not match"); + Assertions.assertTrue( + filter.getAllowedMethodsHeader() + .compareTo("GET,HEAD") == 0, "Allowed methods do not match"); + Assertions.assertTrue(filter.areOriginsAllowed("newexample.com")); //destroy filter values filter.destroy(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestRestCsrfPreventionFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestRestCsrfPreventionFilter.java index b346e615ab142..f39dd1a103b45 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestRestCsrfPreventionFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestRestCsrfPreventionFilter.java @@ -29,7 +29,7 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import static org.apache.hadoop.test.MockitoUtil.verifyZeroInteractions; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java index 0f9f691322e70..497d8d150c25c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java @@ -24,8 +24,8 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -57,10 +57,10 @@ public void testDefaultOptionsValue() throws Exception { @Override public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); - Assert.assertTrue( - "header should be visible inside chain and filters.", - ((HttpServletResponse)args[1]). - containsHeader(X_FRAME_OPTIONS)); + Assertions.assertTrue( + + ((HttpServletResponse)args[1]). + containsHeader(X_FRAME_OPTIONS), "header should be visible inside chain and filters."); return null; } } @@ -71,9 +71,9 @@ public Object answer(InvocationOnMock invocation) throws Throwable { @Override public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); - Assert.assertTrue( - "Options value incorrect should be DENY but is: " - + args[1], "DENY".equals(args[1])); + Assertions.assertTrue( + "DENY".equals(args[1]), "Options value incorrect should be DENY but is: " + + args[1]); headers.add((String)args[1]); return null; } @@ -108,12 +108,12 @@ public void testCustomOptionsValueAndNoOverrides() throws Exception { public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); HttpServletResponse resp = (HttpServletResponse) args[1]; - Assert.assertTrue( - "Header should be visible inside chain and filters.", - resp.containsHeader(X_FRAME_OPTIONS)); + Assertions.assertTrue( + + resp.containsHeader(X_FRAME_OPTIONS), "Header should be visible inside chain and filters."); // let's try and set another value for the header and make // sure that it doesn't overwrite the configured value - Assert.assertTrue(resp instanceof + Assertions.assertTrue(resp instanceof XFrameOptionsFilter.XFrameOptionsResponseWrapper); resp.setHeader(X_FRAME_OPTIONS, "LJM"); return null; @@ -126,9 +126,8 @@ public Object answer(InvocationOnMock invocation) throws Throwable { @Override public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); - Assert.assertEquals( - "Options value incorrect should be SAMEORIGIN but is: " - + args[1], "SAMEORIGIN", args[1]); + Assertions.assertEquals("SAMEORIGIN", args[1], + "Options value incorrect should be SAMEORIGIN but is: " + args[1]); headers.add((String)args[1]); return null; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestDelegatingSSLSocketFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestDelegatingSSLSocketFactory.java index f19f65b18cfe6..f02c7570e28f8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestDelegatingSSLSocketFactory.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestDelegatingSSLSocketFactory.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.util.Arrays; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.util.NativeCodeLoader; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java index a0ce721ecf05b..768b88f9dd832 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java @@ -19,8 +19,9 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.File; import java.io.FileOutputStream; @@ -34,8 +35,7 @@ import java.util.function.Supplier; import static org.apache.hadoop.security.ssl.KeyStoreTestUtil.*; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.*; public class TestReloadingX509KeyManager { @@ -45,37 +45,41 @@ public class TestReloadingX509KeyManager { private final GenericTestUtils.LogCapturer reloaderLog = GenericTestUtils.LogCapturer.captureLogs( FileMonitoringTimerTask.LOG); - @BeforeClass + @BeforeAll public static void setUp() throws Exception { File base = new File(BASEDIR); FileUtil.fullyDelete(base); base.mkdirs(); } - @Test(expected = IOException.class) + @Test public void testLoadMissingKeyStore() throws Exception { - String keystoreLocation = BASEDIR + "/testmissing.jks"; + assertThrows(IOException.class, () -> { + String keystoreLocation = BASEDIR + "/testmissing.jks"; - ReloadingX509KeystoreManager tm = + ReloadingX509KeystoreManager tm = new ReloadingX509KeystoreManager("jks", keystoreLocation, - "password", - "password"); + "password", + "password"); + }); } - @Test(expected = IOException.class) + @Test public void testLoadCorruptKeyStore() throws Exception { - String keystoreLocation = BASEDIR + "/testcorrupt.jks"; - OutputStream os = new FileOutputStream(keystoreLocation); - os.write(1); - os.close(); + assertThrows(IOException.class, () -> { + String keystoreLocation = BASEDIR + "/testcorrupt.jks"; + OutputStream os = new FileOutputStream(keystoreLocation); + os.write(1); + os.close(); - ReloadingX509KeystoreManager tm = + ReloadingX509KeystoreManager tm = new ReloadingX509KeystoreManager("jks", keystoreLocation, - "password", - "password"); + "password", "password"); + }); } - @Test (timeout = 3000000) + @Test + @Timeout(value = 3000) public void testReload() throws Exception { KeyPair kp = generateKeyPair("RSA"); X509Certificate sCert = generateCertificate("CN=localhost, O=server", kp, 30, @@ -114,7 +118,8 @@ public Boolean get() { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testReloadMissingTrustStore() throws Exception { KeyPair kp = generateKeyPair("RSA"); X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); @@ -153,7 +158,8 @@ public void testReloadMissingTrustStore() throws Exception { } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testReloadCorruptTrustStore() throws Exception { KeyPair kp = generateKeyPair("RSA"); X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java index 63589592f35dd..2080f1825fc43 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java @@ -23,8 +23,9 @@ import java.util.function.Supplier; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.File; import java.io.FileOutputStream; @@ -38,11 +39,10 @@ import java.util.Timer; import java.util.concurrent.TimeoutException; -import static org.junit.Assert.assertEquals; import static org.apache.hadoop.security.ssl.KeyStoreTestUtil.createTrustStore; import static org.apache.hadoop.security.ssl.KeyStoreTestUtil.generateCertificate; import static org.apache.hadoop.security.ssl.KeyStoreTestUtil.generateKeyPair; -import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.*; public class TestReloadingX509TrustManager { @@ -54,33 +54,37 @@ public class TestReloadingX509TrustManager { private final LogCapturer reloaderLog = LogCapturer.captureLogs( FileMonitoringTimerTask.LOG); - @BeforeClass + @BeforeAll public static void setUp() throws Exception { File base = new File(BASEDIR); FileUtil.fullyDelete(base); base.mkdirs(); } - @Test(expected = IOException.class) + @Test public void testLoadMissingTrustStore() throws Exception { - String truststoreLocation = BASEDIR + "/testmissing.jks"; - - ReloadingX509TrustManager tm = - new ReloadingX509TrustManager("jks", truststoreLocation, "password"); + assertThrows(IOException.class, () -> { + String truststoreLocation = BASEDIR + "/testmissing.jks"; + ReloadingX509TrustManager tm = + new ReloadingX509TrustManager("jks", truststoreLocation, "password"); + }); } - @Test(expected = IOException.class) + @Test public void testLoadCorruptTrustStore() throws Exception { - String truststoreLocation = BASEDIR + "/testcorrupt.jks"; - OutputStream os = new FileOutputStream(truststoreLocation); - os.write(1); - os.close(); + assertThrows(IOException.class, () -> { + String truststoreLocation = BASEDIR + "/testcorrupt.jks"; + OutputStream os = new FileOutputStream(truststoreLocation); + os.write(1); + os.close(); - ReloadingX509TrustManager tm = - new ReloadingX509TrustManager("jks", truststoreLocation, "password"); + ReloadingX509TrustManager tm = + new ReloadingX509TrustManager("jks", truststoreLocation, "password"); + }); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testReload() throws Exception { KeyPair kp = generateKeyPair("RSA"); cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); @@ -117,7 +121,8 @@ public Boolean get() { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testReloadMissingTrustStore() throws Exception { KeyPair kp = generateKeyPair("RSA"); cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); @@ -157,7 +162,8 @@ public void testReloadMissingTrustStore() throws Exception { } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testReloadCorruptTrustStore() throws Exception { KeyPair kp = generateKeyPair("RSA"); cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java index ece6a05ef5878..89846a04305ea 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java @@ -24,10 +24,7 @@ import static org.apache.hadoop.security.ssl.SSLFactory.Mode.CLIENT; import static org.apache.hadoop.security.ssl.SSLFactory.SSL_CLIENT_CONF_KEY; import static org.apache.hadoop.security.ssl.SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.*; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; @@ -36,11 +33,11 @@ import org.apache.hadoop.security.alias.JavaKeyStoreProvider; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.StringUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -80,7 +77,7 @@ public class TestSSLFactory { "keystorePassword", "keyPassword", "trustStoreLocation", "trustStorePassword"); - @BeforeClass + @BeforeAll public static void setUp() throws Exception { File base = new File(BASEDIR); FileUtil.fullyDelete(base); @@ -96,8 +93,8 @@ private Configuration createConfiguration(boolean clientCert, return conf; } - @After - @Before + @AfterEach + @BeforeEach public void cleanUp() throws Exception { sslConfsDir = KeyStoreTestUtil.getClasspathDir(TestSSLFactory.class); KeyStoreTestUtil.cleanupSSLConfig(KEYSTORES_DIR, sslConfsDir); @@ -152,18 +149,20 @@ public void testSslConfClassPathFirst() throws Exception { assertNotEquals(conf, sslConfLoaded); } - @Test(expected = IllegalStateException.class) + @Test public void clientMode() throws Exception { - Configuration conf = createConfiguration(false, true); - SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); - try { - sslFactory.init(); - Assert.assertNotNull(sslFactory.createSSLSocketFactory()); - Assert.assertNotNull(sslFactory.getHostnameVerifier()); - sslFactory.createSSLServerSocketFactory(); - } finally { - sslFactory.destroy(); - } + assertThrows(IllegalArgumentException.class, ()->{ + Configuration conf = createConfiguration(false, true); + SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); + try { + sslFactory.init(); + Assertions.assertNotNull(sslFactory.createSSLSocketFactory()); + Assertions.assertNotNull(sslFactory.getHostnameVerifier()); + sslFactory.createSSLServerSocketFactory(); + } finally { + sslFactory.destroy(); + } + }); } private void serverMode(boolean clientCert, boolean socket) throws Exception { @@ -171,8 +170,8 @@ private void serverMode(boolean clientCert, boolean socket) throws Exception { SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); try { sslFactory.init(); - Assert.assertNotNull(sslFactory.createSSLServerSocketFactory()); - Assert.assertEquals(clientCert, sslFactory.isClientCertRequired()); + Assertions.assertNotNull(sslFactory.createSSLServerSocketFactory()); + Assertions.assertEquals(clientCert, sslFactory.isClientCertRequired()); if (socket) { sslFactory.createSSLSocketFactory(); } else { @@ -184,24 +183,25 @@ private void serverMode(boolean clientCert, boolean socket) throws Exception { } - @Test(expected = IllegalStateException.class) + @Test public void serverModeWithoutClientCertsSocket() throws Exception { - serverMode(false, true); + assertThrows(IllegalArgumentException.class, + () -> serverMode(false, true)); } - @Test(expected = IllegalStateException.class) + @Test public void serverModeWithClientCertsSocket() throws Exception { - serverMode(true, true); + assertThrows(IllegalArgumentException.class, () -> serverMode(true, true)); } - @Test(expected = IllegalStateException.class) + @Test public void serverModeWithoutClientCertsVerifier() throws Exception { - serverMode(false, false); + assertThrows(IllegalStateException.class, () -> serverMode(false, false)); } - @Test(expected = IllegalStateException.class) + @Test public void serverModeWithClientCertsVerifier() throws Exception { - serverMode(true, false); + assertThrows(IllegalArgumentException.class, ()-> serverMode(true, false)); } private void runDelegatedTasks(SSLEngineResult result, SSLEngine engine) @@ -228,7 +228,7 @@ private static void checkTransfer(ByteBuffer a, ByteBuffer b) throws Exception { a.flip(); b.flip(); - assertTrue("transfer did not complete", a.equals(b)); + assertTrue(a.equals(b), "transfer did not complete"); a.position(a.limit()); b.position(b.limit()); @@ -299,7 +299,7 @@ public void testServerWeakCiphers() throws Exception { dataDone = true; } } - Assert.fail("The exception was not thrown"); + Assertions.fail("The exception was not thrown"); } catch (SSLHandshakeException e) { GenericTestUtils.assertExceptionContains("no cipher suites in common", e); } @@ -326,47 +326,49 @@ public void validHostnameVerifier() throws Exception { SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assert.assertEquals("DEFAULT", sslFactory.getHostnameVerifier().toString()); + Assertions.assertEquals("DEFAULT", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "ALLOW_ALL"); sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assert.assertEquals("ALLOW_ALL", + Assertions.assertEquals("ALLOW_ALL", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "DEFAULT_AND_LOCALHOST"); sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assert.assertEquals("DEFAULT_AND_LOCALHOST", + Assertions.assertEquals("DEFAULT_AND_LOCALHOST", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "STRICT"); sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assert.assertEquals("STRICT", sslFactory.getHostnameVerifier().toString()); + Assertions.assertEquals("STRICT", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "STRICT_IE6"); sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assert.assertEquals("STRICT_IE6", + Assertions.assertEquals("STRICT_IE6", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); } - @Test(expected = GeneralSecurityException.class) + @Test public void invalidHostnameVerifier() throws Exception { - Configuration conf = createConfiguration(false, true); - conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "foo"); - SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); - try { - sslFactory.init(); - } finally { - sslFactory.destroy(); - } + assertThrows(GeneralSecurityException.class, () -> { + Configuration conf = createConfiguration(false, true); + conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "foo"); + SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); + try { + sslFactory.init(); + } finally { + sslFactory.destroy(); + } + }); } @Test @@ -392,10 +394,10 @@ public void testConnectionConfigurator() throws Exception { sslFactory.init(); HttpsURLConnection sslConn = (HttpsURLConnection) new URL("https://foo").openConnection(); - Assert.assertNotSame("STRICT_IE6", + Assertions.assertNotSame("STRICT_IE6", sslConn.getHostnameVerifier().toString()); sslFactory.configure(sslConn); - Assert.assertEquals("STRICT_IE6", + Assertions.assertEquals("STRICT_IE6", sslConn.getHostnameVerifier().toString()); } finally { sslFactory.destroy(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java index 08554fc515bf4..eb68d226bb47e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java @@ -30,13 +30,13 @@ import org.apache.hadoop.security.Credentials; import org.apache.hadoop.test.GenericTestUtils; -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 org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -89,7 +89,7 @@ public class TestDtUtilShell { private DtUtilShell dt = null; private int rc = 0; - @Before + @BeforeEach public void setup() throws Exception { localFs.mkdirs(localFs.makeQualified(workDir)); makeTokenFile(tokenFile, false, null); @@ -102,7 +102,7 @@ public void setup() throws Exception { rc = 0; } - @After + @AfterEach public void teardown() throws Exception { localFs.delete(localFs.makeQualified(workDir), true); } @@ -128,40 +128,40 @@ public void makeTokenFile(Path tokenPath, boolean legacy, Text service) public void testPrint() throws Exception { args = new String[] {"print", tokenFilename}; rc = dt.run(args); - assertEquals("test simple print exit code", 0, rc); - assertTrue("test simple print output kind:\n" + outContent.toString(), - outContent.toString().contains(KIND.toString())); - assertTrue("test simple print output service:\n" + outContent.toString(), - outContent.toString().contains(SERVICE.toString())); + assertEquals(0, rc, "test simple print exit code"); + assertTrue( + outContent.toString().contains(KIND.toString()), "test simple print output kind:\n" + outContent.toString()); + assertTrue( + outContent.toString().contains(SERVICE.toString()), "test simple print output service:\n" + outContent.toString()); outContent.reset(); args = new String[] {"print", tokenLegacyFile.toString()}; rc = dt.run(args); - assertEquals("test legacy print exit code", 0, rc); - assertTrue("test simple print output kind:\n" + outContent.toString(), - outContent.toString().contains(KIND.toString())); - assertTrue("test simple print output service:\n" + outContent.toString(), - outContent.toString().contains(SERVICE.toString())); + assertEquals(0, rc, "test legacy print exit code"); + assertTrue( + outContent.toString().contains(KIND.toString()), "test simple print output kind:\n" + outContent.toString()); + assertTrue( + outContent.toString().contains(SERVICE.toString()), "test simple print output service:\n" + outContent.toString()); outContent.reset(); args = new String[] { "print", "-alias", SERVICE.toString(), tokenFilename}; rc = dt.run(args); - assertEquals("test alias print exit code", 0, rc); - assertTrue("test simple print output kind:\n" + outContent.toString(), - outContent.toString().contains(KIND.toString())); - assertTrue("test simple print output service:\n" + outContent.toString(), - outContent.toString().contains(SERVICE.toString())); + assertEquals(0, rc, "test alias print exit code"); + assertTrue( + outContent.toString().contains(KIND.toString()), "test simple print output kind:\n" + outContent.toString()); + assertTrue( + outContent.toString().contains(SERVICE.toString()), "test simple print output service:\n" + outContent.toString()); outContent.reset(); args = new String[] { "print", "-alias", "not-a-serivce", tokenFilename}; rc = dt.run(args); - assertEquals("test no alias print exit code", 0, rc); - assertFalse("test no alias print output kind:\n" + outContent.toString(), - outContent.toString().contains(KIND.toString())); - assertFalse("test no alias print output service:\n" + outContent.toString(), - outContent.toString().contains(SERVICE.toString())); + assertEquals(0, rc, "test no alias print exit code"); + assertFalse( + outContent.toString().contains(KIND.toString()), "test no alias print output kind:\n" + outContent.toString()); + assertFalse( + outContent.toString().contains(SERVICE.toString()), "test no alias print output service:\n" + outContent.toString()); } @Test @@ -171,66 +171,66 @@ public void testEdit() throws Exception { args = new String[] {"edit", "-service", oldService, "-alias", newAlias, tokenFilename2}; rc = dt.run(args); - assertEquals("test simple edit exit code", 0, rc); + assertEquals(0, rc, "test simple edit exit code"); args = new String[] {"print", "-alias", oldService, tokenFilename2}; rc = dt.run(args); - assertEquals("test simple edit print old exit code", 0, rc); - assertTrue("test simple edit output kind old:\n" + outContent.toString(), - outContent.toString().contains(KIND.toString())); - assertTrue("test simple edit output service old:\n" + outContent.toString(), - outContent.toString().contains(oldService)); + assertEquals(0, rc, "test simple edit print old exit code"); + assertTrue( + outContent.toString().contains(KIND.toString()), "test simple edit output kind old:\n" + outContent.toString()); + assertTrue( + outContent.toString().contains(oldService), "test simple edit output service old:\n" + outContent.toString()); args = new String[] {"print", "-alias", newAlias, tokenFilename2}; rc = dt.run(args); - assertEquals("test simple edit print new exit code", 0, rc); - assertTrue("test simple edit output kind new:\n" + outContent.toString(), - outContent.toString().contains(KIND.toString())); - assertTrue("test simple edit output service new:\n" + outContent.toString(), - outContent.toString().contains(newAlias)); + assertEquals(0, rc, "test simple edit print new exit code"); + assertTrue( + outContent.toString().contains(KIND.toString()), "test simple edit output kind new:\n" + outContent.toString()); + assertTrue( + outContent.toString().contains(newAlias), "test simple edit output service new:\n" + outContent.toString()); } @Test public void testAppend() throws Exception { args = new String[] {"append", tokenFilename, tokenFilename2}; rc = dt.run(args); - assertEquals("test simple append exit code", 0, rc); + assertEquals(0, rc, "test simple append exit code"); args = new String[] {"print", tokenFilename2}; rc = dt.run(args); - assertEquals("test simple append print exit code", 0, rc); - assertTrue("test simple append output kind:\n" + outContent.toString(), - outContent.toString().contains(KIND.toString())); - assertTrue("test simple append output service:\n" + outContent.toString(), - outContent.toString().contains(SERVICE.toString())); - assertTrue("test simple append output service:\n" + outContent.toString(), - outContent.toString().contains(SERVICE2.toString())); + assertEquals(0, rc, "test simple append print exit code"); + assertTrue( + outContent.toString().contains(KIND.toString()), "test simple append output kind:\n" + outContent.toString()); + assertTrue( + outContent.toString().contains(SERVICE.toString()), "test simple append output service:\n" + outContent.toString()); + assertTrue( + outContent.toString().contains(SERVICE2.toString()), "test simple append output service:\n" + outContent.toString()); } @Test public void testRemove() throws Exception { args = new String[] {"remove", "-alias", SERVICE.toString(), tokenFilename}; rc = dt.run(args); - assertEquals("test simple remove exit code", 0, rc); + assertEquals(0, rc, "test simple remove exit code"); args = new String[] {"print", tokenFilename}; rc = dt.run(args); - assertEquals("test simple remove print exit code", 0, rc); - assertFalse("test simple remove output kind:\n" + outContent.toString(), - outContent.toString().contains(KIND.toString())); - assertFalse("test simple remove output service:\n" + outContent.toString(), - outContent.toString().contains(SERVICE.toString())); + assertEquals(0, rc, "test simple remove print exit code"); + assertFalse( + outContent.toString().contains(KIND.toString()), "test simple remove output kind:\n" + outContent.toString()); + assertFalse( + outContent.toString().contains(SERVICE.toString()), "test simple remove output service:\n" + outContent.toString()); } @Test public void testGet() throws Exception { args = new String[] {"get", getUrl, tokenFilenameGet}; rc = dt.run(args); - assertEquals("test mocked get exit code", 0, rc); + assertEquals(0, rc, "test mocked get exit code"); args = new String[] {"print", tokenFilenameGet}; rc = dt.run(args); String oc = outContent.toString(); - assertEquals("test print after get exit code", 0, rc); - assertTrue("test print after get output kind:\n" + oc, - oc.contains(KIND_GET.toString())); - assertTrue("test print after get output service:\n" + oc, - oc.contains(SERVICE_GET.toString())); + assertEquals(0, rc, "test print after get exit code"); + assertTrue( + oc.contains(KIND_GET.toString()), "test print after get output kind:\n" + oc); + assertTrue( + oc.contains(SERVICE_GET.toString()), "test print after get output service:\n" + oc); } @Test @@ -238,39 +238,39 @@ public void testGetWithServiceFlag() throws Exception { args = new String[] {"get", getUrl2, "-service", SERVICE_GET.toString(), tokenFilenameGet}; rc = dt.run(args); - assertEquals("test mocked get with service flag exit code", 0, rc); + assertEquals(0, rc, "test mocked get with service flag exit code"); args = new String[] {"print", tokenFilenameGet}; rc = dt.run(args); String oc = outContent.toString(); - assertEquals("test print after get with service flag exit code", 0, rc); - assertTrue("test print after get with service flag output kind:\n" + oc, - oc.contains(KIND_GET.toString())); - assertTrue("test print after get with service flag output service:\n" + oc, - oc.contains(SERVICE_GET.toString())); + assertEquals(0, rc, "test print after get with service flag exit code"); + assertTrue( + oc.contains(KIND_GET.toString()), "test print after get with service flag output kind:\n" + oc); + assertTrue( + oc.contains(SERVICE_GET.toString()), "test print after get with service flag output service:\n" + oc); } @Test public void testGetWithAliasFlag() throws Exception { args = new String[] {"get", getUrl, "-alias", alias, tokenFilenameGet}; rc = dt.run(args); - assertEquals("test mocked get with alias flag exit code", 0, rc); + assertEquals(0, rc, "test mocked get with alias flag exit code"); args = new String[] {"print", tokenFilenameGet}; rc = dt.run(args); String oc = outContent.toString(); - assertEquals("test print after get with alias flag exit code", 0, rc); - assertTrue("test print after get with alias flag output kind:\n" + oc, - oc.contains(KIND_GET.toString())); - assertTrue("test print after get with alias flag output alias:\n" + oc, - oc.contains(alias)); - assertFalse("test print after get with alias flag output old service:\n" + - oc, oc.contains(SERVICE_GET.toString())); + assertEquals(0, rc, "test print after get with alias flag exit code"); + assertTrue( + oc.contains(KIND_GET.toString()), "test print after get with alias flag output kind:\n" + oc); + assertTrue( + oc.contains(alias), "test print after get with alias flag output alias:\n" + oc); + assertFalse(oc.contains(SERVICE_GET.toString()), "test print after get with alias flag output old service:\n" + + oc); } @Test public void testFormatJavaFlag() throws Exception { args = new String[] {"get", getUrl, "-format", "java", tokenFilenameGet}; rc = dt.run(args); - assertEquals("test mocked get with java format flag exit code", 0, rc); + assertEquals(0, rc, "test mocked get with java format flag exit code"); Credentials creds = new Credentials(); Credentials spyCreds = Mockito.spy(creds); DataInputStream in = new DataInputStream( @@ -284,7 +284,7 @@ public void testFormatProtoFlag() throws Exception { args = new String[] { "get", getUrl, "-format", "protobuf", tokenFilenameGet}; rc = dt.run(args); - assertEquals("test mocked get with protobuf format flag exit code", 0, rc); + assertEquals(0, rc, "test mocked get with protobuf format flag exit code"); Credentials creds = new Credentials(); Credentials spyCreds = Mockito.spy(creds); DataInputStream in = new DataInputStream( @@ -298,17 +298,17 @@ public void testImport() throws Exception { String base64 = IMPORT_TOKEN.encodeToUrlString(); args = new String[] {"import", base64, tokenFilenameImport}; rc = dt.run(args); - assertEquals("test simple import print old exit code", 0, rc); + assertEquals(0, rc, "test simple import print old exit code"); args = new String[] {"print", tokenFilenameImport}; rc = dt.run(args); - assertEquals("test simple import print old exit code", 0, rc); - assertTrue("test print after import output:\n" + outContent, - outContent.toString().contains(KIND_IMPORT.toString())); - assertTrue("test print after import output:\n" + outContent, - outContent.toString().contains(SERVICE_IMPORT.toString())); - assertTrue("test print after simple import output:\n" + outContent, - outContent.toString().contains(base64)); + assertEquals(0, rc, "test simple import print old exit code"); + assertTrue( + outContent.toString().contains(KIND_IMPORT.toString()), "test print after import output:\n" + outContent); + assertTrue( + outContent.toString().contains(SERVICE_IMPORT.toString()), "test print after import output:\n" + outContent); + assertTrue( + outContent.toString().contains(base64), "test print after simple import output:\n" + outContent); } @Test @@ -317,14 +317,14 @@ public void testImportWithAliasFlag() throws Exception { args = new String[] {"import", base64, "-alias", alias, tokenFilenameImport}; rc = dt.run(args); - assertEquals("test import with alias print old exit code", 0, rc); + assertEquals(0, rc, "test import with alias print old exit code"); args = new String[] {"print", tokenFilenameImport}; rc = dt.run(args); - assertEquals("test simple import print old exit code", 0, rc); - assertTrue("test print after import output:\n" + outContent, - outContent.toString().contains(KIND_IMPORT.toString())); - assertTrue("test print after import with alias output:\n" + outContent, - outContent.toString().contains(alias)); + assertEquals(0, rc, "test simple import print old exit code"); + assertTrue( + outContent.toString().contains(KIND_IMPORT.toString()), "test print after import output:\n" + outContent); + assertTrue( + outContent.toString().contains(alias), "test print after import with alias output:\n" + outContent); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestToken.java index 10eacdebc5567..738c263751fd5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestToken.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestToken.java @@ -26,10 +26,10 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier; import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenIdentifier; import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenSecretManager; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; /** Unit tests for Token */ public class TestToken { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java index 225cc658d39ba..61aab71e668a8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java @@ -37,7 +37,7 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong; import org.apache.hadoop.metrics2.lib.MutableRate; import org.apache.hadoop.test.LambdaTestUtils; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; @@ -53,12 +53,13 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.DelegationTokenInformation; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Time; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; public class TestDelegationToken { private static final Logger LOG = @@ -126,7 +127,7 @@ protected void storeNewMasterKey(DelegationKey key) throws IOException { @Override protected void removeStoredMasterKey(DelegationKey key) { isRemoveStoredMasterKeyCalled = true; - Assert.assertFalse(key.equals(allKeys.get(currentId))); + Assertions.assertFalse(key.equals(allKeys.get(currentId))); } @Override @@ -263,10 +264,10 @@ private void shouldThrow(PrivilegedExceptionAction action, Class except) { try { action.run(); - Assert.fail("action did not throw " + except); + Assertions.fail("action did not throw " + except); } catch (Throwable th) { LOG.info("Caught an exception: ", th); - assertEquals("action threw wrong exception", except, th.getClass()); + assertEquals(except, th.getClass(), "action threw wrong exception"); } } @@ -349,7 +350,7 @@ public void testDelegationTokenSecretManager() throws Exception { final Token token = generateDelegationToken( dtSecretManager, "SomeUser", "JobTracker"); - Assert.assertTrue(dtSecretManager.isStoreNewTokenCalled); + Assertions.assertTrue(dtSecretManager.isStoreNewTokenCalled); // Fake renewer should not be able to renew shouldThrow(new PrivilegedExceptionAction() { @Override @@ -359,21 +360,21 @@ public Object run() throws Exception { } }, AccessControlException.class); long time = dtSecretManager.renewToken(token, "JobTracker"); - Assert.assertTrue(dtSecretManager.isUpdateStoredTokenCalled); - assertTrue("renew time is in future", time > Time.now()); + Assertions.assertTrue(dtSecretManager.isUpdateStoredTokenCalled); + assertTrue(time > Time.now(), "renew time is in future"); TestDelegationTokenIdentifier identifier = new TestDelegationTokenIdentifier(); byte[] tokenId = token.getIdentifier(); identifier.readFields(new DataInputStream( new ByteArrayInputStream(tokenId))); - Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier)); + Assertions.assertTrue(null != dtSecretManager.retrievePassword(identifier)); LOG.info("Sleep to expire the token"); Thread.sleep(2000); //Token should be expired try { dtSecretManager.retrievePassword(identifier); //Should not come here - Assert.fail("Token should have expired"); + Assertions.fail("Token should have expired"); } catch (InvalidToken e) { //Success } @@ -411,7 +412,7 @@ public Object run() throws Exception { } }, AccessControlException.class); dtSecretManager.cancelToken(token, "JobTracker"); - Assert.assertTrue(dtSecretManager.isRemoveStoredTokenCalled); + Assertions.assertTrue(dtSecretManager.isRemoveStoredTokenCalled); shouldThrow(new PrivilegedExceptionAction() { @Override public Object run() throws Exception { @@ -424,7 +425,8 @@ public Object run() throws Exception { } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testRollMasterKey() throws Exception { TestDelegationTokenSecretManager dtSecretManager = new TestDelegationTokenSecretManager(800, @@ -439,7 +441,7 @@ public void testRollMasterKey() throws Exception { int prevNumKeys = dtSecretManager.getAllKeys().length; dtSecretManager.rollMasterKey(); - Assert.assertTrue(dtSecretManager.isStoreNewMasterKeyCalled); + Assertions.assertTrue(dtSecretManager.isStoreNewMasterKeyCalled); //after rolling, the length of the keys list must increase int currNumKeys = dtSecretManager.getAllKeys().length; @@ -456,7 +458,7 @@ public void testRollMasterKey() throws Exception { byte[] newPasswd = dtSecretManager.retrievePassword(identifier); //compare the passwords - Assert.assertEquals(oldPasswd, newPasswd); + Assertions.assertEquals(oldPasswd, newPasswd); // wait for keys to expire while(!dtSecretManager.isRemoveStoredMasterKeyCalled) { Thread.sleep(200); @@ -494,7 +496,7 @@ public void testDelegationTokenSelector() throws Exception { //try to select a token with a given service name (created earlier) Token t = ds.selectToken(new Text("MY-SERVICE1"), tokens); - Assert.assertEquals(t, token1); + Assertions.assertEquals(t, token1); } finally { dtSecretManager.stopThreads(); } @@ -532,17 +534,17 @@ public void run() { } Map tokenCache = dtSecretManager .getAllTokens(); - Assert.assertEquals(numTokensPerThread*numThreads, tokenCache.size()); + Assertions.assertEquals(numTokensPerThread*numThreads, tokenCache.size()); Iterator iter = tokenCache.keySet().iterator(); while (iter.hasNext()) { TestDelegationTokenIdentifier id = iter.next(); DelegationTokenInformation info = tokenCache.get(id); - Assert.assertTrue(info != null); + Assertions.assertTrue(info != null); DelegationKey key = dtSecretManager.getKey(id); - Assert.assertTrue(key != null); + Assertions.assertTrue(key != null); byte[] storedPassword = dtSecretManager.retrievePassword(id); byte[] password = dtSecretManager.createPassword(id, key); - Assert.assertTrue(Arrays.equals(password, storedPassword)); + Assertions.assertTrue(Arrays.equals(password, storedPassword)); //verify by secret manager api dtSecretManager.verifyToken(id, password); } @@ -561,10 +563,10 @@ public void testDelegationTokenNullRenewer() throws Exception { "theuser"), null, null); Token token = new Token( dtId, dtSecretManager); - Assert.assertTrue(token != null); + Assertions.assertTrue(token != null); try { dtSecretManager.renewToken(token, ""); - Assert.fail("Renewal must not succeed"); + Assertions.fail("Renewal must not succeed"); } catch (IOException e) { //PASS } @@ -618,8 +620,8 @@ public void testDelegationKeyEqualAndHash() { DelegationKey key1 = new DelegationKey(1111, 2222, "keyBytes".getBytes()); DelegationKey key2 = new DelegationKey(1111, 2222, "keyBytes".getBytes()); DelegationKey key3 = new DelegationKey(3333, 2222, "keyBytes".getBytes()); - Assert.assertEquals(key1, key2); - Assert.assertFalse(key2.equals(key3)); + Assertions.assertEquals(key1, key2); + Assertions.assertFalse(key2.equals(key3)); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java index 25dae7e4fd5cc..c7ab5164edc62 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java @@ -53,16 +53,16 @@ import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.auth.DigestAuthenticationProvider; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; public class TestZKDelegationTokenSecretManager { @@ -82,13 +82,13 @@ public class TestZKDelegationTokenSecretManager { @Rule public Timeout globalTimeout = new Timeout(300000, TimeUnit.MILLISECONDS); - @Before + @BeforeEach public void setup() throws Exception { zkServer = new TestingServer(); zkServer.start(); } - @After + @AfterEach public void tearDown() throws Exception { if (zkServer != null) { zkServer.close(); @@ -136,7 +136,7 @@ public void testMultiNodeOperationsImpl(boolean setZeroRetry) throws Exception { Token token = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token); + Assertions.assertNotNull(token); tm2.verifyToken(token); tm2.renewToken(token, "foo"); tm1.verifyToken(token); @@ -150,7 +150,7 @@ public void testMultiNodeOperationsImpl(boolean setZeroRetry) throws Exception { token = (Token) tm2.createToken( UserGroupInformation.getCurrentUser(), "bar"); - Assert.assertNotNull(token); + Assertions.assertNotNull(token); tm1.verifyToken(token); tm1.renewToken(token, "bar"); tm2.verifyToken(token); @@ -177,15 +177,15 @@ public void testNodeUpAferAWhile() throws Exception { Token token1 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token1); + Assertions.assertNotNull(token1); Token token2 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "bar"); - Assert.assertNotNull(token2); + Assertions.assertNotNull(token2); Token token3 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "boo"); - Assert.assertNotNull(token3); + Assertions.assertNotNull(token3); tm1.verifyToken(token1); tm1.verifyToken(token2); @@ -212,7 +212,7 @@ public void testNodeUpAferAWhile() throws Exception { Token token4 = (Token) tm2.createToken( UserGroupInformation.getCurrentUser(), "xyz"); - Assert.assertNotNull(token4); + Assertions.assertNotNull(token4); tm2.verifyToken(token4); tm1.verifyToken(token4); @@ -253,19 +253,19 @@ public void testMultiNodeCompeteForSeqNum() throws Exception { Token token1 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token1); + Assertions.assertNotNull(token1); AbstractDelegationTokenIdentifier id1 = tm1.getDelegationTokenSecretManager().decodeTokenIdentifier(token1); - Assert.assertEquals( - "Token seq should be the same", 1, id1.getSequenceNumber()); + Assertions.assertEquals( + 1, id1.getSequenceNumber(), "Token seq should be the same"); Token token2 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token2); + Assertions.assertNotNull(token2); AbstractDelegationTokenIdentifier id2 = tm1.getDelegationTokenSecretManager().decodeTokenIdentifier(token2); - Assert.assertEquals( - "Token seq should be the same", 2, id2.getSequenceNumber()); + Assertions.assertEquals( + 2, id2.getSequenceNumber(), "Token seq should be the same"); tm2 = new DelegationTokenManager(conf, new Text("bla")); tm2.init(); @@ -273,19 +273,19 @@ public void testMultiNodeCompeteForSeqNum() throws Exception { Token token3 = (Token) tm2.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token3); + Assertions.assertNotNull(token3); AbstractDelegationTokenIdentifier id3 = tm2.getDelegationTokenSecretManager().decodeTokenIdentifier(token3); - Assert.assertEquals( - "Token seq should be the same", 1001, id3.getSequenceNumber()); + Assertions.assertEquals( + 1001, id3.getSequenceNumber(), "Token seq should be the same"); Token token4 = (Token) tm2.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token4); + Assertions.assertNotNull(token4); AbstractDelegationTokenIdentifier id4 = tm2.getDelegationTokenSecretManager().decodeTokenIdentifier(token4); - Assert.assertEquals( - "Token seq should be the same", 1002, id4.getSequenceNumber()); + Assertions.assertEquals( + 1002, id4.getSequenceNumber(), "Token seq should be the same"); verifyDestroy(tm1, conf); verifyDestroy(tm2, conf); @@ -304,7 +304,7 @@ public void testRenewTokenSingleManager() throws Exception { Token token = (Token) tm1.createToken(UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token); + Assertions.assertNotNull(token); tm1.renewToken(token, "foo"); tm1.verifyToken(token); verifyDestroy(tm1, conf); @@ -324,7 +324,7 @@ public void testCancelTokenSingleManager() throws Exception { Token token = (Token) tm1.createToken(UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token); + Assertions.assertNotNull(token); tm1.cancelToken(token, "foo"); try { verifyTokenFail(tm1, token); @@ -371,7 +371,7 @@ public void testStopThreads() throws Exception { Token token = (Token) tm1.createToken(UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token); + Assertions.assertNotNull(token); tm1.destroy(); } @@ -420,8 +420,8 @@ public List getDefaultAcl() { private void verifyACL(CuratorFramework curatorFramework, String path, ACL expectedACL) throws Exception { List acls = curatorFramework.getACL().forPath(path); - Assert.assertEquals(1, acls.size()); - Assert.assertEquals(expectedACL, acls.get(0)); + Assertions.assertEquals(1, acls.size()); + Assertions.assertEquals(expectedACL, acls.get(0)); } // Since it is possible that there can be a delay for the cancel token message @@ -470,11 +470,11 @@ public void testNodesLoadedAfterRestart() throws Exception { Token token = (Token) tm .createToken(UserGroupInformation.getCurrentUser(), "good"); - Assert.assertNotNull(token); + Assertions.assertNotNull(token); Token cancelled = (Token) tm .createToken(UserGroupInformation.getCurrentUser(), "cancelled"); - Assert.assertNotNull(cancelled); + Assertions.assertNotNull(cancelled); tm.verifyToken(token); tm.verifyToken(cancelled); @@ -511,12 +511,12 @@ public Boolean get() { smNew.decodeTokenIdentifier(cancelled); AbstractDelegationTokenSecretManager.DelegationTokenInformation dtinfo = zksmNew.getTokenInfo(id); - Assert.assertNull("canceled dt should be gone!", dtinfo); + Assertions.assertNull(dtinfo, "canceled dt should be gone!"); // The good token should be loaded on startup, and removed after expiry. id = smNew.decodeTokenIdentifier(token); dtinfo = zksmNew.getTokenInfoFromMemory(id); - Assert.assertNotNull("good dt should be in memory!", dtinfo); + Assertions.assertNotNull(dtinfo, "good dt should be in memory!"); // Wait for the good token to expire. Thread.sleep(5000); @@ -556,7 +556,7 @@ public void testCreatingParentContainersIfNeeded() throws Exception { // Check if the created NameSpace exists. Stat stat = curatorFramework.checkExists().forPath(workingPath); - Assert.assertNotNull(stat); + Assertions.assertNotNull(stat); tm1.destroy(); curatorFramework.close(); @@ -584,7 +584,7 @@ public void testCreateNameSpaceRepeatedly() throws Exception { // Check if the created NameSpace exists. Stat stat = curatorFramework.checkExists().forPath(workingPath); - Assert.assertNotNull(stat); + Assertions.assertNotNull(stat); // Repeated creation will throw NodeExists exception LambdaTestUtils.intercept(KeeperException.class, @@ -624,10 +624,10 @@ public void testMultipleInit() throws Exception { List> futures = executorService.invokeAll( Arrays.asList(tm1Callable, tm2Callable)); for(Future future : futures) { - Assert.assertTrue(future.get()); + Assertions.assertTrue(future.get()); } executorService.shutdownNow(); - Assert.assertTrue(executorService.awaitTermination(1, TimeUnit.SECONDS)); + Assertions.assertTrue(executorService.awaitTermination(1, TimeUnit.SECONDS)); tm1.destroy(); tm2.destroy(); @@ -636,7 +636,7 @@ public void testMultipleInit() throws Exception { // Check if the created NameSpace exists. Stat stat = curatorFramework.checkExists().forPath(workingPath); - Assert.assertNotNull(stat); + Assertions.assertNotNull(stat); curatorFramework.close(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java index bc140fa7b1075..4965d5041f92d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java @@ -30,11 +30,11 @@ import org.apache.hadoop.security.authentication.server.AuthenticationToken; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.Token; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.Timeout; import org.mockito.Mockito; @@ -96,7 +96,7 @@ public AuthenticationToken authenticate(HttpServletRequest request, @Rule public Timeout testTimeout = new Timeout(120000, TimeUnit.MILLISECONDS); - @Before + @BeforeEach public void setUp() throws Exception { Properties conf = new Properties(); @@ -105,7 +105,7 @@ public void setUp() throws Exception { handler.initTokenManager(conf); } - @After + @AfterEach public void cleanUp() { handler.destroy(); } @@ -133,10 +133,10 @@ private void testNonManagementOperation() throws Exception { HttpServletRequest request = Mockito.mock(HttpServletRequest.class); Mockito.when(request.getParameter( DelegationTokenAuthenticator.OP_PARAM)).thenReturn(null); - Assert.assertTrue(handler.managementOperation(null, request, null)); + Assertions.assertTrue(handler.managementOperation(null, request, null)); Mockito.when(request.getParameter( DelegationTokenAuthenticator.OP_PARAM)).thenReturn("CREATE"); - Assert.assertTrue(handler.managementOperation(null, request, null)); + Assertions.assertTrue(handler.managementOperation(null, request, null)); } private void testManagementOperationErrors() throws Exception { @@ -148,7 +148,7 @@ private void testManagementOperationErrors() throws Exception { GETDELEGATIONTOKEN.toString() ); Mockito.when(request.getMethod()).thenReturn("FOO"); - Assert.assertFalse(handler.managementOperation(null, request, response)); + Assertions.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).sendError( Mockito.eq(HttpServletResponse.SC_BAD_REQUEST), Mockito.startsWith("Wrong HTTP method")); @@ -158,7 +158,7 @@ private void testManagementOperationErrors() throws Exception { DelegationTokenAuthenticator.DelegationTokenOperation. GETDELEGATIONTOKEN.getHttpMethod() ); - Assert.assertFalse(handler.managementOperation(null, request, response)); + Assertions.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus( Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED)); Mockito.verify(response).setHeader( @@ -181,7 +181,7 @@ private Token testGetToken(String renewer, Mockito.when(token.getUserName()).thenReturn("user"); Mockito.when(response.getWriter()).thenReturn(new PrintWriter( new StringWriter())); - Assert.assertFalse(handler.managementOperation(token, request, response)); + Assertions.assertFalse(handler.managementOperation(token, request, response)); String queryString = DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" + @@ -197,7 +197,7 @@ private Token testGetToken(String renewer, StringWriter writer = new StringWriter(); PrintWriter pwriter = new PrintWriter(writer); Mockito.when(response.getWriter()).thenReturn(pwriter); - Assert.assertFalse(handler.managementOperation(token, request, response)); + Assertions.assertFalse(handler.managementOperation(token, request, response)); if (renewer == null) { Mockito.verify(token).getUserName(); } else { @@ -209,8 +209,8 @@ private Token testGetToken(String renewer, String responseOutput = writer.toString(); String tokenLabel = DelegationTokenAuthenticator. DELEGATION_TOKEN_JSON; - Assert.assertTrue(responseOutput.contains(tokenLabel)); - Assert.assertTrue(responseOutput.contains( + Assertions.assertTrue(responseOutput.contains(tokenLabel)); + Assertions.assertTrue(responseOutput.contains( DelegationTokenAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON)); ObjectMapper jsonMapper = new ObjectMapper(); Map json = jsonMapper.readValue(responseOutput, Map.class); @@ -221,11 +221,11 @@ private Token testGetToken(String renewer, Token dt = new Token(); dt.decodeFromUrlString(tokenStr); handler.getTokenManager().verifyToken(dt); - Assert.assertEquals(expectedTokenKind, dt.getKind()); + Assertions.assertEquals(expectedTokenKind, dt.getKind()); if (service != null) { - Assert.assertEquals(service, dt.getService().toString()); + Assertions.assertEquals(service, dt.getService().toString()); } else { - Assert.assertEquals(0, dt.getService().getLength()); + Assertions.assertEquals(0, dt.getService().getLength()); } return dt; } @@ -251,7 +251,7 @@ private void testCancelToken(Token token) Mockito.when(request.getMethod()). thenReturn(op.getHttpMethod()); - Assert.assertFalse(handler.managementOperation(null, request, response)); + Assertions.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).sendError( Mockito.eq(HttpServletResponse.SC_BAD_REQUEST), Mockito.contains("requires the parameter [token]")); @@ -262,15 +262,15 @@ private void testCancelToken(Token token) DelegationTokenAuthenticator.TOKEN_PARAM + "=" + token.encodeToUrlString() ); - Assert.assertFalse(handler.managementOperation(null, request, response)); + Assertions.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_OK); try { handler.getTokenManager().verifyToken(token); - Assert.fail(); + Assertions.fail(); } catch (SecretManager.InvalidToken ex) { //NOP } catch (Throwable ex) { - Assert.fail(); + Assertions.fail(); } } @@ -295,7 +295,7 @@ private void testRenewToken(Token dToken, Mockito.when(request.getMethod()). thenReturn(op.getHttpMethod()); - Assert.assertFalse(handler.managementOperation(null, request, response)); + Assertions.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus( Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED)); Mockito.verify(response).setHeader(Mockito.eq( @@ -306,7 +306,7 @@ private void testRenewToken(Token dToken, Mockito.reset(response); AuthenticationToken token = Mockito.mock(AuthenticationToken.class); Mockito.when(token.getUserName()).thenReturn(testRenewer); - Assert.assertFalse(handler.managementOperation(token, request, response)); + Assertions.assertFalse(handler.managementOperation(token, request, response)); Mockito.verify(response).sendError( Mockito.eq(HttpServletResponse.SC_BAD_REQUEST), Mockito.contains("requires the parameter [token]")); @@ -320,10 +320,10 @@ private void testRenewToken(Token dToken, thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" + dToken.encodeToUrlString()); - Assert.assertFalse(handler.managementOperation(token, request, response)); + Assertions.assertFalse(handler.managementOperation(token, request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_OK); pwriter.close(); - Assert.assertTrue(writer.toString().contains("long")); + Assertions.assertTrue(writer.toString().contains("long")); handler.getTokenManager().verifyToken(dToken); } @@ -347,12 +347,12 @@ private void testValidDelegationTokenQueryString() throws Exception { dToken.encodeToUrlString()); AuthenticationToken token = handler.authenticate(request, response); - Assert.assertEquals(UserGroupInformation.getCurrentUser(). + Assertions.assertEquals(UserGroupInformation.getCurrentUser(). getShortUserName(), token.getUserName()); - Assert.assertEquals(0, token.getExpires()); - Assert.assertEquals(handler.getType(), + Assertions.assertEquals(0, token.getExpires()); + Assertions.assertEquals(handler.getType(), token.getType()); - Assert.assertTrue(token.isExpired()); + Assertions.assertTrue(token.isExpired()); } @SuppressWarnings("unchecked") @@ -367,12 +367,12 @@ private void testValidDelegationTokenHeader() throws Exception { dToken.encodeToUrlString()); AuthenticationToken token = handler.authenticate(request, response); - Assert.assertEquals(UserGroupInformation.getCurrentUser(). + Assertions.assertEquals(UserGroupInformation.getCurrentUser(). getShortUserName(), token.getUserName()); - Assert.assertEquals(0, token.getExpires()); - Assert.assertEquals(handler.getType(), + Assertions.assertEquals(0, token.getExpires()); + Assertions.assertEquals(handler.getType(), token.getType()); - Assert.assertTrue(token.isExpired()); + Assertions.assertTrue(token.isExpired()); } private void testInvalidDelegationTokenQueryString() throws Exception { @@ -382,9 +382,9 @@ private void testInvalidDelegationTokenQueryString() throws Exception { DelegationTokenAuthenticator.DELEGATION_PARAM + "=invalid"); StringWriter writer = new StringWriter(); Mockito.when(response.getWriter()).thenReturn(new PrintWriter(writer)); - Assert.assertNull(handler.authenticate(request, response)); + Assertions.assertNull(handler.authenticate(request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_FORBIDDEN); - Assert.assertTrue(writer.toString().contains("AuthenticationException")); + Assertions.assertTrue(writer.toString().contains("AuthenticationException")); } private void testInvalidDelegationTokenHeader() throws Exception { @@ -395,8 +395,8 @@ private void testInvalidDelegationTokenHeader() throws Exception { "invalid"); StringWriter writer = new StringWriter(); Mockito.when(response.getWriter()).thenReturn(new PrintWriter(writer)); - Assert.assertNull(handler.authenticate(request, response)); - Assert.assertTrue(writer.toString().contains("AuthenticationException")); + Assertions.assertNull(handler.authenticate(request, response)); + Assertions.assertTrue(writer.toString().contains("AuthenticationException")); } private String getToken() throws Exception { @@ -413,7 +413,7 @@ private String getToken() throws Exception { Mockito.when(token.getUserName()).thenReturn("user"); Mockito.when(response.getWriter()).thenReturn(new PrintWriter( new StringWriter())); - Assert.assertFalse(handler.managementOperation(token, request, response)); + Assertions.assertFalse(handler.managementOperation(token, request, response)); Mockito.when(request.getQueryString()). thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + @@ -425,7 +425,7 @@ private String getToken() throws Exception { StringWriter writer = new StringWriter(); PrintWriter pwriter = new PrintWriter(writer); Mockito.when(response.getWriter()).thenReturn(pwriter); - Assert.assertFalse(handler.managementOperation(token, request, response)); + Assertions.assertFalse(handler.managementOperation(token, request, response)); Mockito.verify(token).getUserName(); Mockito.verify(response).setStatus(HttpServletResponse.SC_OK); Mockito.verify(response).setContentType(MediaType.APPLICATION_JSON); @@ -433,8 +433,8 @@ private String getToken() throws Exception { String responseOutput = writer.toString(); String tokenLabel = DelegationTokenAuthenticator. DELEGATION_TOKEN_JSON; - Assert.assertTrue(responseOutput.contains(tokenLabel)); - Assert.assertTrue(responseOutput.contains( + Assertions.assertTrue(responseOutput.contains(tokenLabel)); + Assertions.assertTrue(responseOutput.contains( DelegationTokenAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON)); ObjectMapper jsonMapper = new ObjectMapper(); Map json = jsonMapper.readValue(responseOutput, Map.class); @@ -468,7 +468,7 @@ public void testCannotGetTokenUsingToken() throws Exception { StringWriter writer = new StringWriter(); PrintWriter pwriter = new PrintWriter(writer); Mockito.when(response.getWriter()).thenReturn(pwriter); - Assert.assertFalse(handler.managementOperation(null, request, response)); + Assertions.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_UNAUTHORIZED); } @@ -492,7 +492,7 @@ public void testCannotRenewTokenUsingToken() throws Exception { StringWriter writer = new StringWriter(); PrintWriter pwriter = new PrintWriter(writer); Mockito.when(response.getWriter()).thenReturn(pwriter); - Assert.assertFalse(handler.managementOperation(null, request, response)); + Assertions.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_UNAUTHORIZED); } @@ -536,7 +536,7 @@ public void write(String str) { }; Mockito.when(response.getWriter()).thenReturn(printWriterCloseCount); - Assert.assertFalse(noAuthCloseHandler.managementOperation(token, request, + Assertions.assertFalse(noAuthCloseHandler.managementOperation(token, request, response)); } finally { noAuthCloseHandler.destroy(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenManager.java index 38ca11860e408..8836749b2e6a5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenManager.java @@ -25,8 +25,8 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -61,17 +61,17 @@ public void testDTManager() throws Exception { Token token = (Token) tm.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token); + Assertions.assertNotNull(token); tm.verifyToken(token); - Assert.assertTrue(tm.renewToken(token, "foo") > System.currentTimeMillis()); + Assertions.assertTrue(tm.renewToken(token, "foo") > System.currentTimeMillis()); tm.cancelToken(token, "foo"); try { tm.verifyToken(token); - Assert.fail(); + Assertions.fail(); } catch (IOException ex) { //NOP } catch (Exception ex) { - Assert.fail(); + Assertions.fail(); } tm.destroy(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java index 69e252222be84..f40aa867b1403 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java @@ -36,10 +36,10 @@ import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletHolder; import org.slf4j.event.Level; @@ -193,7 +193,7 @@ protected String getJettyURL() { return "http://" + c.getHost() + ":" + c.getLocalPort(); } - @Before + @BeforeEach public void setUp() throws Exception { // resetting hadoop security to simple org.apache.hadoop.conf.Configuration conf = @@ -205,7 +205,7 @@ public void setUp() throws Exception { Level.TRACE); } - @After + @AfterEach public void cleanUp() throws Exception { jetty.stop(); @@ -235,45 +235,45 @@ public void testRawHttpCalls() throws Exception { // unauthenticated access to URL HttpURLConnection conn = (HttpURLConnection) nonAuthURL.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + Assertions.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // authenticated access to URL conn = (HttpURLConnection) authURL.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // unauthenticated access to get delegation token URL url = new URL(nonAuthURL.toExternalForm() + "?op=GETDELEGATIONTOKEN"); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + Assertions.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // authenticated access to get delegation token url = new URL(authURL.toExternalForm() + "&op=GETDELEGATIONTOKEN&renewer=foo"); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ObjectMapper mapper = new ObjectMapper(); Map map = mapper.readValue(conn.getInputStream(), Map.class); String dt = (String) ((Map) map.get("Token")).get("urlString"); - Assert.assertNotNull(dt); + Assertions.assertNotNull(dt); // delegation token access to URL url = new URL(nonAuthURL.toExternalForm() + "?delegation=" + dt); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // delegation token and authenticated access to URL url = new URL(authURL.toExternalForm() + "&delegation=" + dt); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // renewew delegation token, unauthenticated access to URL url = new URL(nonAuthURL.toExternalForm() + "?op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + Assertions.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // renewew delegation token, authenticated access to URL @@ -281,14 +281,14 @@ public void testRawHttpCalls() throws Exception { "&op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // renewew delegation token, authenticated access to URL, not renewer url = new URL(getJettyURL() + "/foo/bar?authenticated=bar&op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, + Assertions.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); // cancel delegation token, nonauthenticated access to URL @@ -296,32 +296,32 @@ public void testRawHttpCalls() throws Exception { "?op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // cancel canceled delegation token, nonauthenticated access to URL url = new URL(nonAuthURL.toExternalForm() + "?op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(HttpURLConnection.HTTP_NOT_FOUND, + Assertions.assertEquals(HttpURLConnection.HTTP_NOT_FOUND, conn.getResponseCode()); // get new delegation token url = new URL(authURL.toExternalForm() + "&op=GETDELEGATIONTOKEN&renewer=foo"); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); mapper = new ObjectMapper(); map = mapper.readValue(conn.getInputStream(), Map.class); dt = (String) ((Map) map.get("Token")).get("urlString"); - Assert.assertNotNull(dt); + Assertions.assertNotNull(dt); // cancel delegation token, authenticated access to URL url = new URL(authURL.toExternalForm() + "&op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); } finally { jetty.stop(); } @@ -364,32 +364,32 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) try { aUrl.getDelegationToken(nonAuthURL, token, FOO_USER); - Assert.fail(); + Assertions.fail(); } catch (Exception ex) { - Assert.assertTrue(ex.getCause().getMessage().contains("401")); + Assertions.assertTrue(ex.getCause().getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, FOO_USER); - Assert.assertNotNull(token.getDelegationToken()); - Assert.assertEquals(new Text("token-kind"), + Assertions.assertNotNull(token.getDelegationToken()); + Assertions.assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); aUrl.renewDelegationToken(authURL, token); try { aUrl.renewDelegationToken(nonAuthURL, token); - Assert.fail(); + Assertions.fail(); } catch (Exception ex) { - Assert.assertTrue(ex.getMessage().contains("401")); + Assertions.assertTrue(ex.getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, FOO_USER); try { aUrl.renewDelegationToken(authURL2, token); - Assert.fail(); + Assertions.fail(); } catch (Exception ex) { - Assert.assertTrue(ex.getMessage().contains("403")); + Assertions.assertTrue(ex.getMessage().contains("403")); } aUrl.getDelegationToken(authURL, token, FOO_USER); @@ -405,7 +405,7 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) try { aUrl.renewDelegationToken(nonAuthURL, token); } catch (Exception ex) { - Assert.assertTrue(ex.getMessage().contains("401")); + Assertions.assertTrue(ex.getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, "foo"); @@ -416,13 +416,13 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) @Override public Void run() throws Exception { HttpURLConnection conn = aUrl.openConnection(nonAuthURL, new DelegationTokenAuthenticatedURL.Token()); - Assert.assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); if (useQS) { - Assert.assertNull(conn.getHeaderField("UsingHeader")); - Assert.assertNotNull(conn.getHeaderField("UsingQueryString")); + Assertions.assertNull(conn.getHeaderField("UsingHeader")); + Assertions.assertNotNull(conn.getHeaderField("UsingQueryString")); } else { - Assert.assertNotNull(conn.getHeaderField("UsingHeader")); - Assert.assertNull(conn.getHeaderField("UsingQueryString")); + Assertions.assertNotNull(conn.getHeaderField("UsingHeader")); + Assertions.assertNull(conn.getHeaderField("UsingQueryString")); } return null; } @@ -472,8 +472,8 @@ public void testExternalDelegationTokenSecretManager() throws Exception { new DelegationTokenAuthenticatedURL(); aUrl.getDelegationToken(authURL, token, FOO_USER); - Assert.assertNotNull(token.getDelegationToken()); - Assert.assertEquals(new Text("fooKind"), + Assertions.assertNotNull(token.getDelegationToken()); + Assertions.assertEquals(new Text("fooKind"), token.getDelegationToken().getKind()); } finally { @@ -553,17 +553,17 @@ public Void run() throws Exception { DelegationTokenAuthenticatedURL aUrl = new DelegationTokenAuthenticatedURL(); HttpURLConnection conn = aUrl.openConnection(url, token); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals(FOO_USER, ret.get(0)); + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals(FOO_USER, ret.get(0)); try { aUrl.getDelegationToken(url, token, FOO_USER); - Assert.fail(); + Assertions.fail(); } catch (AuthenticationException ex) { - Assert.assertTrue(ex.getMessage().contains( + Assertions.assertTrue(ex.getMessage().contains( "delegation token operation")); } return null; @@ -623,15 +623,15 @@ public Void run() throws Exception { DelegationTokenAuthenticatedURL aUrl = new DelegationTokenAuthenticatedURL(); HttpURLConnection conn = aUrl.openConnection(url, token); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals(FOO_USER, ret.get(0)); + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals(FOO_USER, ret.get(0)); aUrl.getDelegationToken(url, token, FOO_USER); - Assert.assertNotNull(token.getDelegationToken()); - Assert.assertEquals(new Text("token-kind"), + Assertions.assertNotNull(token.getDelegationToken()); + Assertions.assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); return null; } @@ -746,7 +746,7 @@ private void testKerberosDelegationTokenAuthenticator( final String doAsUser = doAs ? OK_USER : null; File testDir = new File("target/" + UUID.randomUUID().toString()); - Assert.assertTrue(testDir.mkdirs()); + Assertions.assertTrue(testDir.mkdirs()); MiniKdc kdc = new MiniKdc(MiniKdc.createConf(), testDir); final Server jetty = createJettyServer(); ServletContextHandler context = new ServletContextHandler(); @@ -774,9 +774,9 @@ private void testKerberosDelegationTokenAuthenticator( try { aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); - Assert.fail(); + Assertions.fail(); } catch (AuthenticationException ex) { - Assert.assertTrue(ex.getCause().getMessage().contains("GSSException")); + Assertions.assertTrue(ex.getCause().getMessage().contains("GSSException")); } doAsKerberosUser("client", keytabFile.getAbsolutePath(), @@ -785,8 +785,8 @@ private void testKerberosDelegationTokenAuthenticator( public Void call() throws Exception { aUrl.getDelegationToken( url, token, doAs ? doAsUser : "client", doAsUser); - Assert.assertNotNull(token.getDelegationToken()); - Assert.assertEquals(new Text("token-kind"), + Assertions.assertNotNull(token.getDelegationToken()); + Assertions.assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); // Make sure the token belongs to the right owner ByteArrayInputStream buf = new ByteArrayInputStream( @@ -796,29 +796,29 @@ public Void call() throws Exception { new DelegationTokenIdentifier(new Text("token-kind")); id.readFields(dis); dis.close(); - Assert.assertEquals( + Assertions.assertEquals( doAs ? new Text(OK_USER) : new Text("client"), id.getOwner()); if (doAs) { - Assert.assertEquals(new Text("client"), id.getRealUser()); + Assertions.assertEquals(new Text("client"), id.getRealUser()); } aUrl.renewDelegationToken(url, token, doAsUser); - Assert.assertNotNull(token.getDelegationToken()); + Assertions.assertNotNull(token.getDelegationToken()); aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); - Assert.assertNotNull(token.getDelegationToken()); + Assertions.assertNotNull(token.getDelegationToken()); try { aUrl.renewDelegationToken(url, token, doAsUser); - Assert.fail(); + Assertions.fail(); } catch (Exception ex) { - Assert.assertTrue(ex.getMessage().contains("403")); + Assertions.assertTrue(ex.getMessage().contains("403")); } aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); aUrl.cancelDelegationToken(url, token, doAsUser); - Assert.assertNull(token.getDelegationToken()); + Assertions.assertNull(token.getDelegationToken()); return null; } @@ -848,17 +848,17 @@ public void testProxyUser() throws Exception { url.toExternalForm(), FOO_USER, OK_USER); HttpURLConnection conn = (HttpURLConnection) new URL(strUrl).openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals(OK_USER, ret.get(0)); + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals(OK_USER, ret.get(0)); strUrl = String.format("%s?user.name=%s&DOAS=%s", url.toExternalForm(), FOO_USER, OK_USER); conn = (HttpURLConnection) new URL(strUrl).openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals(OK_USER, ret.get(0)); + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals(OK_USER, ret.get(0)); UserGroupInformation ugi = UserGroupInformation.createRemoteUser(FOO_USER); ugi.doAs(new PrivilegedExceptionAction() { @@ -871,16 +871,16 @@ public Void run() throws Exception { // proxyuser using authentication handler authentication HttpURLConnection conn = aUrl.openConnection(url, token, OK_USER); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals(OK_USER, ret.get(0)); + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals(OK_USER, ret.get(0)); // unauthorized proxy user using authentication handler authentication conn = aUrl.openConnection(url, token, FAIL_USER); - Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, + Assertions.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); // proxy using delegation token authentication @@ -892,12 +892,12 @@ public Void run() throws Exception { // requests using delegation token as auth do not honor doAs conn = aUrl.openConnection(url, token, OK_USER); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals(FOO_USER, ret.get(0)); + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals(FOO_USER, ret.get(0)); return null; } @@ -954,21 +954,21 @@ public Void run() throws Exception { // user foo HttpURLConnection conn = aUrl.openConnection(url, token); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals("remoteuser=" + FOO_USER+ ":ugi=" + FOO_USER, + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals("remoteuser=" + FOO_USER+ ":ugi=" + FOO_USER, ret.get(0)); // user ok-user via proxyuser foo conn = aUrl.openConnection(url, token, OK_USER); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + ":ugi=" + OK_USER, ret.get(0)); return null; @@ -1016,11 +1016,11 @@ public Void run() throws Exception { // user ok-user via proxyuser foo HttpURLConnection conn = aUrl.openConnection(url, token, OK_USER); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(1, ret.size()); - Assert.assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + + Assertions.assertEquals(1, ret.size()); + Assertions.assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + ":ugi=" + OK_USER, ret.get(0)); return null; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/ServiceAssert.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/ServiceAssert.java index 9f7250d129888..e341e5993a3d2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/ServiceAssert.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/ServiceAssert.java @@ -19,12 +19,12 @@ package org.apache.hadoop.service; import org.apache.hadoop.service.Service; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; /** * A set of assertions about the state of any service */ -public class ServiceAssert extends Assert { +public class ServiceAssert extends Assertions { public static void assertServiceStateCreated(Service service) { assertServiceInState(service, Service.STATE.NOTINITED); @@ -43,9 +43,9 @@ public static void assertServiceStateStopped(Service service) { } public static void assertServiceInState(Service service, Service.STATE state) { - assertNotNull("Null service", service); - assertEquals("Service in wrong state: " + service, state, - service.getServiceState()); + assertNotNull(service, "Null service"); + assertEquals(state +, service.getServiceState(), "Service in wrong state: " + service); } /** @@ -58,7 +58,7 @@ public static void assertServiceInState(Service service, Service.STATE state) { public static void assertStateCount(BreakableService service, Service.STATE state, int expected) { - assertNotNull("Null service", service); + assertNotNull(service, "Null service"); int actual = service.getCount(state); if (expected != actual) { fail("Expected entry count for state [" + state +"] of " + service @@ -74,7 +74,7 @@ public static void assertStateCount(BreakableService service, */ public static void assertServiceConfigurationContains(Service service, String key) { - assertNotNull("No option "+ key + " in service configuration", - service.getConfig().get(key)); + assertNotNull( + service.getConfig().get(key), "No option "+ key + " in service configuration"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestCompositeService.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestCompositeService.java index ad3dfcf0c5d38..f518c669e108c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestCompositeService.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestCompositeService.java @@ -20,15 +20,16 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.Service.STATE; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -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.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; public class TestCompositeService { @@ -47,7 +48,7 @@ public class TestCompositeService { private static final boolean STOP_ONLY_STARTED_SERVICES = CompositeServiceImpl.isPolicyToStopOnlyStartedServices(); - @Before + @BeforeEach public void setup() { CompositeServiceImpl.resetCounter(); } @@ -65,8 +66,8 @@ public void testCallSequence() { CompositeServiceImpl[] services = serviceManager.getServices().toArray( new CompositeServiceImpl[0]); - assertEquals("Number of registered services ", NUM_OF_SERVICES, - services.length); + assertEquals(NUM_OF_SERVICES +, services.length, "Number of registered services "); Configuration conf = new Configuration(); // Initialise the composite service @@ -77,9 +78,9 @@ public void testCallSequence() { // Verify the init() call sequence numbers for every service for (int i = 0; i < NUM_OF_SERVICES; i++) { - assertEquals("For " + services[i] - + " service, init() call sequence number should have been ", i, - services[i].getCallSequenceNumber()); + assertEquals(i +, services[i].getCallSequenceNumber(), "For " + services[i] + + " service, init() call sequence number should have been "); } // Reset the call sequence numbers @@ -91,9 +92,9 @@ public void testCallSequence() { // Verify the start() call sequence numbers for every service for (int i = 0; i < NUM_OF_SERVICES; i++) { - assertEquals("For " + services[i] - + " service, start() call sequence number should have been ", i, - services[i].getCallSequenceNumber()); + assertEquals(i +, services[i].getCallSequenceNumber(), "For " + services[i] + + " service, start() call sequence number should have been "); } resetServices(services); @@ -104,18 +105,18 @@ public void testCallSequence() { // Verify the stop() call sequence numbers for every service for (int i = 0; i < NUM_OF_SERVICES; i++) { - assertEquals("For " + services[i] - + " service, stop() call sequence number should have been ", - ((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber()); + assertEquals( + ((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber(), "For " + services[i] + + " service, stop() call sequence number should have been "); } // Try to stop again. This should be a no-op. serviceManager.stop(); // Verify that stop() call sequence numbers for every service don't change. for (int i = 0; i < NUM_OF_SERVICES; i++) { - assertEquals("For " + services[i] - + " service, stop() call sequence number should have been ", - ((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber()); + assertEquals( + ((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber(), "For " + services[i] + + " service, stop() call sequence number should have been "); } } @@ -155,11 +156,11 @@ public void testServiceStartup() { for (int i = 0; i < NUM_OF_SERVICES - 1; i++) { if (i >= FAILED_SERVICE_SEQ_NUMBER && STOP_ONLY_STARTED_SERVICES) { // Failed service state should be INITED - assertEquals("Service state should have been ", STATE.INITED, - services[NUM_OF_SERVICES - 1].getServiceState()); + assertEquals(STATE.INITED +, services[NUM_OF_SERVICES - 1].getServiceState(), "Service state should have been "); } else { - assertEquals("Service state should have been ", STATE.STOPPED, - services[i].getServiceState()); + assertEquals(STATE.STOPPED +, services[i].getServiceState(), "Service state should have been "); } } @@ -223,10 +224,10 @@ private void assertInState(STATE expected, } private void assertInState(STATE expected, Service service) { - assertEquals("Service state should have been " + expected + " in " - + service, - expected, - service.getServiceState()); + assertEquals( + expected +, service.getServiceState(), "Service state should have been " + expected + " in " + + service); } /** @@ -313,24 +314,25 @@ public void testAddServiceInInit() throws Throwable { assertInState(STATE.INITED, child); } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testAddIfService() { CompositeService testService = new CompositeService("TestService") { Service service; @Override public void serviceInit(Configuration conf) { Integer notAService = new Integer(0); - assertFalse("Added an integer as a service", - addIfService(notAService)); + assertFalse( + addIfService(notAService), "Added an integer as a service"); service = new AbstractService("Service") {}; - assertTrue("Unable to add a service", addIfService(service)); + assertTrue(addIfService(service), "Unable to add a service"); } }; testService.init(new Configuration()); - assertEquals("Incorrect number of services", - 1, testService.getServices().size()); + assertEquals( + 1, testService.getServices().size(), "Incorrect number of services"); } @Test @@ -339,8 +341,8 @@ public void testRemoveService() { @Override public void serviceInit(Configuration conf) { Integer notAService = new Integer(0); - assertFalse("Added an integer as a service", - addIfService(notAService)); + assertFalse( + addIfService(notAService), "Added an integer as a service"); Service service1 = new AbstractService("Service1") {}; addIfService(service1); @@ -356,15 +358,16 @@ public void serviceInit(Configuration conf) { }; testService.init(new Configuration()); - assertEquals("Incorrect number of services", - 2, testService.getServices().size()); + assertEquals( + 2, testService.getServices().size(), "Incorrect number of services"); } // // Tests for adding child service to parent // - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddUninitedChildBeforeInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -377,7 +380,8 @@ public void testAddUninitedChildBeforeInit() throws Throwable { assertInState(STATE.STOPPED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddUninitedChildInInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -393,11 +397,12 @@ public void testAddUninitedChildInInit() throws Throwable { assertInState(STATE.NOTINITED, child); parent.stop(); assertInState(STATE.NOTINITED, child); - assertEquals("Incorrect number of services", - 1, parent.getServices().size()); + assertEquals( + 1, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddUninitedChildInStart() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -409,7 +414,8 @@ public void testAddUninitedChildInStart() throws Throwable { assertInState(STATE.NOTINITED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddUninitedChildInStop() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -420,7 +426,8 @@ public void testAddUninitedChildInStop() throws Throwable { assertInState(STATE.NOTINITED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddInitedChildBeforeInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -434,7 +441,8 @@ public void testAddInitedChildBeforeInit() throws Throwable { assertInState(STATE.STOPPED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddInitedChildInInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -447,7 +455,8 @@ public void testAddInitedChildInInit() throws Throwable { assertInState(STATE.STOPPED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddInitedChildInStart() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -460,7 +469,8 @@ public void testAddInitedChildInStart() throws Throwable { assertInState(STATE.STOPPED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddInitedChildInStop() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -472,7 +482,8 @@ public void testAddInitedChildInStop() throws Throwable { assertInState(STATE.INITED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStartedChildBeforeInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -486,11 +497,12 @@ public void testAddStartedChildBeforeInit() throws Throwable { //expected } parent.stop(); - assertEquals("Incorrect number of services", - 1, parent.getServices().size()); + assertEquals( + 1, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStartedChildInInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -504,7 +516,8 @@ public void testAddStartedChildInInit() throws Throwable { assertInState(STATE.STOPPED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStartedChildInStart() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -518,7 +531,8 @@ public void testAddStartedChildInStart() throws Throwable { assertInState(STATE.STOPPED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStartedChildInStop() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -531,7 +545,8 @@ public void testAddStartedChildInStop() throws Throwable { assertInState(STATE.STARTED, child); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStoppedChildBeforeInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -546,11 +561,12 @@ public void testAddStoppedChildBeforeInit() throws Throwable { //expected } parent.stop(); - assertEquals("Incorrect number of services", - 1, parent.getServices().size()); + assertEquals( + 1, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStoppedChildInInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -567,11 +583,12 @@ public void testAddStoppedChildInInit() throws Throwable { } assertInState(STATE.STOPPED, child); parent.stop(); - assertEquals("Incorrect number of services", - 1, parent.getServices().size()); + assertEquals( + 1, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStoppedChildInStart() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -584,7 +601,8 @@ public void testAddStoppedChildInStart() throws Throwable { parent.stop(); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStoppedChildInStop() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService child = new BreakableService(); @@ -601,7 +619,8 @@ public void testAddStoppedChildInStop() throws Throwable { // Tests for adding sibling service to parent // - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddUninitedSiblingBeforeInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -614,11 +633,12 @@ public void testAddUninitedSiblingBeforeInit() throws Throwable { assertInState(STATE.NOTINITED, sibling); parent.stop(); assertInState(STATE.NOTINITED, sibling); - assertEquals("Incorrect number of services", - 1, parent.getServices().size()); + assertEquals( + 1, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddUninitedSiblingInInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -633,11 +653,12 @@ public void testAddUninitedSiblingInInit() throws Throwable { //expected } parent.stop(); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddUninitedSiblingInStart() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -650,11 +671,12 @@ public void testAddUninitedSiblingInStart() throws Throwable { assertInState(STATE.NOTINITED, sibling); parent.stop(); assertInState(STATE.NOTINITED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddUninitedSiblingInStop() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -667,11 +689,12 @@ public void testAddUninitedSiblingInStop() throws Throwable { assertInState(STATE.NOTINITED, sibling); parent.stop(); assertInState(STATE.NOTINITED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddInitedSiblingBeforeInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -685,11 +708,12 @@ public void testAddInitedSiblingBeforeInit() throws Throwable { assertInState(STATE.INITED, sibling); parent.stop(); assertInState(STATE.INITED, sibling); - assertEquals("Incorrect number of services", - 1, parent.getServices().size()); + assertEquals( + 1, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddInitedSiblingInInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -703,11 +727,12 @@ public void testAddInitedSiblingInInit() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddInitedSiblingInStart() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -721,11 +746,12 @@ public void testAddInitedSiblingInStart() throws Throwable { assertInState(STATE.INITED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddInitedSiblingInStop() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -736,7 +762,8 @@ public void testAddInitedSiblingInStop() throws Throwable { parent.init(new Configuration()); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStartedSiblingBeforeInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -751,11 +778,12 @@ public void testAddStartedSiblingBeforeInit() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STARTED, sibling); - assertEquals("Incorrect number of services", - 1, parent.getServices().size()); + assertEquals( + 1, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStartedSiblingInInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -770,12 +798,13 @@ public void testAddStartedSiblingInInit() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStartedSiblingInStart() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -790,11 +819,12 @@ public void testAddStartedSiblingInStart() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStartedSiblingInStop() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -809,11 +839,12 @@ public void testAddStartedSiblingInStop() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STARTED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStoppedSiblingBeforeInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -829,11 +860,12 @@ public void testAddStoppedSiblingBeforeInit() throws Throwable { assertInState(STATE.STOPPED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals("Incorrect number of services", - 1, parent.getServices().size()); + assertEquals( + 1, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStoppedSiblingInInit() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -853,11 +885,12 @@ public void testAddStoppedSiblingInInit() throws Throwable { } parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStoppedSiblingInStart() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -873,11 +906,12 @@ public void testAddStoppedSiblingInStart() throws Throwable { assertInState(STATE.STOPPED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAddStoppedSiblingInStop() throws Throwable { CompositeService parent = new CompositeService("parent"); BreakableService sibling = new BreakableService(); @@ -893,8 +927,8 @@ public void testAddStoppedSiblingInStop() throws Throwable { assertInState(STATE.STOPPED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals("Incorrect number of services", - 2, parent.getServices().size()); + assertEquals( + 2, parent.getServices().size(), "Incorrect number of services"); } public static class CompositeServiceAddingAChild extends CompositeService{ diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestGlobalStateChangeListener.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestGlobalStateChangeListener.java index 7bee2d68c6d18..7d9b9e66d7b8e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestGlobalStateChangeListener.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestGlobalStateChangeListener.java @@ -23,8 +23,8 @@ import org.apache.hadoop.service.LoggingStateChangeListener; import org.apache.hadoop.service.Service; import org.apache.hadoop.service.ServiceStateChangeListener; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; /** * Test global state changes. It is critical for all tests to clean up the @@ -58,7 +58,7 @@ private boolean unregister(ServiceStateChangeListener l) { /** * After every test case reset the list of global listeners. */ - @After + @AfterEach public void cleanup() { AbstractService.resetGlobalListeners(); } @@ -70,7 +70,7 @@ public void cleanup() { */ public void assertListenerState(BreakableStateChangeListener breakable, Service.STATE state) { - assertEquals("Wrong state in " + breakable, state, breakable.getLastState()); + assertEquals(state, breakable.getLastState(), "Wrong state in " + breakable); } /** @@ -80,8 +80,8 @@ public void assertListenerState(BreakableStateChangeListener breakable, */ public void assertListenerEventCount(BreakableStateChangeListener breakable, int count) { - assertEquals("Wrong event count in " + breakable, count, - breakable.getEventCount()); + assertEquals(count +, breakable.getEventCount(), "Wrong event count in " + breakable); } /** @@ -90,7 +90,7 @@ public void assertListenerEventCount(BreakableStateChangeListener breakable, @Test public void testRegisterListener() { register(); - assertTrue("listener not registered", unregister()); + assertTrue(unregister(), "listener not registered"); } /** @@ -100,9 +100,9 @@ public void testRegisterListener() { public void testRegisterListenerTwice() { register(); register(); - assertTrue("listener not registered", unregister()); + assertTrue(unregister(), "listener not registered"); //there should be no listener to unregister the second time - assertFalse("listener double registered", unregister()); + assertFalse(unregister(), "listener double registered"); } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceLifecycle.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceLifecycle.java index f72e130d75011..ccbc0a009fbf5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceLifecycle.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceLifecycle.java @@ -25,7 +25,7 @@ import org.apache.hadoop.service.Service; import org.apache.hadoop.service.ServiceStateChangeListener; import org.apache.hadoop.service.ServiceStateException; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -204,10 +204,10 @@ public void testStopFailingInitAndStop() throws Throwable { assertEquals(Service.STATE.INITED, svc.getFailureState()); Throwable failureCause = svc.getFailureCause(); - assertNotNull("Null failure cause in " + svc, failureCause); + assertNotNull(failureCause, "Null failure cause in " + svc); BreakableService.BrokenLifecycleEvent cause = (BreakableService.BrokenLifecycleEvent) failureCause; - assertNotNull("null state in " + cause + " raised by " + svc, cause.state); + assertNotNull(cause.state, "null state in " + cause + " raised by " + svc); assertEquals(Service.STATE.INITED, cause.state); } @@ -299,7 +299,7 @@ public synchronized void stateChanged(Service service) { private void assertEventCount(BreakableStateChangeListener listener, int expected) { - assertEquals(listener.toString(), expected, listener.getEventCount()); + assertEquals(expected, listener.getEventCount(), listener.toString()); } @Test @@ -343,7 +343,7 @@ public void testListenerWithNotifications() throws Throwable { long duration = System.currentTimeMillis() - start; assertEquals(Service.STATE.STOPPED, listener.notifyingState); assertServiceInState(service, Service.STATE.STOPPED); - assertTrue("Duration of " + duration + " too long", duration < 10000); + assertTrue(duration < 10000, "Duration of " + duration + " too long"); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceOperations.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceOperations.java index b7b86b7aa0dc0..e7596f2a45a3a 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceOperations.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceOperations.java @@ -19,7 +19,7 @@ package org.apache.hadoop.service; import org.apache.hadoop.test.GenericTestUtils.LogCapturer; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.Mockito; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java index 4be670d4638d1..6ac07a6942767 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java @@ -25,10 +25,10 @@ import static org.apache.hadoop.test.GenericTestUtils.*; import org.apache.hadoop.util.ExitCodeProvider; import org.apache.hadoop.util.ExitUtil; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; import org.junit.Rule; import org.junit.rules.TestName; import org.junit.rules.Timeout; @@ -43,7 +43,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; -public class AbstractServiceLauncherTestBase extends Assert implements +public class AbstractServiceLauncherTestBase extends Assertions implements LauncherExitCodes { private static final Logger LOG = LoggerFactory.getLogger( AbstractServiceLauncherTestBase.class); @@ -69,7 +69,7 @@ public class AbstractServiceLauncherTestBase extends Assert implements /** * Turn off the exit util JVM exits, downgrading them to exception throws. */ - @BeforeClass + @BeforeAll public static void disableJVMExits() { ExitUtil.disableSystemExit(); ExitUtil.disableSystemHalt(); @@ -78,12 +78,12 @@ public static void disableJVMExits() { /** * rule to name the thread JUnit. */ - @Before + @BeforeEach public void nameThread() { Thread.currentThread().setName("JUnit"); } - @After + @AfterEach public void stopService() { ServiceOperations.stopQuietly(serviceToTeardown); } @@ -209,7 +209,7 @@ protected String configFile(Configuration conf) throws IOException { */ protected Configuration newConf(String... kvp) { int len = kvp.length; - assertEquals("unbalanced keypair len of " + len, 0, len % 2); + assertEquals(0, len % 2, "unbalanced keypair len of " + len); Configuration conf = new Configuration(false); for (int i = 0; i < len; i += 2) { conf.set(kvp[i], kvp[i + 1]); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java index 6eb6372dcd928..668a523530d14 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java @@ -23,7 +23,7 @@ import org.apache.hadoop.service.launcher.testservices.LaunchableRunningService; import org.apache.hadoop.service.launcher.testservices.RunningService; import static org.apache.hadoop.service.launcher.LauncherArguments.*; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.FileWriter; @@ -97,8 +97,8 @@ public void testConfExtraction() throws Throwable { List args = launcher.extractCommandOptions(extracted, argsList); if (!args.isEmpty()) { - assertEquals("args beginning with " + args.get(0), - 0, args.size()); + assertEquals( + 0, args.size(), "args beginning with " + args.get(0)); } assertEquals("true", extracted.get("propagated", "unset")); } @@ -121,8 +121,8 @@ ARG_CONF_PREFIXED, configFile(conf1), List args = launcher.extractCommandOptions(extracted, argsList); if (!args.isEmpty()) { - assertEquals("args beginning with " + args.get(0), - 0, args.size()); + assertEquals( + 0, args.size(), "args beginning with " + args.get(0)); } assertTrue(extracted.getBoolean(key1, false)); assertEquals(7, extracted.getInt(key2, -1)); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java index 8181e07fae01f..b25dc46e1ae45 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java @@ -22,7 +22,7 @@ import org.apache.hadoop.service.launcher.testservices.FailureTestService; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.ExitUtil; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,10 +68,10 @@ public void testInterruptEscalationShutdown() throws Throwable { } //the service is now stopped assertStopped(service); - assertTrue("isSignalAlreadyReceived() == false in " + escalator, - escalator.isSignalAlreadyReceived()); - assertFalse("isForcedShutdownTimedOut() == true in " + escalator, - escalator.isForcedShutdownTimedOut()); + assertTrue( + escalator.isSignalAlreadyReceived(), "isSignalAlreadyReceived() == false in " + escalator); + assertFalse( + escalator.isForcedShutdownTimedOut(), "isForcedShutdownTimedOut() == true in " + escalator); // now interrupt it a second time and expect it to escalate to a halt try { @@ -99,8 +99,8 @@ public void testBlockingShutdownTimeouts() throws Throwable { assertExceptionDetails(EXIT_INTERRUPTED, "", e); } - assertTrue("isForcedShutdownTimedOut() == false in " + escalator, - escalator.isForcedShutdownTimedOut()); + assertTrue( + escalator.isForcedShutdownTimedOut(), "isForcedShutdownTimedOut() == false in " + escalator); } private static class InterruptCatcher implements IrqHandler.Interrupted { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java index 72757e4b1c182..747f2c51ffd36 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java @@ -32,7 +32,7 @@ import static org.apache.hadoop.test.GenericTestUtils.*; import static org.apache.hadoop.service.launcher.testservices.ExceptionInExecuteLaunchableService.*; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestServiceLauncher extends AbstractServiceLauncherTestBase { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java index c3506b32a9ddb..67dc39f7794db 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java @@ -22,7 +22,7 @@ import org.apache.hadoop.service.launcher.testservices.FailInInitService; import org.apache.hadoop.service.launcher.testservices.FailInStartService; import org.apache.hadoop.service.launcher.testservices.FailingStopInStartService; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Explore the ways in which the launcher is expected to (safely) fail. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java index 5869f347b6f34..c69f6cde70bc4 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java @@ -24,7 +24,7 @@ import org.apache.hadoop.service.launcher.testservices.ExceptionInExecuteLaunchableService; import org.apache.hadoop.service.launcher.testservices.LaunchableRunningService; import org.apache.hadoop.service.launcher.testservices.NoArgsAllowedService; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.List; @@ -40,7 +40,7 @@ public void testLaunchService() throws Throwable { ServiceLauncher launcher = launchService(NoArgsAllowedService.class, new Configuration()); NoArgsAllowedService service = launcher.getService(); - assertNotNull("null service from " + launcher, service); + assertNotNull(service, "null service from " + launcher); service.stop(); } @@ -78,7 +78,7 @@ public void testBreakableServiceLifecycle() throws Throwable { ServiceLauncher launcher = launchService(BreakableService.class, new Configuration()); BreakableService service = launcher.getService(); - assertNotNull("null service from " + launcher, service); + assertNotNull(service, "null service from " + launcher); service.stop(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/InitInConstructorLaunchableService.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/InitInConstructorLaunchableService.java index 541ac68b3ef0f..a4c747547a625 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/InitInConstructorLaunchableService.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/InitInConstructorLaunchableService.java @@ -20,7 +20,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.launcher.AbstractLaunchableService; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import java.util.List; @@ -41,23 +41,23 @@ public InitInConstructorLaunchableService() { @Override public void init(Configuration conf) { - Assert.assertEquals(STATE.NOTINITED, getServiceState()); + Assertions.assertEquals(STATE.NOTINITED, getServiceState()); super.init(conf); } @Override public Configuration bindArgs(Configuration config, List args) throws Exception { - Assert.assertEquals(STATE.INITED, getServiceState()); - Assert.assertTrue(isInState(STATE.INITED)); - Assert.assertNotSame(getConfig(), config); + Assertions.assertEquals(STATE.INITED, getServiceState()); + Assertions.assertTrue(isInState(STATE.INITED)); + Assertions.assertNotSame(getConfig(), config); return null; } @Override public int execute() throws Exception { - Assert.assertEquals(STATE.STARTED, getServiceState()); - Assert.assertSame(originalConf, getConfig()); + Assertions.assertEquals(STATE.STARTED, getServiceState()); + Assertions.assertSame(originalConf, getConfig()); return super.execute(); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/LaunchableRunningService.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/LaunchableRunningService.java index 91d0f2ee02138..eae90a8a0975f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/LaunchableRunningService.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/LaunchableRunningService.java @@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.launcher.LaunchableService; import org.apache.hadoop.service.launcher.LauncherExitCodes; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,7 +63,7 @@ public LaunchableRunningService(String name) { @Override public Configuration bindArgs(Configuration config, List args) throws Exception { - Assert.assertEquals(STATE.NOTINITED, getServiceState()); + Assertions.assertEquals(STATE.NOTINITED, getServiceState()); for (String arg : args) { LOG.info(arg); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestGenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestGenericTestUtils.java index 8489e3d24f368..d64ae7eadc919 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestGenericTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestGenericTestUtils.java @@ -18,7 +18,8 @@ package org.apache.hadoop.test; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -26,9 +27,9 @@ import java.util.function.Supplier; import org.slf4j.event.Level; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; public class TestGenericTestUtils extends GenericTestUtils { @@ -85,7 +86,8 @@ public String toString() { } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testLogCapturer() { final Logger log = LoggerFactory.getLogger(TestGenericTestUtils.class); LogCapturer logCapturer = LogCapturer.captureLogs(log); @@ -103,7 +105,8 @@ public void testLogCapturer() { assertTrue(logCapturer.getOutput().isEmpty()); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testLogCapturerSlf4jLogger() { final Logger logger = LoggerFactory.getLogger(TestGenericTestUtils.class); LogCapturer logCapturer = LogCapturer.captureLogs(logger); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestJUnitSetup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestJUnitSetup.java index 4c0b965a9737d..a41618665b4da 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestJUnitSetup.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestJUnitSetup.java @@ -18,8 +18,8 @@ package org.apache.hadoop.test; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +35,6 @@ public void testJavaAssert() { LOG.info("The AssertionError is expected.", ae); return; } - Assert.fail("Java assert does not work."); + Assertions.fail("Java assert does not work."); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java index 479dd35b0aa1d..15159129a7cea 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java @@ -18,8 +18,8 @@ package org.apache.hadoop.test; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.FileNotFoundException; import java.io.IOException; @@ -39,7 +39,7 @@ * This test suite includes Java 8 and Java 7 code; the Java 8 code exists * to verify that the API is easily used with Lambda expressions. */ -public class TestLambdaTestUtils extends Assert { +public class TestLambdaTestUtils extends Assertions { public static final int INTERVAL = 10; public static final int TIMEOUT = 50; @@ -116,7 +116,7 @@ public Long call() throws Exception { * @param expected expected value */ protected void assertRetryCount(int expected) { - assertEquals(retry.toString(), expected, retry.getInvocationCount()); + assertEquals(expected, retry.getInvocationCount(), retry.toString()); } /** @@ -124,8 +124,8 @@ protected void assertRetryCount(int expected) { * @param minCount minimum value */ protected void assertMinRetryCount(int minCount) { - assertTrue("retry count of " + retry + " is not >= " + minCount, - minCount <= retry.getInvocationCount()); + assertTrue( + minCount <= retry.getInvocationCount(), "retry count of " + retry + " is not >= " + minCount); } /** @@ -181,8 +181,8 @@ public void testAwaitLinearRetry() throws Throwable { TIMEOUT_FAILURE_HANDLER); fail("should not have got here"); } catch (TimeoutException e) { - assertEquals(linearRetry.toString(), - 2, linearRetry.getInvocationCount()); + assertEquals( + 2, linearRetry.getInvocationCount(), linearRetry.toString()); } } @@ -499,7 +499,7 @@ public void testAwaitRethrowsVMErrors() throws Throwable { @Test public void testEvalToSuccess() { - assertTrue("Eval to success", eval(() -> true)); + assertTrue(eval(() -> true), "Eval to success"); } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java index 5d94413d01ce5..7d87070abed19 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java @@ -17,12 +17,12 @@ */ package org.apache.hadoop.test; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.test.MultithreadedTestUtil.TestContext; import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread; import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread; @@ -56,8 +56,8 @@ public void doWork() throws Exception { assertEquals(3, threadsRun.get()); // Test shouldn't have waited the full 30 seconds, since // the threads exited faster than that. - assertTrue("Test took " + (et - st) + "ms", - et - st < 5000); + assertTrue( + et - st < 5000, "Test took " + (et - st) + "ms"); } @Test @@ -81,8 +81,8 @@ public void doWork() throws Exception { long et = Time.now(); // Test shouldn't have waited the full 30 seconds, since // the thread throws faster than that - assertTrue("Test took " + (et - st) + "ms", - et - st < 5000); + assertTrue( + et - st < 5000, "Test took " + (et - st) + "ms"); } @Test @@ -106,8 +106,8 @@ public void doWork() throws Exception { long et = Time.now(); // Test shouldn't have waited the full 30 seconds, since // the thread throws faster than that - assertTrue("Test took " + (et - st) + "ms", - et - st < 5000); + assertTrue( + et - st < 5000, "Test took " + (et - st) + "ms"); } @Test @@ -129,11 +129,11 @@ public void doAnAction() throws Exception { long elapsed = et - st; // Test should have waited just about 3 seconds - assertTrue("Test took " + (et - st) + "ms", - Math.abs(elapsed - 3000) < 500); + assertTrue( + Math.abs(elapsed - 3000) < 500, "Test took " + (et - st) + "ms"); // Counter should have been incremented lots of times in 3 full seconds - assertTrue("Counter value = " + counter.get(), - counter.get() > 1000); + assertTrue( + counter.get() > 1000, "Counter value = " + counter.get()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java index 42ed8c8775570..5478390c9dab5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java @@ -23,8 +23,9 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.runner.notification.Failure; public class TestTimedOutTestsListener { @@ -143,7 +144,8 @@ class Monitor { } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testThreadDumpAndDeadlocks() throws Exception { new Deadlock(); String s = null; @@ -154,7 +156,7 @@ public void testThreadDumpAndDeadlocks() throws Exception { Thread.sleep(100); } - Assert.assertEquals(3, countStringOccurrences(s, "BLOCKED")); + Assertions.assertEquals(3, countStringOccurrences(s, "BLOCKED")); Failure failure = new Failure( null, new Exception(TimedOutTestsListener.TEST_TIMED_OUT_PREFIX)); @@ -162,8 +164,8 @@ public void testThreadDumpAndDeadlocks() throws Exception { new TimedOutTestsListener(new PrintWriter(writer)).testFailure(failure); String out = writer.toString(); - Assert.assertTrue(out.contains("THREAD DUMP")); - Assert.assertTrue(out.contains("DEADLOCKS DETECTED")); + Assertions.assertTrue(out.contains("THREAD DUMP")); + Assertions.assertTrue(out.contains("DEADLOCKS DETECTED")); System.out.println(out); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java index e9c5950b729c6..dcdd2650d1525 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java @@ -23,10 +23,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.tools.CommandShell; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import org.junit.Before; -import org.junit.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestCommandShell { @@ -92,7 +92,7 @@ private String outMsg(String message) { return "OUT:\n" + outContent.toString() + "\n" + message; } - @Before + @BeforeEach public void setup() throws Exception { System.setOut(new PrintStream(outContent)); } @@ -106,22 +106,22 @@ public void testCommandShellExample() throws Exception { outContent.reset(); String[] args1 = {"hello"}; rc = ex.run(args1); - assertEquals(outMsg("test exit code - normal hello"), 0, rc); - assertTrue(outMsg("test normal hello message"), - outContent.toString().contains(Example.HELLO_MSG)); + assertEquals(0, rc, outMsg("test exit code - normal hello")); + assertTrue(outContent.toString().contains(Example.HELLO_MSG), + outMsg("test normal hello message")); outContent.reset(); String[] args2 = {"hello", "x"}; rc = ex.run(args2); - assertEquals(outMsg("test exit code - bad hello"), 1, rc); - assertTrue(outMsg("test bad hello message"), - outContent.toString().contains(Example.Hello.HELLO_USAGE)); + assertEquals(1, rc, outMsg("test exit code - bad hello")); + assertTrue(outContent.toString().contains(Example.Hello.HELLO_USAGE), + outMsg("test bad hello message")); outContent.reset(); String[] args3 = {"goodbye"}; rc = ex.run(args3); - assertEquals(outMsg("test exit code - normal goodbye"), 0, rc); - assertTrue(outMsg("test normal goodbye message"), - outContent.toString().contains(Example.GOODBYE_MSG)); + assertEquals(0, rc, outMsg("test exit code - normal goodbye")); + assertTrue(outContent.toString().contains(Example.GOODBYE_MSG), + outMsg("test normal goodbye message")); } } From ad311756e9821476dba3b249e86db4e65cb329d0 Mon Sep 17 00:00:00 2001 From: fanshilun Date: Thu, 6 Feb 2025 19:14:11 +0800 Subject: [PATCH 2/7] HADOOP-19415. Fix CheckStyle & Fix Unit Test. --- .../hadoop/net/TestClusterTopology.java | 54 +++-- .../java/org/apache/hadoop/net/TestDNS.java | 21 +- .../hadoop/net/TestDNSDomainNameResolver.java | 7 +- .../org/apache/hadoop/net/TestNetUtils.java | 26 ++- .../net/TestNetworkTopologyWithNodeGroup.java | 8 +- .../hadoop/net/TestScriptBasedMapping.java | 9 +- .../TestScriptBasedMappingWithDependency.java | 9 +- .../hadoop/net/TestSocketIOWithTimeout.java | 6 +- .../apache/hadoop/net/TestStaticMapping.java | 19 +- .../apache/hadoop/net/TestSwitchMapping.java | 10 +- .../hadoop/net/unix/TestDomainSocket.java | 51 +++-- .../net/unix/TestDomainSocketWatcher.java | 5 +- .../hadoop/oncrpc/TestFrameDecoder.java | 11 +- .../hadoop/oncrpc/TestRpcAcceptedReply.java | 4 +- .../org/apache/hadoop/oncrpc/TestRpcCall.java | 3 +- .../hadoop/oncrpc/TestRpcCallCache.java | 9 +- .../hadoop/oncrpc/TestRpcDeniedReply.java | 14 +- .../apache/hadoop/oncrpc/TestRpcMessage.java | 6 +- .../apache/hadoop/oncrpc/TestRpcReply.java | 12 +- .../org/apache/hadoop/oncrpc/TestXDR.java | 8 +- .../apache/hadoop/portmap/TestPortmap.java | 10 +- .../security/TestAuthenticationFilter.java | 4 +- .../hadoop/security/TestCredentials.java | 62 +++--- .../security/TestDoAsEffectiveUser.java | 26 +-- .../security/TestFixKerberosTicketOrder.java | 44 ++-- .../hadoop/security/TestGroupsCaching.java | 12 +- .../TestHttpCrossOriginFilterInitializer.java | 13 +- .../TestIngressPortBasedResolver.java | 3 +- .../hadoop/security/TestJNIGroupsMapping.java | 2 +- .../org/apache/hadoop/security/TestKDiag.java | 18 +- .../hadoop/security/TestKDiagNoKDC.java | 17 +- .../security/TestLdapGroupsMapping.java | 22 +- .../TestLdapGroupsMappingWithOneQuery.java | 16 +- .../TestLdapGroupsMappingWithPosixGroup.java | 7 +- .../security/TestNullGroupsMapping.java | 11 +- .../hadoop/security/TestRaceWhenRelogin.java | 10 +- .../TestRuleBasedLdapGroupsMapping.java | 27 +-- .../hadoop/security/TestSecurityUtil.java | 6 +- .../TestShellBasedUnixGroupsMapping.java | 77 +++---- .../security/TestUGILoginFromKeytab.java | 123 +++++------ .../security/TestUGIWithExternalKdc.java | 16 +- .../hadoop/security/TestUserFromEnv.java | 5 +- .../security/TestUserGroupInformation.java | 197 ++++++++---------- .../security/TestWhitelistBasedResolver.java | 2 +- .../apache/hadoop/service/ServiceAssert.java | 16 +- .../hadoop/service/TestCompositeService.java | 132 ++++++------ .../TestGlobalStateChangeListener.java | 18 +- .../hadoop/service/TestServiceLifecycle.java | 10 +- .../hadoop/service/TestServiceOperations.java | 6 +- .../AbstractServiceLauncherTestBase.java | 20 +- .../service/launcher/TestServiceConf.java | 11 +- .../TestServiceInterruptHandling.java | 12 +- .../apache/hadoop/test/TestJUnitSetup.java | 5 +- .../hadoop/test/TestLambdaTestUtils.java | 8 +- .../test/TestMultithreadedTestUtil.java | 21 +- .../test/TestTimedOutTestsListener.java | 15 +- .../hadoop/test/TimedOutTestsListener.java | 7 +- 57 files changed, 624 insertions(+), 679 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java index d5dd4173dab01..c0b9e21f459bb 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java @@ -96,34 +96,31 @@ public void testCountNumNodes() throws Exception { // create exclude list List excludedNodes = new ArrayList(); - assertEquals(4 -, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "4 nodes should be available"); + assertEquals(4, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), + "4 nodes should be available"); NodeElement deadNode = getNewNode("node5", "/d1/r2"); excludedNodes.add(deadNode); - assertEquals(4 -, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "4 nodes should be available with extra excluded Node"); + assertEquals(4, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), + "4 nodes should be available with extra excluded Node"); // add one existing node to exclude list excludedNodes.add(node4); - assertEquals(3 -, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "excluded nodes with ROOT scope should be considered"); - assertEquals(2 -, cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), - excludedNodes), "excluded nodes without ~ scope should be considered"); - assertEquals(1 -, cluster.countNumOfAvailableNodes(deadNode.getNetworkLocation(), - excludedNodes), "excluded nodes with rack scope should be considered"); + assertEquals(3, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), + "excluded nodes with ROOT scope should be considered"); + assertEquals(2, cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), excludedNodes), + "excluded nodes without ~ scope should be considered"); + assertEquals(1, cluster.countNumOfAvailableNodes(deadNode.getNetworkLocation(), + excludedNodes), "excluded nodes with rack scope should be considered"); // adding the node in excluded scope to excluded list excludedNodes.add(node2); - assertEquals(2 -, cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), - excludedNodes), "excluded nodes with ~ scope should be considered"); + assertEquals(2, cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), excludedNodes), + "excluded nodes with ~ scope should be considered"); // getting count with non-exist scope. - assertEquals(0 -, cluster.countNumOfAvailableNodes("/non-exist", excludedNodes), "No nodes should be considered for non-exist scope"); + assertEquals(0, cluster.countNumOfAvailableNodes("/non-exist", excludedNodes), + "No nodes should be considered for non-exist scope"); // remove a node from the cluster cluster.remove(node1); - assertEquals(1 -, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "1 node should be available"); + assertEquals(1, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), + "1 node should be available"); } /** @@ -181,8 +178,8 @@ public void testChooseRandom() { } // Check that they have the proper distribution - assertFalse( - chiSquareTestRejectedCounter==3, "Random not choosing nodes with proper distribution"); + assertFalse(chiSquareTestRejectedCounter == 3, + "Random not choosing nodes with proper distribution"); // Pick random nodes excluding the 2 nodes in /d1/r3 HashMap histogram = new HashMap(); @@ -193,8 +190,8 @@ public void testChooseRandom() { } histogram.put(randomNode, histogram.get(randomNode) + 1); } - assertEquals( - 2, histogram.size(), "Random is not selecting the nodes it should"); + assertEquals(2, histogram.size(), + "Random is not selecting the nodes it should"); Node val = cluster.chooseRandom("/d1", "/d", Collections.emptyList()); assertNotNull(val); @@ -268,9 +265,10 @@ public void testWeights() { for (Pair test: new Pair[]{Pair.of(0, node1), Pair.of(2, node2), Pair.of(4, node3)}) { int expect = test.getLeft(); - assertEquals(expect, cluster.getWeight(node1, test.getRight()), test.toString()); - assertEquals(expect -, cluster.getWeightUsingNetworkLocation(node1, test.getRight()), test.toString()); + assertEquals(expect, cluster.getWeight(node1, test.getRight()), + test.toString()); + assertEquals(expect, cluster.getWeightUsingNetworkLocation(node1, test.getRight()), + test.toString()); } // Reset so that we can have 2 levels cluster = NetworkTopology.getInstance(new Configuration()); @@ -282,8 +280,8 @@ public void testWeights() { Pair.of(2, node6), Pair.of(4, node7), Pair.of(6, node8)}) { int expect = test.getLeft(); assertEquals(expect, cluster.getWeight(node5, test.getRight()), test.toString()); - assertEquals(expect -, cluster.getWeightUsingNetworkLocation(node5, test.getRight()), test.toString()); + assertEquals(expect, cluster.getWeightUsingNetworkLocation(node5, test.getRight()), + test.toString()); } } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java index 8a131ac2c4f08..c2ead4b509993 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java @@ -29,15 +29,18 @@ import org.apache.hadoop.util.Time; -import org.assertj.core.api.Assertions; -import org.junit.Assume; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows; -import static org.junit.jupiter.api.Assertions.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Test host name and IP resolution and caching. @@ -104,7 +107,7 @@ private InetAddress getLocalIPAddr() throws UnknownHostException { @Test public void testNullInterface() throws Exception { String host = DNS.getDefaultHost(null); // should work. - Assertions.assertThat(host).isEqualTo(DNS.getDefaultHost(DEFAULT)); + assertThat(host).isEqualTo(DNS.getDefaultHost(DEFAULT)); try { String ip = DNS.getDefaultIP(null); fail("Expected a NullPointerException, got " + ip); @@ -120,7 +123,7 @@ public void testNullInterface() throws Exception { @Test public void testNullDnsServer() throws Exception { String host = DNS.getDefaultHost(getLoopbackInterface(), null); - Assertions.assertThat(host) + assertThat(host) .isEqualTo(DNS.getDefaultHost(getLoopbackInterface())); } @@ -131,7 +134,7 @@ public void testNullDnsServer() throws Exception { @Test public void testDefaultDnsServer() throws Exception { String host = DNS.getDefaultHost(getLoopbackInterface(), DEFAULT); - Assertions.assertThat(host) + assertThat(host) .isEqualTo(DNS.getDefaultHost(getLoopbackInterface())); } @@ -179,7 +182,7 @@ public void testRDNS() throws Exception { + " Loopback=" + localhost.isLoopbackAddress() + " Linklocal=" + localhost.isLinkLocalAddress()); } - Assume.assumeNoException(e); + assumeTrue(false, e.getMessage()); } } @@ -208,7 +211,7 @@ public void testLookupWithHostsFallback() throws Exception { getLoopbackInterface(), INVALID_DNS_SERVER, true); // Expect to get back something other than the cached host name. - Assertions.assertThat(hostname).isNotEqualTo(DUMMY_HOSTNAME); + assertThat(hostname).isNotEqualTo(DUMMY_HOSTNAME); } finally { // Restore DNS#cachedHostname for subsequent tests. DNS.setCachedHostname(oldHostname); @@ -232,7 +235,7 @@ public void testLookupWithoutHostsFallback() throws Exception { // Expect to get back the cached host name since there was no hosts // file lookup. - Assertions.assertThat(hostname).isEqualTo(DUMMY_HOSTNAME); + assertThat(hostname).isEqualTo(DUMMY_HOSTNAME); } finally { // Restore DNS#cachedHostname for subsequent tests. DNS.setCachedHostname(oldHostname); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java index b854524e498ea..c260680f5df5d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java @@ -25,8 +25,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.Assume.assumeFalse; - +import static org.junit.jupiter.api.Assumptions.assumeFalse; public class TestDNSDomainNameResolver { @@ -35,8 +34,8 @@ public class TestDNSDomainNameResolver { @Test public void testGetHostNameByIP() throws UnknownHostException { InetAddress localhost = InetAddress.getLocalHost(); - assumeFalse("IP lookup support required", - Objects.equals(localhost.getCanonicalHostName(), localhost.getHostAddress())); + assumeFalse(Objects.equals(localhost.getCanonicalHostName(), localhost.getHostAddress()), + "IP lookup support required"); // Precondition: host name and canonical host name for unresolved returns an IP address. InetAddress unresolved = InetAddress.getByAddress(localhost.getHostAddress(), diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java index ff3218504fa2f..c332a8842be54 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java @@ -17,7 +17,14 @@ */ package org.apache.hadoop.net; -import static org.junit.jupiter.api.Assertions.*; +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.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assumptions.assumeTrue; import java.io.EOFException; import java.io.IOException; @@ -47,7 +54,6 @@ import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Shell; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -145,7 +151,7 @@ private void doSocketReadTimeoutTest(boolean withChannel) if (withChannel) { s = NetUtils.getDefaultSocketFactory(new Configuration()) .createSocket(); - Assume.assumeNotNull(s.getChannel()); + assumeTrue(s.getChannel() != null); } else { s = new Socket(); assertNull(s.getChannel()); @@ -193,8 +199,8 @@ private void assertTimeSince(long startNanos, int expectedMillis) { long durationNano = System.nanoTime() - startNanos; long millis = TimeUnit.MILLISECONDS.convert( durationNano, TimeUnit.NANOSECONDS); - assertTrue( - Math.abs(millis - expectedMillis) < TIME_FUDGE_MILLIS, "Expected " + expectedMillis + "ms, but took " + millis); + assertTrue(Math.abs(millis - expectedMillis) < TIME_FUDGE_MILLIS, + "Expected " + expectedMillis + "ms, but took " + millis); } /** @@ -731,7 +737,7 @@ public void testNormalizeHostName() { try { InetAddress.getByName(oneHost); } catch (UnknownHostException e) { - Assume.assumeTrue("Network not resolving "+ oneHost, false); + assumeTrue(false, "Network not resolving " + oneHost); } List hosts = Arrays.asList("127.0.0.1", "localhost", oneHost, "UnknownHost123"); @@ -741,13 +747,13 @@ public void testNormalizeHostName() { // when ipaddress is normalized, same address is expected in return assertEquals(summary, hosts.get(0), normalizedHosts.get(0)); // for normalizing a resolvable hostname, resolved ipaddress is expected in return - assertFalse( - normalizedHosts.get(1).equals(hosts.get(1)), "Element 1 equal "+ summary); + assertFalse(normalizedHosts.get(1).equals(hosts.get(1)), + "Element 1 equal "+ summary); assertEquals(summary, hosts.get(0), normalizedHosts.get(1)); // this address HADOOP-8372: when normalizing a valid resolvable hostname start with numeric, // its ipaddress is expected to return - assertFalse( - normalizedHosts.get(2).equals(hosts.get(2)), "Element 2 equal " + summary); + assertFalse(normalizedHosts.get(2).equals(hosts.get(2)), + "Element 2 equal " + summary); // return the same hostname after normalizing a irresolvable hostname. assertEquals(summary, hosts.get(3), normalizedHosts.get(3)); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java index fed5898db94fa..cf819aada1a10 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java @@ -182,13 +182,13 @@ public void testChooseRandomExcludedNode() { @Test public void testNodeGroup() throws Exception { String res = cluster.getNodeGroup(""); - assertTrue( - res.equals(NodeBase.ROOT), "NodeGroup should be NodeBase.ROOT for empty location"); + assertTrue(res.equals(NodeBase.ROOT), + "NodeGroup should be NodeBase.ROOT for empty location"); try { cluster.getNodeGroup(null); } catch (IllegalArgumentException e) { - assertTrue( - e.getMessage().contains("Network Location is null"), "Null Network Location should throw exception!"); + assertTrue(e.getMessage().contains("Network Location is null"), + "Null Network Location should throw exception!"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java index 40578cfb58476..970e593417890 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java @@ -20,7 +20,10 @@ import java.util.ArrayList; import java.util.List; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.*; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.hadoop.conf.Configuration; @@ -50,8 +53,8 @@ public void testNoFilenameMeansSingleSwitch() throws Throwable { Configuration conf = new Configuration(); ScriptBasedMapping mapping = createMapping(conf); assertTrue(mapping.isSingleSwitch(), "Expected to be single switch"); - assertTrue( - AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), "Expected to be single switch"); + assertTrue(AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), + "Expected to be single switch"); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java index 4233f7506448b..f05c40086d3c5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java @@ -20,7 +20,10 @@ import java.util.ArrayList; import java.util.List; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.*; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.hadoop.conf.Configuration; @@ -56,8 +59,8 @@ public void testNoFilenameMeansSingleSwitch() throws Throwable { Configuration conf = new Configuration(); ScriptBasedMapping mapping = createMapping(conf); assertTrue(mapping.isSingleSwitch(), "Expected to be single switch"); - assertTrue( - AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), "Expected to be single switch"); + assertTrue(AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), + "Expected to be single switch"); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java index 682c7cfb304e5..1ab5e5714adee 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java @@ -42,7 +42,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.jupiter.api.Assertions.*; +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; /** * This tests timeout out from SocketInputStream and diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java index d019b0b21445f..2e4d33a135ffe 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java @@ -78,23 +78,18 @@ private Configuration createConf(String script) { } private void assertSingleSwitch(DNSToSwitchMapping mapping) { - assertEquals( - true -, AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), "Expected a single switch mapping " - + mapping); + assertEquals(true, AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), + "Expected a single switch mapping " + mapping); } private void assertMultiSwitch(DNSToSwitchMapping mapping) { - assertEquals( - false -, AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), "Expected a multi switch mapping " - + mapping); + assertEquals(false, AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping), + "Expected a multi switch mapping " + mapping); } protected void assertMapSize(AbstractDNSToSwitchMapping switchMapping, int expectedSize) { - assertEquals( - - expectedSize, switchMapping.getSwitchMap().size(), "Expected two entries in the map " + switchMapping.dumpTopology()); + assertEquals(expectedSize, switchMapping.getSwitchMap().size(), + "Expected two entries in the map " + switchMapping.dumpTopology()); } private List createQueryList() { @@ -131,7 +126,7 @@ public void testAddResolveNodes() throws Throwable { String topology = mapping.dumpTopology(); LOG.info(topology); assertEquals(1, switchMap.size(), topology); - assertEquals(topology, "/r1", switchMap.get("n1")); + assertEquals("/r1", switchMap.get("n1"), topology); } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java index 2c4b7609beff7..fab2233b7f7b6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java @@ -91,14 +91,12 @@ public void testCachingRelaysStringOperations() throws Throwable { public void testCachingRelaysStringOperationsToNullScript() throws Throwable { Configuration conf = new Configuration(); ScriptBasedMapping scriptMapping = new ScriptBasedMapping(conf); - assertTrue( - scriptMapping.toString().contains(ScriptBasedMapping.NO_SCRIPT), "Did not find " + ScriptBasedMapping.NO_SCRIPT - + " in " + scriptMapping); + assertTrue(scriptMapping.toString().contains(ScriptBasedMapping.NO_SCRIPT), + "Did not find " + ScriptBasedMapping.NO_SCRIPT + " in " + scriptMapping); CachedDNSToSwitchMapping mapping = new CachedDNSToSwitchMapping(scriptMapping); - assertTrue( - mapping.toString().contains(ScriptBasedMapping.NO_SCRIPT), "Did not find " + ScriptBasedMapping.NO_SCRIPT - + " in " + mapping); + assertTrue(mapping.toString().contains(ScriptBasedMapping.NO_SCRIPT), + "Did not find " + ScriptBasedMapping.NO_SCRIPT + " in " + mapping); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java index fdd9ef51387ad..a1704a0ec3013 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java @@ -38,8 +38,6 @@ import java.util.concurrent.atomic.AtomicInteger; import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -52,6 +50,13 @@ import org.apache.hadoop.thirdparty.com.google.common.io.Files; +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.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + public class TestDomainSocket { private static TemporarySocketDirectory sockDir; @@ -68,7 +73,7 @@ public static void shutdown() throws IOException { @BeforeEach public void before() { - Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null); + assumeTrue(DomainSocket.getLoadingFailureReason() == null); } /** @@ -94,7 +99,7 @@ public void testSocketCreateAndClose() throws IOException { @Test @Timeout(value = 180) public void testSocketPathSetGet() throws IOException { - Assertions.assertEquals("/var/run/hdfs/sock.100", + assertEquals("/var/run/hdfs/sock.100", DomainSocket.getEffectivePath("/var/run/hdfs/sock._PORT", 100)); } @@ -123,7 +128,7 @@ public Void call(){ buf[i] = 0; } try { - Assertions.assertEquals(-1, conn.getInputStream().read()); + assertEquals(-1, conn.getInputStream().read()); } catch (IOException e) { throw new RuntimeException("unexpected IOException", e); } @@ -295,19 +300,19 @@ public void testServerOptions() throws Exception { int newBufSize = bufSize / 2; serv.setAttribute(DomainSocket.RECEIVE_BUFFER_SIZE, newBufSize); int nextBufSize = serv.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE); - Assertions.assertEquals(newBufSize, nextBufSize); + assertEquals(newBufSize, nextBufSize); // Let's set a server timeout int newTimeout = 1000; serv.setAttribute(DomainSocket.RECEIVE_TIMEOUT, newTimeout); int nextTimeout = serv.getAttribute(DomainSocket.RECEIVE_TIMEOUT); - Assertions.assertEquals(newTimeout, nextTimeout); + assertEquals(newTimeout, nextTimeout); ExecutorService exeServ = Executors.newSingleThreadExecutor(); Callable callable = new Callable() { public Void call() { try { serv.accept(); - Assertions.fail("expected the accept() to time out and fail"); + fail("expected the accept() to time out and fail"); } catch (SocketTimeoutException e) { GenericTestUtils.assertExceptionContains("accept(2) error: ", e); } catch (AsynchronousCloseException e) { @@ -322,7 +327,7 @@ public Void call() { Thread.sleep(500); serv.close(true); future.get(); - Assertions.assertFalse(serv.isOpen()); + assertFalse(serv.isOpen()); } /** @@ -463,17 +468,17 @@ public void run(){ ReadStrategy reader = readStrategyClass.newInstance(); reader.init(conn); reader.readFully(in1, 0, in1.length); - Assertions.assertTrue(Arrays.equals(clientMsg1, in1)); + assertTrue(Arrays.equals(clientMsg1, in1)); WriteStrategy writer = writeStrategyClass.newInstance(); writer.init(conn); writer.write(serverMsg1); InputStream connInputStream = conn.getInputStream(); int in2 = connInputStream.read(); - Assertions.assertEquals((int)clientMsg2, in2); + assertEquals((int)clientMsg2, in2); conn.close(); } catch (Throwable e) { threadResults.add(e); - Assertions.fail(e.getMessage()); + fail(e.getMessage()); } threadResults.add(new Success()); } @@ -492,7 +497,7 @@ public void run(){ reader.init(client); byte in1[] = new byte[serverMsg1.length]; reader.readFully(in1, 0, in1.length); - Assertions.assertTrue(Arrays.equals(serverMsg1, in1)); + assertTrue(Arrays.equals(serverMsg1, in1)); OutputStream clientOutputStream = client.getOutputStream(); clientOutputStream.write(clientMsg2); client.close(); @@ -507,7 +512,7 @@ public void run(){ for (int i = 0; i < 2; i++) { Throwable t = threadResults.take(); if (!(t instanceof Success)) { - Assertions.fail(t.getMessage() + ExceptionUtils.getStackTrace(t)); + fail(t.getMessage() + ExceptionUtils.getStackTrace(t)); } } serverThread.join(120000); @@ -630,14 +635,14 @@ public void run(){ byte in1[] = new byte[clientMsg1.length]; InputStream connInputStream = conn.getInputStream(); IOUtils.readFully(connInputStream, in1, 0, in1.length); - Assertions.assertTrue(Arrays.equals(clientMsg1, in1)); + assertTrue(Arrays.equals(clientMsg1, in1)); DomainSocket domainConn = (DomainSocket)conn; domainConn.sendFileDescriptors(passedFds, serverMsg1, 0, serverMsg1.length); conn.close(); } catch (Throwable e) { threadResults.add(e); - Assertions.fail(e.getMessage()); + fail(e.getMessage()); } threadResults.add(new Success()); } @@ -656,11 +661,11 @@ public void run(){ FileInputStream recvFis[] = new FileInputStream[passedFds.length]; int r = domainConn. recvFileInputStreams(recvFis, in1, 0, in1.length - 1); - Assertions.assertTrue(r > 0); + assertTrue(r > 0); IOUtils.readFully(clientInputStream, in1, r, in1.length - r); - Assertions.assertTrue(Arrays.equals(serverMsg1, in1)); + assertTrue(Arrays.equals(serverMsg1, in1)); for (int i = 0; i < passedFds.length; i++) { - Assertions.assertNotNull(recvFis[i]); + assertNotNull(recvFis[i]); passedFiles[i].checkInputStream(recvFis[i]); } for (FileInputStream fis : recvFis) { @@ -678,7 +683,7 @@ public void run(){ for (int i = 0; i < 2; i++) { Throwable t = threadResults.take(); if (!(t instanceof Success)) { - Assertions.fail(t.getMessage() + ExceptionUtils.getStackTrace(t)); + fail(t.getMessage() + ExceptionUtils.getStackTrace(t)); } } serverThread.join(120000); @@ -783,11 +788,11 @@ public void run() { socks[0].getOutputStream().write(1); socks[0].getOutputStream().write(2); socks[0].getOutputStream().write(3); - Assertions.assertTrue(readerThread.isAlive()); + assertTrue(readerThread.isAlive()); socks[0].shutdown(); readerThread.join(); - Assertions.assertFalse(failed.get()); - Assertions.assertEquals(3, bytesRead.get()); + assertFalse(failed.get()); + assertEquals(3, bytesRead.get()); IOUtils.cleanupWithLogger(null, socks); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocketWatcher.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocketWatcher.java index 6e9f3073b27f0..f78005a6ed3f2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocketWatcher.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocketWatcher.java @@ -27,7 +27,6 @@ import java.util.concurrent.locks.ReentrantLock; import org.junit.jupiter.api.AfterEach; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -36,6 +35,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + public class TestDomainSocketWatcher { static final Logger LOG = LoggerFactory.getLogger(TestDomainSocketWatcher.class); @@ -44,7 +45,7 @@ public class TestDomainSocketWatcher { @BeforeEach public void before() { - Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null); + assumeTrue(DomainSocket.getLoadingFailureReason() == null); } @AfterEach diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java index 4bd3bcc4ff302..a1d917514245c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java @@ -36,9 +36,10 @@ import org.apache.hadoop.oncrpc.security.VerifierNone; import org.apache.hadoop.test.GenericTestUtils; import org.junit.jupiter.api.Test; -import org.mockito.Mockito; import org.slf4j.event.Level; +import static org.mockito.Mockito.mock; + public class TestFrameDecoder { static { @@ -103,7 +104,7 @@ public void testSingleFrame() { ByteBuf buf = Unpooled.directBuffer(1); List outputBufs = new ArrayList<>(); decoder.decode( - Mockito.mock(ChannelHandlerContext.class), buf, + mock(ChannelHandlerContext.class), buf, outputBufs); assertTrue(outputBufs.isEmpty()); @@ -122,7 +123,7 @@ public void testSingleFrame() { buf.writeBytes(fragment); outputBufs = new ArrayList<>(); decoder.decode( - Mockito.mock(ChannelHandlerContext.class), buf, + mock(ChannelHandlerContext.class), buf, outputBufs); assertTrue(decoder.isLast()); buf.release(); @@ -147,7 +148,7 @@ public void testMultipleFrames() { ByteBuf buf = Unpooled.directBuffer(4 + 10, 4 + 10); buf.writeBytes(fragment1); decoder.decode( - Mockito.mock(ChannelHandlerContext.class), buf, + mock(ChannelHandlerContext.class), buf, outputBufs); byte[] fragment2 = new byte[4 + 10]; @@ -162,7 +163,7 @@ public void testMultipleFrames() { buf = Unpooled.directBuffer(4 + 10, 4 + 10); buf.writeBytes(fragment2); decoder.decode( - Mockito.mock(ChannelHandlerContext.class), buf, + mock(ChannelHandlerContext.class), buf, outputBufs); // Expect two completed frames each 10 bytes decoder.isLast(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java index 99863943c436c..89651c64d122c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java @@ -42,8 +42,8 @@ public void testAcceptState() { @Test public void testAcceptStateFromInvalidValue() { - assertThrows(IndexOutOfBoundsException.class, ()-> - AcceptState.fromValue(6)); + assertThrows(IndexOutOfBoundsException.class, () -> + AcceptState.fromValue(6)); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java index 0219094607a4f..ad9bc85b4214c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java @@ -62,7 +62,8 @@ public void testInvalidRpcVersion() { @Test public void testInvalidRpcMessageType() { assertThrows(IllegalArgumentException.class, () -> { - RpcMessage.Type invalidMessageType = RpcMessage.Type.RPC_REPLY; // Message typ is not RpcMessage.RPC_CALL + // Message typ is not RpcMessage.RPC_CALL + RpcMessage.Type invalidMessageType = RpcMessage.Type.RPC_REPLY; new RpcCall(0, invalidMessageType, RpcCall.RPC_VERSION, 2, 3, 4, null, null); }); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCallCache.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCallCache.java index 0e1df10bf5449..19fb4dc044fbb 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCallCache.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcCallCache.java @@ -26,8 +26,13 @@ import org.apache.hadoop.oncrpc.RpcCallCache.ClientRequest; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.*; -import static org.mockito.Mockito.*; +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.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; /** * Unit tests for {@link RpcCallCache} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java index a2880f76a80d6..8d499c75458cd 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java @@ -20,9 +20,9 @@ import org.apache.hadoop.oncrpc.RpcDeniedReply.RejectState; import org.apache.hadoop.oncrpc.RpcReply.ReplyState; import org.apache.hadoop.oncrpc.security.VerifierNone; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; /** @@ -31,8 +31,8 @@ public class TestRpcDeniedReply { @Test public void testRejectStateFromValue() { - Assertions.assertEquals(RejectState.RPC_MISMATCH, RejectState.fromValue(0)); - Assertions.assertEquals(RejectState.AUTH_ERROR, RejectState.fromValue(1)); + assertEquals(RejectState.RPC_MISMATCH, RejectState.fromValue(0)); + assertEquals(RejectState.AUTH_ERROR, RejectState.fromValue(1)); } @Test @@ -45,9 +45,9 @@ public void testRejectStateFromInvalidValue1() { public void testConstructor() { RpcDeniedReply reply = new RpcDeniedReply(0, ReplyState.MSG_ACCEPTED, RejectState.AUTH_ERROR, new VerifierNone()); - Assertions.assertEquals(0, reply.getXid()); - Assertions.assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType()); - Assertions.assertEquals(ReplyState.MSG_ACCEPTED, reply.getState()); - Assertions.assertEquals(RejectState.AUTH_ERROR, reply.getRejectState()); + assertEquals(0, reply.getXid()); + assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType()); + assertEquals(ReplyState.MSG_ACCEPTED, reply.getState()); + assertEquals(RejectState.AUTH_ERROR, reply.getRejectState()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java index 62729b0bab423..a675134fcda36 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.oncrpc; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; /** @@ -38,8 +38,8 @@ public XDR write(XDR xdr) { @Test public void testRpcMessage() { RpcMessage msg = getRpcMessage(0, RpcMessage.Type.RPC_CALL); - Assertions.assertEquals(0, msg.getXid()); - Assertions.assertEquals(RpcMessage.Type.RPC_CALL, msg.getMessageType()); + assertEquals(0, msg.getXid()); + assertEquals(RpcMessage.Type.RPC_CALL, msg.getMessageType()); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java index 16663572b3cbc..264bd7cd08970 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java @@ -20,9 +20,9 @@ import org.apache.hadoop.oncrpc.RpcReply.ReplyState; import org.apache.hadoop.oncrpc.security.VerifierNone; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; /** @@ -31,8 +31,8 @@ public class TestRpcReply { @Test public void testReplyStateFromValue() { - Assertions.assertEquals(ReplyState.MSG_ACCEPTED, ReplyState.fromValue(0)); - Assertions.assertEquals(ReplyState.MSG_DENIED, ReplyState.fromValue(1)); + assertEquals(ReplyState.MSG_ACCEPTED, ReplyState.fromValue(0)); + assertEquals(ReplyState.MSG_DENIED, ReplyState.fromValue(1)); } @Test @@ -49,8 +49,8 @@ public XDR write(XDR xdr) { return null; } }; - Assertions.assertEquals(0, reply.getXid()); - Assertions.assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType()); - Assertions.assertEquals(ReplyState.MSG_ACCEPTED, reply.getState()); + assertEquals(0, reply.getXid()); + assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType()); + assertEquals(ReplyState.MSG_ACCEPTED, reply.getState()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java index 7c64f6d81acc6..7132fb949e703 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java @@ -17,9 +17,10 @@ */ package org.apache.hadoop.oncrpc; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; + public class TestXDR { static final int WRITE_VALUE=23; private void serializeInt(int times) { @@ -29,8 +30,7 @@ private void serializeInt(int times) { XDR r = w.asReadOnlyWrap(); for (int i = 0; i < times; ++i) - Assertions.assertEquals( - WRITE_VALUE,r.readInt()); + assertEquals(WRITE_VALUE,r.readInt()); } private void serializeLong(int times) { @@ -40,7 +40,7 @@ private void serializeLong(int times) { XDR r = w.asReadOnlyWrap(); for (int i = 0; i < times; ++i) - Assertions.assertEquals(WRITE_VALUE, r.readHyper()); + assertEquals(WRITE_VALUE, r.readHyper()); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/portmap/TestPortmap.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/portmap/TestPortmap.java index 90bd2a1e314fc..c2e6fd746c776 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/portmap/TestPortmap.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/portmap/TestPortmap.java @@ -27,7 +27,6 @@ import java.util.Map; import org.apache.hadoop.oncrpc.RpcReply; -import org.junit.jupiter.api.Assertions; import org.apache.hadoop.oncrpc.RpcCall; import org.apache.hadoop.oncrpc.XDR; @@ -39,6 +38,7 @@ import org.junit.jupiter.api.Timeout; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestPortmap { private static Portmap pm = new Portmap(); @@ -70,11 +70,11 @@ public void testIdle() throws InterruptedException, IOException { Thread.sleep(SHORT_TIMEOUT_MILLISECONDS); } - Assertions.assertTrue(s.isConnected() - && i < RETRY_TIMES, "Failed to connect to the server"); + assertTrue(s.isConnected() && i < RETRY_TIMES, + "Failed to connect to the server"); int b = s.getInputStream().read(); - Assertions.assertTrue(b == -1, "The server failed to disconnect"); + assertTrue(b == -1, "The server failed to disconnect"); } finally { s.close(); } @@ -128,6 +128,6 @@ public void testRegistration() throws IOException, InterruptedException, Illegal break; } } - Assertions.assertTrue(found, "Registration failed"); + assertTrue(found, "Registration failed"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java index b8b56c522894c..c1a2dec04211b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java @@ -16,8 +16,8 @@ */ package org.apache.hadoop.security; - -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.mockito.ArgumentMatchers.any; import org.apache.hadoop.http.HttpServer2; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java index 58859f08d4b39..74a70eb8809ef 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java @@ -45,7 +45,10 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestCredentials { private static final String DEFAULT_HMAC_ALGORITHM = "HmacSHA1"; @@ -111,29 +114,27 @@ public void testReadWriteStorage() // get the tokens and compare the services Map> tokenMap = ts.getTokenMap(); - assertEquals(2 -, tokenMap.size(), "getTokenMap should return collection of size 2"); - assertTrue( - tokenMap.containsKey(alias1), "Token for alias " + alias1 + " must be present"); - assertTrue( - tokenMap.containsKey(alias2), "Token for alias " + alias2 + " must be present"); - assertEquals(service1 -, tokenMap.get(alias1).getService(), "Token for service " + service1 + " must be present"); - assertEquals(service2 -, tokenMap.get(alias2).getService(), "Token for service " + service2 + " must be present"); + assertEquals(2, tokenMap.size(), + "getTokenMap should return collection of size 2"); + assertTrue(tokenMap.containsKey(alias1), + "Token for alias " + alias1 + " must be present"); + assertTrue(tokenMap.containsKey(alias2), "Token for alias " + alias2 + " must be present"); + assertEquals(service1, tokenMap.get(alias1).getService(), + "Token for service " + service1 + " must be present"); + assertEquals(service2, + tokenMap.get(alias2).getService(), "Token for service " + service2 + " must be present"); // compare secret keys Map secretKeyMap = ts.getSecretKeyMap(); - assertEquals(m.size() -, ts.numberOfSecretKeys(), "wrong number of keys in the Storage"); + assertEquals(m.size(), ts.numberOfSecretKeys(), + "wrong number of keys in the Storage"); for (Map.Entry entry : m.entrySet()) { byte[] key = secretKeyMap.get(entry.getKey()); - assertNotNull( - key, "Secret key for alias " + entry.getKey() + " not found"); - assertTrue( - Arrays.equals(key, entry.getValue()), "Keys don't match for alias " + entry.getKey()); + assertNotNull(key, "Secret key for alias " + entry.getKey() + " not found"); + assertTrue(Arrays.equals(key, entry.getValue()), + "Keys don't match for alias " + entry.getKey()); } tmpFileName.delete(); @@ -245,22 +246,19 @@ private Credentials generateCredentials(Text t1, Text t2, Text t3) private void assertCredentials(String tag, Text alias, Text keykey, Credentials a, Credentials b) { - assertEquals(a.numberOfTokens() -, b.numberOfTokens(), tag + ": test token count"); - assertEquals(a.getToken(alias).getService() -, b.getToken(alias).getService(), tag + ": test service"); - assertEquals(a.getToken(alias).getKind() -, b.getToken(alias).getKind(), tag + ": test kind"); - assertTrue( - Arrays.equals(a.getToken(alias).getPassword(), - b.getToken(alias).getPassword()), tag + ": test password"); - assertTrue( - Arrays.equals(a.getToken(alias).getIdentifier(), - b.getToken(alias).getIdentifier()), tag + ": test identifier"); - assertEquals(a.numberOfSecretKeys() -, b.numberOfSecretKeys(), tag + ": test number of keys"); + assertEquals(a.numberOfTokens(), b.numberOfTokens(), tag + ": test token count"); + assertEquals(a.getToken(alias).getService(), + b.getToken(alias).getService(), tag + ": test service"); + assertEquals(a.getToken(alias).getKind(), + b.getToken(alias).getKind(), tag + ": test kind"); + assertTrue(Arrays.equals(a.getToken(alias).getPassword(), + b.getToken(alias).getPassword()), tag + ": test password"); + assertTrue(Arrays.equals(a.getToken(alias).getIdentifier(), + b.getToken(alias).getIdentifier()), tag + ": test identifier"); + assertEquals(a.numberOfSecretKeys(), + b.numberOfSecretKeys(), tag + ": test number of keys"); assertTrue(Arrays.equals(a.getSecretKey(keykey), - b.getSecretKey(keykey)), tag + ":test key values"); + b.getSecretKey(keykey)), tag + ":test key values"); } private void writeCredentialsStream(Credentials creds, String filename) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java index 5d58fc5277b1b..fafe0bc903d31 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java @@ -29,7 +29,6 @@ import org.apache.hadoop.security.authorize.DefaultImpersonationProvider; import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.security.token.Token; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -43,6 +42,9 @@ import java.util.ArrayList; import java.util.Enumeration; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + /** * Test do as effective user. */ @@ -121,7 +123,7 @@ public UserGroupInformation run() throws IOException { return UserGroupInformation.getCurrentUser(); } }); - Assertions.assertEquals( + assertEquals( PROXY_USER_NAME + " (auth:PROXY) via " + REAL_USER_NAME + " (auth:SIMPLE)", curUGI.toString()); } @@ -137,8 +139,8 @@ public Void run() throws ServiceException { String serverRemoteUser = client.getServerRemoteUser(null, newEmptyRequest()).getUser(); - Assertions.assertEquals(ugi.toString(), currentUser); - Assertions.assertEquals(ugi.toString(), serverRemoteUser); + assertEquals(ugi.toString(), currentUser); + assertEquals(ugi.toString(), serverRemoteUser); return null; } }); @@ -169,7 +171,7 @@ public void testRealUserSetup() throws IOException { checkRemoteUgi(proxyUserUgi, conf); } catch (Exception e) { e.printStackTrace(); - Assertions.fail(); + fail(); } finally { stop(server, client); } @@ -199,7 +201,7 @@ public void testRealUserAuthorizationSuccess() throws IOException { checkRemoteUgi(proxyUserUgi, conf); } catch (Exception e) { e.printStackTrace(); - Assertions.fail(); + fail(); } finally { stop(server, client); } @@ -240,7 +242,7 @@ public String run() throws ServiceException { } }); - Assertions.fail("The RPC must have failed " + retVal); + fail("The RPC must have failed " + retVal); } catch (Exception e) { e.printStackTrace(); } finally { @@ -276,7 +278,7 @@ public String run() throws ServiceException { } }); - Assertions.fail("The RPC must have failed " + retVal); + fail("The RPC must have failed " + retVal); } catch (Exception e) { e.printStackTrace(); } finally { @@ -309,7 +311,7 @@ public String run() throws ServiceException { } }); - Assertions.fail("The RPC must have failed " + retVal); + fail("The RPC must have failed " + retVal); } catch (Exception e) { e.printStackTrace(); } finally { @@ -347,7 +349,7 @@ public String run() throws ServiceException { } }); - Assertions.fail("The RPC must have failed " + retVal); + fail("The RPC must have failed " + retVal); } catch (Exception e) { e.printStackTrace(); } finally { @@ -400,7 +402,7 @@ public String run() throws Exception { } }); //The user returned by server must be the one in the token. - Assertions.assertEquals(REAL_USER_NAME + " (auth:TOKEN) via SomeSuperUser (auth:SIMPLE)", retVal); + assertEquals(REAL_USER_NAME + " (auth:TOKEN) via SomeSuperUser (auth:SIMPLE)", retVal); } /* @@ -444,7 +446,7 @@ public String run() throws Exception { } }); String expected = REAL_USER_NAME + " (auth:TOKEN) via SomeSuperUser (auth:SIMPLE)"; - Assertions.assertEquals(expected, retVal, retVal + "!=" + expected); + assertEquals(expected, retVal, retVal + "!=" + expected); } // diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java index 234037bd54ce2..cbea393d93164 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java @@ -18,8 +18,8 @@ package org.apache.hadoop.security; import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.io.File; import java.security.PrivilegedExceptionAction; @@ -37,8 +37,8 @@ import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.Before; +import org.junit.Test; /** * Testcase for HADOOP-13433 that verifies the logic of fixKerberosTicketOrder. @@ -63,7 +63,7 @@ public class TestFixKerberosTicketOrder extends KerberosSecurityTestcase { private Map props; - @BeforeEach + @Before public void setUp() throws Exception { keytabFile = new File(getWorkDir(), "keytab"); getKdc().createPrincipal(keytabFile, clientPrincipal, server1Principal, @@ -106,13 +106,13 @@ public Void run() throws Exception { } // make sure the first ticket is not tgt assertFalse( - - subject.getPrivateCredentials().stream() + "The first ticket is still tgt, " + + "the implementation in jdk may have been changed, " + + "please reconsider the problem in HADOOP-13433", + subject.getPrivateCredentials().stream() .filter(c -> c instanceof KerberosTicket) .map(c -> ((KerberosTicket) c).getServer().getName()).findFirst() - .get().startsWith("krbtgt"), "The first ticket is still tgt, " - + "the implementation in jdk may have been changed, " - + "please reconsider the problem in HADOOP-13433"); + .get().startsWith("krbtgt")); // should fail as we send a service ticket instead of tgt to KDC. intercept(SaslException.class, () -> ugi.doAs(new PrivilegedExceptionAction() { @@ -131,11 +131,11 @@ public Void run() throws Exception { ugi.fixKerberosTicketOrder(); // check if TGT is the first ticket after the fix. - assertTrue( - subject.getPrivateCredentials().stream() + assertTrue("The first ticket is not tgt", + subject.getPrivateCredentials().stream() .filter(c -> c instanceof KerberosTicket) .map(c -> ((KerberosTicket) c).getServer().getName()).findFirst() - .get().startsWith("krbtgt"), "The first ticket is not tgt"); + .get().startsWith("krbtgt")); // make sure we can still get new service ticket after the fix. ugi.doAs(new PrivilegedExceptionAction() { @@ -150,10 +150,10 @@ public Void run() throws Exception { return null; } }); - assertTrue( - subject.getPrivateCredentials(KerberosTicket.class).stream() + assertTrue("No service ticket for " + server2Protocol + " found", + subject.getPrivateCredentials(KerberosTicket.class).stream() .filter(t -> t.getServer().getName().startsWith(server2Protocol)) - .findAny().isPresent(), "No service ticket for " + server2Protocol + " found"); + .findAny().isPresent()); } @Test @@ -188,11 +188,11 @@ public Void run() throws Exception { ugi.fixKerberosTicketOrder(); // verify that after fixing, the tgt ticket should be removed - assertFalse( - subject.getPrivateCredentials().stream() + assertFalse("The first ticket is not tgt", + subject.getPrivateCredentials().stream() .filter(c -> c instanceof KerberosTicket) .map(c -> ((KerberosTicket) c).getServer().getName()).findFirst() - .isPresent(), "The first ticket is not tgt"); + .isPresent()); // should fail as we send a service ticket instead of tgt to KDC. @@ -227,9 +227,9 @@ public Void run() throws Exception { } }); - assertTrue( - subject.getPrivateCredentials(KerberosTicket.class).stream() + assertTrue("No service ticket for " + server2Protocol + " found", + subject.getPrivateCredentials(KerberosTicket.class).stream() .filter(t -> t.getServer().getName().startsWith(server2Protocol)) - .findAny().isPresent(), "No service ticket for " + server2Protocol + " found"); + .findAny().isPresent()); } } \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java index be6c136df7f22..5a2927f71c18e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java @@ -272,8 +272,8 @@ public void testGroupLookupForStaticUsers() throws Exception { Groups groups = new Groups(conf); List userGroups = groups.getGroups("me"); assertTrue(userGroups.isEmpty(), "non-empty groups for static user"); - assertFalse( - FakeunPrivilegedGroupMapping.invoked, "group lookup done for static user"); + assertFalse(FakeunPrivilegedGroupMapping.invoked, + "group lookup done for static user"); List expected = new ArrayList(); expected.add("group1"); @@ -281,15 +281,15 @@ public void testGroupLookupForStaticUsers() throws Exception { FakeunPrivilegedGroupMapping.invoked = false; userGroups = groups.getGroups("user1"); assertTrue(expected.equals(userGroups), "groups not correct"); - assertFalse( - FakeunPrivilegedGroupMapping.invoked, "group lookup done for unprivileged user"); + assertFalse(FakeunPrivilegedGroupMapping.invoked, + "group lookup done for unprivileged user"); expected.add("group2"); FakeunPrivilegedGroupMapping.invoked = false; userGroups = groups.getGroups("user2"); assertTrue(expected.equals(userGroups), "groups not correct"); - assertFalse( - FakeunPrivilegedGroupMapping.invoked, "group lookup done for unprivileged user"); + assertFalse(FakeunPrivilegedGroupMapping.invoked, + "group lookup done for unprivileged user"); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestHttpCrossOriginFilterInitializer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestHttpCrossOriginFilterInitializer.java index e05b0acd6682a..04e345ba97840 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestHttpCrossOriginFilterInitializer.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestHttpCrossOriginFilterInitializer.java @@ -22,9 +22,11 @@ import org.apache.hadoop.conf.Configuration; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + public class TestHttpCrossOriginFilterInitializer { @Test @@ -48,11 +50,8 @@ public void testGetFilterParameters() { String outofscopeparam = filterParameters.get("outofscopeparam"); // verify expected values are in place - Assertions.assertEquals("Could not find filter parameter", "rootvalue", - rootvalue); - Assertions.assertEquals("Could not find filter parameter", "nestedvalue", - nestedvalue); - Assertions.assertNull( - outofscopeparam, "Found unexpected value in filter parameters"); + assertEquals("rootvalue", rootvalue, "Could not find filter parameter"); + assertEquals("nestedvalue", nestedvalue, "Could not find filter parameter"); + assertNull(outofscopeparam, "Found unexpected value in filter parameters"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestIngressPortBasedResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestIngressPortBasedResolver.java index 1a67a69b84ce6..e551f54d1d50b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestIngressPortBasedResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestIngressPortBasedResolver.java @@ -21,8 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.*; - +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Test class for IngressPortBasedResolver. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestJNIGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestJNIGroupsMapping.java index 8e20aa2e2b128..5183e894a986b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestJNIGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestJNIGroupsMapping.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.security; import static org.junit.Assume.assumeTrue; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.fail; import java.util.Arrays; import java.util.List; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java index fdee97ba794a0..71c7a4cf3eb0d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java @@ -23,13 +23,11 @@ import org.apache.hadoop.minikdc.MiniKdc; import org.apache.hadoop.test.GenericTestUtils; import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeAll; -import org.junit.Rule; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.rules.TestName; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,11 +36,11 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Properties; -import java.util.concurrent.TimeUnit; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; import static org.apache.hadoop.security.KDiag.*; +@Timeout(30) public class TestKDiag extends Assertions { private static final Logger LOG = LoggerFactory.getLogger(TestKDiag.class); @@ -50,12 +48,6 @@ public class TestKDiag extends Assertions { public static final String HDFS_SITE_XML = "org/apache/hadoop/security/secure-hdfs-site.xml"; - @Rule - public TestName methodName = new TestName(); - - @Rule - public Timeout testTimeout = new Timeout(30000, TimeUnit.MILLISECONDS); - @BeforeAll public static void nameThread() { Thread.currentThread().setName("JUnit"); @@ -155,7 +147,7 @@ public void testKeytabNoPrincipal() throws Throwable { @Test public void testConfIsSecure() throws Throwable { - Assertions.assertFalse(SecurityUtil.getAuthenticationMethod(conf) + assertFalse(SecurityUtil.getAuthenticationMethod(conf) .equals(UserGroupInformation.AuthenticationMethod.SIMPLE)); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java index b28c8292df7b8..babcfac5c5207 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java @@ -20,18 +20,10 @@ import org.apache.hadoop.conf.Configuration; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.Rule; -import org.junit.jupiter.api.Test; -import org.junit.rules.TestName; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.TimeUnit; - import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES; import static org.apache.hadoop.security.KDiag.ARG_KEYLEN; import static org.apache.hadoop.security.KDiag.ARG_NOFAIL; @@ -41,17 +33,12 @@ import static org.apache.hadoop.security.KDiag.KerberosDiagsFailure; import static org.apache.hadoop.security.KDiag.exec; +@Timeout(30) public class TestKDiagNoKDC extends Assertions { private static final Logger LOG = LoggerFactory.getLogger(TestKDiagNoKDC.class); public static final String KEYLEN = "128"; - @Rule - public TestName methodName = new TestName(); - - @Rule - public Timeout testTimeout = new Timeout(30000, TimeUnit.MILLISECONDS); - @BeforeAll public static void nameThread() { Thread.currentThread().setName("JUnit"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java index 32d76a774b92c..c3cc7cf54393e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java @@ -23,6 +23,7 @@ import static org.apache.hadoop.security.LdapGroupsMapping.READ_TIMEOUT; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; 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.assertNull; import static org.junit.jupiter.api.Assertions.fail; @@ -59,7 +60,6 @@ import org.apache.hadoop.security.alias.JavaKeyStoreProvider; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -166,7 +166,7 @@ public void testGetGroupsWithDynamicGroupFilter() throws Exception { // Check the group filter got resolved and get the desired values. List groups = groupsMapping.getGroups(userName); - Assertions.assertEquals(Arrays.asList(getTestGroups()), groups); + assertEquals(Arrays.asList(getTestGroups()), groups); } /** @@ -191,7 +191,7 @@ private void doTestGetGroupsWithBaseDN(Configuration conf, String userBaseDN, .thenReturn(getUserNames(), getGroupNames()); List groups = groupsMapping.getGroups(userName); - Assertions.assertEquals(Arrays.asList(getTestGroups()), groups); + assertEquals(Arrays.asList(getTestGroups()), groups); // We should have searched for the username and groups with default base dn verify(getContext(), times(1)).search(userBaseDN, @@ -259,7 +259,7 @@ private void doTestGetGroups(List expectedGroups, int searchTimes) // regardless of input List groups = groupsMapping.getGroups("some_user"); - Assertions.assertEquals(expectedGroups, groups); + assertEquals(expectedGroups, groups); // We should have searched for a user, and then two groups verify(getContext(), times(searchTimes)).search(anyString(), @@ -279,7 +279,7 @@ private void doTestGetGroupsWithParent(List expectedGroups, List groups = groupsMapping.getGroups("some_user"); // compare lists, ignoring the order - Assertions.assertEquals(new HashSet<>(expectedGroups), new HashSet<>(groups)); + assertEquals(new HashSet<>(expectedGroups), new HashSet<>(groups)); // We should have searched for a user, and group verify(getContext(), times(searchTimesGroup)).search(anyString(), @@ -299,7 +299,7 @@ public void testExtractPassword() throws IOException { writer.close(); LdapGroupsMapping mapping = new LdapGroupsMapping(); - Assertions.assertEquals("hadoop", + assertEquals("hadoop", mapping.extractPassword(secretFile.getPath())); } @@ -345,15 +345,15 @@ public void testConfGetPassword() throws Exception { LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY).getCredential()); LdapGroupsMapping mapping = new LdapGroupsMapping(); - Assertions.assertEquals("bindpass", + assertEquals("bindpass", mapping.getPassword(conf, LdapGroupsMapping.BIND_PASSWORD_KEY, "")); - Assertions.assertEquals("storepass", + assertEquals("storepass", mapping.getPassword(conf, LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY, "")); // let's make sure that a password that doesn't exist returns an // empty string as currently expected and used to trigger a call to // extract password - Assertions.assertEquals("", mapping.getPassword(conf,"invalid-alias", "")); + assertEquals("", mapping.getPassword(conf,"invalid-alias", "")); } @Test @@ -388,11 +388,11 @@ public void testConfGetPasswordUsingAlias() throws Exception { bindpassAlias).getCredential()); LdapGroupsMapping mapping = new LdapGroupsMapping(); - Assertions.assertEquals("bindpass", + assertEquals("bindpass", mapping.getPasswordFromCredentialProviders(conf, bindpassAlias, "")); // Empty for an invalid alias - Assertions.assertEquals("", mapping.getPasswordFromCredentialProviders( + assertEquals("", mapping.getPasswordFromCredentialProviders( conf, "invalid-alias", "")); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java index e7fe020341a82..0cf1eac3a10fd 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java @@ -31,10 +31,12 @@ import javax.naming.directory.SearchResult; import org.apache.hadoop.conf.Configuration; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.stubbing.Stubber; +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.anyString; import static org.mockito.ArgumentMatchers.eq; @@ -114,9 +116,9 @@ private void doTestGetGroups(List expectedGroups) // regardless of input List groups = groupsMapping.getGroups("some_user"); - Assertions.assertEquals(expectedGroups, groups); - Assertions.assertFalse( - groupsMapping.isSecondaryQueryCalled(), "Second LDAP query should NOT have been called."); + assertEquals(expectedGroups, groups); + assertFalse(groupsMapping.isSecondaryQueryCalled(), + "Second LDAP query should NOT have been called."); // We should have only made one query because single-query lookup is enabled verify(getContext(), times(1)).search(anyString(), anyString(), @@ -144,11 +146,11 @@ private void doTestGetGroupsWithFallback() List groups = groupsMapping.getGroups("some_user"); // expected to be empty due to invalid memberOf - Assertions.assertEquals(0, groups.size()); + assertEquals(0, groups.size()); // expect secondary query to be called: getGroups() - Assertions.assertTrue( - groupsMapping.isSecondaryQueryCalled(), "Second LDAP query should have been called."); + assertTrue(groupsMapping.isSecondaryQueryCalled(), + "Second LDAP query should have been called."); // We should have fallen back to the second query because first threw // NamingException expected count is 3 since testGetGroups calls diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java index 345f7df08b414..6816f8a551321 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java @@ -37,10 +37,11 @@ import javax.naming.directory.SearchControls; import org.apache.hadoop.conf.Configuration; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; + @SuppressWarnings("unchecked") public class TestLdapGroupsMappingWithPosixGroup extends TestLdapGroupsMappingBase { @@ -90,11 +91,11 @@ private void doTestGetGroups(List expectedGroups, int searchTimes) // regardless of input List groups = groupsMapping.getGroups("some_user"); - Assertions.assertEquals(expectedGroups, groups); + assertEquals(expectedGroups, groups); groupsMapping.getConf().set(LdapGroupsMapping.POSIX_UID_ATTR_KEY, "uid"); - Assertions.assertEquals(expectedGroups, groups); + assertEquals(expectedGroups, groups); // We should have searched for a user, and then two groups verify(getContext(), times(searchTimes)).search(anyString(), diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNullGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNullGroupsMapping.java index a03ff36a032f1..3be6d2557b4bf 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNullGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestNullGroupsMapping.java @@ -20,7 +20,7 @@ import java.util.List; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Test that the {@link NullGroupsMapping} really does nothing. @@ -42,19 +42,16 @@ public void testGetGroups() { List expResult = Collections.emptyList(); List result = ngm.getGroups(user); - assertEquals( - expResult, result, "No groups should be returned"); + assertEquals(expResult, result, "No groups should be returned"); ngm.cacheGroupsAdd(Arrays.asList(new String[] {"group1", "group2"})); result = ngm.getGroups(user); - assertEquals( - expResult, result, "No groups should be returned"); + assertEquals(expResult, result, "No groups should be returned"); ngm.cacheGroupsRefresh(); result = ngm.getGroups(user); - assertEquals( - expResult, result, "No groups should be returned"); + assertEquals(expResult, result, "No groups should be returned"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRaceWhenRelogin.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRaceWhenRelogin.java index 473afcc6c44a7..4f9946c3e27ba 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRaceWhenRelogin.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRaceWhenRelogin.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.security; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; @@ -37,8 +37,8 @@ import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.Before; +import org.junit.Test; /** * Testcase for HADOOP-13433 that confirms that tgt will always be the first @@ -68,7 +68,7 @@ public class TestRaceWhenRelogin extends KerberosSecurityTestcase { private UserGroupInformation ugi; - @BeforeEach + @Before public void setUp() throws Exception { keytabFile = new File(getWorkDir(), "keytab"); serverProtocols = new String[numThreads]; @@ -157,6 +157,6 @@ public void test() throws InterruptedException, IOException { for (Thread getServiceTicketThread : getServiceTicketThreads) { getServiceTicketThread.join(); } - assertTrue(pass.get(), "tgt is not the first ticket after relogin"); + assertTrue("tgt is not the first ticket after relogin", pass.get()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java index 23c8197bac528..80f55c78592db 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java @@ -18,9 +18,7 @@ package org.apache.hadoop.security; import org.apache.hadoop.conf.Configuration; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.mockito.Mockito; import javax.naming.NamingException; import java.util.ArrayList; @@ -30,8 +28,11 @@ import static org.apache.hadoop.security.RuleBasedLdapGroupsMapping .CONVERSION_RULE_KEY; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.eq; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.spy; /** * Test cases to verify the rules supported by RuleBasedLdapGroupsMapping. @@ -40,12 +41,12 @@ public class TestRuleBasedLdapGroupsMapping { @Test public void testGetGroupsToUpper() throws NamingException { - RuleBasedLdapGroupsMapping groupsMapping = Mockito.spy( + RuleBasedLdapGroupsMapping groupsMapping = spy( new RuleBasedLdapGroupsMapping()); Set groups = new LinkedHashSet<>(); groups.add("group1"); groups.add("group2"); - Mockito.doReturn(groups).when((LdapGroupsMapping) groupsMapping) + doReturn(groups).when((LdapGroupsMapping) groupsMapping) .doGetGroups(eq("admin"), anyInt()); Configuration conf = new Configuration(); @@ -56,17 +57,17 @@ public void testGetGroupsToUpper() throws NamingException { List groupsUpper = new ArrayList<>(); groupsUpper.add("GROUP1"); groupsUpper.add("GROUP2"); - Assertions.assertEquals(groupsUpper, groupsMapping.getGroups("admin")); + assertEquals(groupsUpper, groupsMapping.getGroups("admin")); } @Test public void testGetGroupsToLower() throws NamingException { - RuleBasedLdapGroupsMapping groupsMapping = Mockito.spy( + RuleBasedLdapGroupsMapping groupsMapping = spy( new RuleBasedLdapGroupsMapping()); Set groups = new LinkedHashSet<>(); groups.add("GROUP1"); groups.add("GROUP2"); - Mockito.doReturn(groups).when((LdapGroupsMapping) groupsMapping) + doReturn(groups).when((LdapGroupsMapping) groupsMapping) .doGetGroups(eq("admin"), anyInt()); Configuration conf = new Configuration(); @@ -77,17 +78,17 @@ public void testGetGroupsToLower() throws NamingException { List groupsLower = new ArrayList<>(); groupsLower.add("group1"); groupsLower.add("group2"); - Assertions.assertEquals(groupsLower, groupsMapping.getGroups("admin")); + assertEquals(groupsLower, groupsMapping.getGroups("admin")); } @Test public void testGetGroupsInvalidRule() throws NamingException { - RuleBasedLdapGroupsMapping groupsMapping = Mockito.spy( + RuleBasedLdapGroupsMapping groupsMapping = spy( new RuleBasedLdapGroupsMapping()); Set groups = new LinkedHashSet<>(); groups.add("group1"); groups.add("GROUP2"); - Mockito.doReturn(groups).when((LdapGroupsMapping) groupsMapping) + doReturn(groups).when((LdapGroupsMapping) groupsMapping) .doGetGroups(eq("admin"), anyInt()); Configuration conf = new Configuration(); @@ -95,7 +96,7 @@ public void testGetGroupsInvalidRule() throws NamingException { conf.set(CONVERSION_RULE_KEY, "none"); groupsMapping.setConf(conf); - Assertions.assertEquals(groups, groupsMapping.getGroupsSet("admin")); + assertEquals(groups, groupsMapping.getGroupsSet("admin")); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java index 3b43459786dca..9c4e1cd2101eb 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java @@ -19,7 +19,11 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.*; -import static org.junit.jupiter.api.Assertions.*; +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.assertTrue; import java.io.File; import java.io.IOException; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java index 360bfece5fbac..a23ea2413abee 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java @@ -32,7 +32,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.jupiter.api.Assertions.*; +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 static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -190,19 +193,16 @@ public long getTimeoutInterval(String timeout) { public void testShellTimeOutConf() { // Test a 1 second max-runtime timeout - assertEquals( - - 1000L, getTimeoutInterval("1s"), "Expected the group names executor to carry the configured timeout"); + assertEquals(1000L, getTimeoutInterval("1s"), + "Expected the group names executor to carry the configured timeout"); // Test a 1 minute max-runtime timeout - assertEquals( - - 60000L, getTimeoutInterval("1m"), "Expected the group names executor to carry the configured timeout"); + assertEquals(60000L, getTimeoutInterval("1m"), + "Expected the group names executor to carry the configured timeout"); // Test a 1 millisecond max-runtime timeout - assertEquals( - - 1L, getTimeoutInterval("1"), "Expected the group names executor to carry the configured timeout"); + assertEquals(1L, getTimeoutInterval("1"), + "Expected the group names executor to carry the configured timeout"); } private class TestGroupResolvable @@ -294,26 +294,19 @@ public void testFiniteGroupResolutionTime() throws Exception { ReflectionUtils.newInstance(TestDelayedGroupCommand.class, conf); ShellCommandExecutor executor = mapping.createGroupExecutor(userName); - assertEquals( - - testTimeout -, executor.getTimeoutInterval(), "Expected the group names executor to carry the configured timeout"); + assertEquals(testTimeout, executor.getTimeoutInterval(), + "Expected the group names executor to carry the configured timeout"); executor = mapping.createGroupIDExecutor(userName); - assertEquals( - - testTimeout -, executor.getTimeoutInterval(), "Expected the group ID executor to carry the configured timeout"); - - assertEquals( - - 0 -, mapping.getGroups(userName).size(), "Expected no groups to be returned given a shell command timeout"); - assertTrue( - - shellMappingLog.getOutput().contains(commandTimeoutMessage), "Expected the logs to carry " + - "a message about command timeout but was: " + - shellMappingLog.getOutput()); + assertEquals(testTimeout, executor.getTimeoutInterval(), + "Expected the group ID executor to carry the configured timeout"); + + assertEquals(0, mapping.getGroups(userName).size(), + "Expected no groups to be returned given a shell command timeout"); + assertTrue(shellMappingLog.getOutput().contains(commandTimeoutMessage), + "Expected the logs to carry " + + "a message about command timeout but was: " + + shellMappingLog.getOutput()); shellMappingLog.clearOutput(); // Test also the parent Groups framework for expected behaviour @@ -327,11 +320,10 @@ public void testFiniteGroupResolutionTime() throws Exception { "The groups framework call should " + "have failed with a command timeout"); } catch (IOException e) { - assertTrue( - - shellMappingLog.getOutput().contains(commandTimeoutMessage), "Expected the logs to carry " + - "a message about command timeout but was: " + - shellMappingLog.getOutput()); + assertTrue(shellMappingLog.getOutput().contains(commandTimeoutMessage), + "Expected the logs to carry " + + "a message about command timeout but was: " + + shellMappingLog.getOutput()); } shellMappingLog.clearOutput(); @@ -345,22 +337,17 @@ public void testFiniteGroupResolutionTime() throws Exception { ReflectionUtils.newInstance(TestDelayedGroupCommand.class, conf); executor = mapping.createGroupExecutor(userName); - assertEquals( - - defaultTimeout -, executor.getTimeoutInterval(), "Expected the group names executor to carry the default timeout"); + assertEquals(defaultTimeout, + executor.getTimeoutInterval(), "Expected the group names executor to carry the default timeout"); executor = mapping.createGroupIDExecutor(userName); - assertEquals( - - defaultTimeout -, executor.getTimeoutInterval(), "Expected the group ID executor to carry the default timeout"); + assertEquals(defaultTimeout, + executor.getTimeoutInterval(), "Expected the group ID executor to carry the default timeout"); mapping.getGroups(userName); - assertFalse( - - shellMappingLog.getOutput().contains(commandTimeoutMessage), "Didn't expect a timeout of command in execution but logs carry it: " + - shellMappingLog.getOutput()); + assertFalse(shellMappingLog.getOutput().contains(commandTimeoutMessage), + "Didn't expect a timeout of command in execution but logs carry it: " + + shellMappingLog.getOutput()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java index 12fcfc5118d93..9af75e209b421 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java @@ -25,12 +25,10 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Time; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; -import org.junit.Rule; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.io.TempDir; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -40,11 +38,17 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN; 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.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.security.Principal; import java.security.PrivilegedExceptionAction; import java.util.Iterator; @@ -74,11 +78,8 @@ public class TestUGILoginFromKeytab { private File workDir; private ExecutorService executor; - @Rule - public final TemporaryFolder folder = new TemporaryFolder(); - @BeforeEach - public void startMiniKdc() throws Exception { + public void startMiniKdc(@TempDir Path tempDir) throws Exception { // This setting below is required. If not enabled, UGI will abort // any attempt to loginUserFromKeytab. Configuration conf = new Configuration(); @@ -86,7 +87,7 @@ public void startMiniKdc() throws Exception { "kerberos"); UserGroupInformation.setConfiguration(conf); UserGroupInformation.setShouldRenewImmediatelyForTests(true); - workDir = folder.getRoot(); + workDir = tempDir.toFile(); kdc = new MiniKdc(MiniKdc.createConf(), workDir); kdc.start(); executor = Executors.newCachedThreadPool(); @@ -115,13 +116,13 @@ public void testUGILoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assertions.assertTrue( + assertTrue( ugi.isFromKeytab(), "UGI should be configured to login from keytab"); User user = getUser(ugi.getSubject()); - Assertions.assertNotNull(user.getLogin()); + assertNotNull(user.getLogin()); - Assertions.assertTrue( + assertTrue( user.getLastLogin() > beforeLogin, "User login time is less than before login time, " + "beforeLoginTime:" + beforeLogin + " userLoginTime:" + user.getLastLogin()); } @@ -138,14 +139,14 @@ public void testUGIReLoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assertions.assertTrue( + assertTrue( ugi.isFromKeytab(), "UGI should be configured to login from keytab"); // Verify relogin from keytab. User user = getUser(ugi.getSubject()); final long firstLogin = user.getLastLogin(); final LoginContext login1 = user.getLogin(); - Assertions.assertNotNull(login1); + assertNotNull(login1); // Sleep for 2 secs to have a difference between first and second login Thread.sleep(2000); @@ -153,10 +154,10 @@ public void testUGIReLoginFromKeytab() throws Exception { ugi.reloginFromKeytab(); final long secondLogin = user.getLastLogin(); final LoginContext login2 = user.getLogin(); - Assertions.assertTrue( + assertTrue( secondLogin > firstLogin, "User should have been able to relogin from keytab"); - Assertions.assertNotNull(login2); - Assertions.assertNotSame(login1, login2); + assertNotNull(login2); + assertNotSame(login1, login2); } /** @@ -173,14 +174,14 @@ public void testUGIForceReLoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assertions.assertTrue( + assertTrue( ugi.isFromKeytab(), "UGI should be configured to login from keytab"); // Verify relogin from keytab. User user = getUser(ugi.getSubject()); final long firstLogin = user.getLastLogin(); final LoginContext login1 = user.getLogin(); - Assertions.assertNotNull(login1); + assertNotNull(login1); // Sleep for 2 secs to have a difference between first and second login Thread.sleep(2000); @@ -189,10 +190,10 @@ public void testUGIForceReLoginFromKeytab() throws Exception { ugi.forceReloginFromKeytab(); final long secondLogin = user.getLastLogin(); final LoginContext login2 = user.getLogin(); - Assertions.assertTrue( + assertTrue( secondLogin > firstLogin, "User should have been able to relogin from keytab"); - Assertions.assertNotNull(login2); - Assertions.assertNotSame(login1, login2); + assertNotNull(login2); + assertNotSame(login1, login2); } @Test @@ -206,14 +207,14 @@ public void testGetUGIFromKnownSubject() throws Exception { principal.getName(), keytab.getPath()); Subject subject = ugi1.getSubject(); User user = getUser(subject); - Assertions.assertNotNull(user); + assertNotNull(user); LoginContext login = user.getLogin(); - Assertions.assertNotNull(login); + assertNotNull(login); // User instance and/or login context should not change. UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject); - Assertions.assertSame(user, getUser(ugi2.getSubject())); - Assertions.assertSame(login, user.getLogin()); + assertSame(user, getUser(ugi2.getSubject())); + assertSame(login, user.getLogin()); } @Test @@ -231,17 +232,17 @@ public void testGetUGIFromExternalSubject() throws Exception { // first call to get the ugi should add the User instance w/o a login // context. UserGroupInformation ugi1 = UserGroupInformation.getUGIFromSubject(subject); - Assertions.assertSame(subject, ugi1.getSubject()); + assertSame(subject, ugi1.getSubject()); User user = getUser(subject); - Assertions.assertNotNull(user); - Assertions.assertEquals(principal.getName(), user.getName()); - Assertions.assertNull(user.getLogin()); + assertNotNull(user); + assertEquals(principal.getName(), user.getName()); + assertNull(user.getLogin()); // subsequent call should not change the existing User instance. UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject); - Assertions.assertSame(subject, ugi2.getSubject()); - Assertions.assertSame(user, getUser(ugi2.getSubject())); - Assertions.assertNull(user.getLogin()); + assertSame(subject, ugi2.getSubject()); + assertSame(user, getUser(ugi2.getSubject())); + assertNull(user.getLogin()); } @Test @@ -262,9 +263,9 @@ public void testGetUGIFromExternalSubjectWithLogin() throws Exception { // nothing should change. UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject); - Assertions.assertSame(subject, ugi2.getSubject()); - Assertions.assertSame(user, getUser(ugi2.getSubject())); - Assertions.assertSame(dummyLogin, user.getLogin()); + assertSame(subject, ugi2.getSubject()); + assertSame(user, getUser(ugi2.getSubject())); + assertSame(dummyLogin, user.getLogin()); } @Test @@ -283,12 +284,12 @@ public void testUGIRefreshFromKeytab() throws Exception { UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assertions.assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, + assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); - Assertions.assertTrue(ugi.isFromKeytab()); - Assertions.assertTrue( + assertTrue(ugi.isFromKeytab()); + assertTrue( UserGroupInformation.isKerberosKeyTabLoginRenewalEnabled()); - Assertions.assertTrue( + assertTrue( UserGroupInformation.getKerberosLoginRenewalExecutor() .isPresent()); } @@ -310,12 +311,12 @@ public void testUGIRefreshFromKeytabDisabled() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - Assertions.assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, + assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); - Assertions.assertTrue(ugi.isFromKeytab()); - Assertions.assertFalse( + assertTrue(ugi.isFromKeytab()); + assertFalse( UserGroupInformation.isKerberosKeyTabLoginRenewalEnabled()); - Assertions.assertFalse( + assertFalse( UserGroupInformation.getKerberosLoginRenewalExecutor() .isPresent()); } @@ -330,13 +331,13 @@ private static KerberosTicket getTicket(UserGroupInformation ugi) { // the expected principal. private static KerberosTicket checkTicketAndKeytab(UserGroupInformation ugi, KerberosPrincipal principal, boolean expectIsKeytab) { - Assertions.assertEquals( + assertEquals( principal.getName(), ugi.getUserName(), "wrong principal"); - Assertions.assertEquals( + assertEquals( expectIsKeytab, ugi.isFromKeytab(), "is not keytab"); KerberosTicket ticket = getTicket(ugi); - Assertions.assertNotNull(ticket, "no ticket"); - Assertions.assertEquals(principal, ticket.getClient(), "wrong principal"); + assertNotNull(ticket, "no ticket"); + assertEquals(principal, ticket.getClient(), "wrong principal"); return ticket; } @@ -378,16 +379,16 @@ public Void run() throws IOException { loginUser.reloginFromKeytab(); KerberosTicket newLoginTicket = checkTicketAndKeytab(loginUser, principal1, true); - Assertions.assertNotEquals(loginTicket.getAuthTime(), + assertNotEquals(loginTicket.getAuthTime(), newLoginTicket.getAuthTime()); // verify an "external" subject ticket does not change. extSubjectUser.reloginFromKeytab(); - Assertions.assertSame(ticket, + assertSame(ticket, checkTicketAndKeytab(extSubjectUser, principal2, false)); // verify subject ugi relogin did not affect the login user. - Assertions.assertSame(newLoginTicket, + assertSame(newLoginTicket, checkTicketAndKeytab(loginUser, principal1, true)); return null; @@ -410,7 +411,7 @@ public void testReloginForLoginFromSubject() throws Exception { principal1.getName(), keytab1.getPath()); final UserGroupInformation originalLoginUser = UserGroupInformation.getLoginUser(); - Assertions.assertNotNull(getUser(originalLoginUser.getSubject()).getLogin()); + assertNotNull(getUser(originalLoginUser.getSubject()).getLogin()); originalLoginUser.doAs(new PrivilegedExceptionAction() { @Override @@ -427,7 +428,7 @@ public Void run() throws IOException { // verify the new login user is external. UserGroupInformation.loginUserFromSubject(subject); - Assertions.assertNull(getUser(subject).getLogin()); + assertNull(getUser(subject).getLogin()); UserGroupInformation extLoginUser = UserGroupInformation.getLoginUser(); KerberosTicket extLoginUserTicket = @@ -436,17 +437,17 @@ public Void run() throws IOException { // verify subject-based login user does not get a new ticket, and // original login user not affected. extLoginUser.reloginFromKeytab(); - Assertions.assertSame(extLoginUserTicket, + assertSame(extLoginUserTicket, checkTicketAndKeytab(extLoginUser, principal2, false)); - Assertions.assertSame(originalLoginUserTicket, + assertSame(originalLoginUserTicket, checkTicketAndKeytab(originalLoginUser, principal1, true)); // verify original login user gets a new ticket, new login user // not affected. originalLoginUser.reloginFromKeytab(); - Assertions.assertNotSame(originalLoginUserTicket, + assertNotSame(originalLoginUserTicket, checkTicketAndKeytab(originalLoginUser, principal1, true)); - Assertions.assertSame(extLoginUserTicket, + assertSame(extLoginUserTicket, checkTicketAndKeytab(extLoginUser, principal2, false)); return null; } @@ -466,20 +467,20 @@ public void testReloginAfterFailedRelogin() throws Exception { checkTicketAndKeytab(loginUser, principal, true); // move the keytab to induce a relogin failure. - Assertions.assertTrue(keytab.renameTo(keytabBackup)); + assertTrue(keytab.renameTo(keytabBackup)); try { loginUser.reloginFromKeytab(); - Assertions.fail("relogin should fail"); + fail("relogin should fail"); } catch (KerberosAuthException kae) { // expected. } // even though no KeyTab object, ugi should know it's keytab based. - Assertions.assertTrue(loginUser.isFromKeytab()); - Assertions.assertNull(getTicket(loginUser)); + assertTrue(loginUser.isFromKeytab()); + assertNull(getTicket(loginUser)); // move keytab back to enable relogin to succeed. - Assertions.assertTrue(keytabBackup.renameTo(keytab)); + assertTrue(keytabBackup.renameTo(keytab)); loginUser.reloginFromKeytab(); checkTicketAndKeytab(loginUser, principal, true); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java index e32ef16e8f375..379bc1d1299ba 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java @@ -18,13 +18,15 @@ import java.io.IOException; -import org.junit.jupiter.api.Assertions; +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 static org.junit.jupiter.api.Assumptions.assumeTrue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import static org.apache.hadoop.security.SecurityUtilTestHelper.isExternalKdcRunning; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -41,15 +43,15 @@ public class TestUGIWithExternalKdc { @BeforeEach public void testExternalKdcRunning() { - Assume.assumeTrue(isExternalKdcRunning()); + assumeTrue(isExternalKdcRunning()); } @Test public void testLogin() throws IOException { String userPrincipal = System.getProperty("user.principal"); String userKeyTab = System.getProperty("user.keytab"); - Assertions.assertNotNull(userPrincipal, "User principal was not specified"); - Assertions.assertNotNull(userKeyTab, "User keytab was not specified"); + assertNotNull(userPrincipal, "User principal was not specified"); + assertNotNull(userKeyTab, "User keytab was not specified"); Configuration conf = new Configuration(); conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, @@ -59,13 +61,13 @@ public void testLogin() throws IOException { UserGroupInformation ugi = UserGroupInformation .loginUserFromKeytabAndReturnUGI(userPrincipal, userKeyTab); - Assertions.assertEquals(AuthenticationMethod.KERBEROS, + assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); try { UserGroupInformation .loginUserFromKeytabAndReturnUGI("bogus@EXAMPLE.COM", userKeyTab); - Assertions.fail("Login should have failed"); + fail("Login should have failed"); } catch (Exception ex) { ex.printStackTrace(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserFromEnv.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserFromEnv.java index b173c97f0e863..1a1e1c8c26c8d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserFromEnv.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserFromEnv.java @@ -18,15 +18,16 @@ package org.apache.hadoop.security; import java.io.IOException; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; + public class TestUserFromEnv { @Test public void testUserFromEnvironment() throws IOException { System.setProperty(UserGroupInformation.HADOOP_USER_NAME, "randomUser"); - Assertions.assertEquals("randomUser", UserGroupInformation.getLoginUser() + assertEquals("randomUser", UserGroupInformation.getLoginUser() .getUserName()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java index a285de8a851e9..1327561cf4356 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java @@ -33,12 +33,11 @@ import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -82,16 +81,16 @@ import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges; import static org.apache.hadoop.test.MetricsAsserts.getDoubleGauge; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -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.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNotSame; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertSame; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.Assert.assertArrayEquals; +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.assertNotSame; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -135,7 +134,7 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) { } /** configure ugi */ - @BeforeAll + @BeforeClass public static void setup() { javax.security.auth.login.Configuration.setConfiguration( new DummyLoginConfiguration()); @@ -146,32 +145,29 @@ public static void setup() { System.setProperty("hadoop.home.dir", (home != null ? home : ".")); } - @BeforeEach + @Before public void setupUgi() { conf = new Configuration(); UserGroupInformation.reset(); UserGroupInformation.setConfiguration(conf); } - @AfterEach + @After public void resetUgi() { UserGroupInformation.setLoginUser(null); } - @Test - @Timeout(value = 30) + @Test(timeout = 30000) public void testSimpleLogin() throws IOException { tryLoginAuthenticationMethod(AuthenticationMethod.SIMPLE, true); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testTokenLogin() throws IOException { tryLoginAuthenticationMethod(AuthenticationMethod.TOKEN, false); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testProxyLogin() throws IOException { tryLoginAuthenticationMethod(AuthenticationMethod.PROXY, false); } @@ -200,8 +196,7 @@ private void tryLoginAuthenticationMethod(AuthenticationMethod method, } } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testGetRealAuthenticationMethod() { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user1"); ugi.setAuthenticationMethod(AuthenticationMethod.SIMPLE); @@ -212,8 +207,7 @@ public void testGetRealAuthenticationMethod() { assertEquals(AuthenticationMethod.SIMPLE, ugi.getRealAuthenticationMethod()); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testCreateRemoteUser() { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user1"); assertEquals(AuthenticationMethod.SIMPLE, ugi.getAuthenticationMethod()); @@ -225,8 +219,7 @@ public void testCreateRemoteUser() { } /** Test login method */ - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testLogin() throws Exception { conf.set(HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS, String.valueOf(PERCENTILES_INTERVAL)); @@ -257,8 +250,7 @@ public UserGroupInformation run() throws IOException { * given user name - get all the groups. * Needs to happen before creating the test users */ - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testGetServerSideGroups() throws IOException, InterruptedException { // get the user name @@ -319,8 +311,7 @@ public Object run() throws IOException { } /** test constructor */ - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testConstructor() throws Exception { // security off, so default should just return simple name testConstructorSuccess("user1", "user1"); @@ -334,8 +325,7 @@ public void testConstructor() throws Exception { } /** test constructor */ - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testConstructorWithRules() throws Exception { // security off, but use rules if explicitly set conf.set(HADOOP_SECURITY_AUTH_TO_LOCAL, @@ -367,8 +357,7 @@ public void testConstructorWithRules() throws Exception { } /** test constructor */ - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testConstructorWithKerberos() throws Exception { // security on, default is remove default realm conf.set(HADOOP_SECURITY_AUTH_TO_LOCAL_MECHANISM, "hadoop"); @@ -398,8 +387,7 @@ public void testConstructorWithKerberos() throws Exception { } /** test constructor */ - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testConstructorWithKerberosRules() throws Exception { // security on, explicit rules SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf); @@ -435,13 +423,12 @@ private void testConstructorFailures(String userName) { String expect = (userName == null || userName.isEmpty()) ? "Null user" : "Illegal principal name "+userName; String expect2 = "Malformed Kerberos name: "+userName; - assertTrue( - e.toString().contains(expect) || e.toString().contains(expect2), "Did not find "+ expect + " or " + expect2 + " in " + e); + assertTrue("Did not find "+ expect + " or " + expect2 + " in " + e, + e.toString().contains(expect) || e.toString().contains(expect2)); } } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testSetConfigWithRules() { String[] rules = { "RULE:[1:TEST1]", "RULE:[1:TEST2]", "RULE:[1:TEST3]" }; @@ -471,8 +458,7 @@ public void testSetConfigWithRules() { assertEquals(rules[2], KerberosName.getRules()); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testEnsureInitWithRules() throws IOException { String rules = "RULE:[1:RULE1]"; @@ -491,8 +477,7 @@ public void testEnsureInitWithRules() throws IOException { assertEquals(rules, KerberosName.getRules()); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testEquals() throws Exception { UserGroupInformation uugi = UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES); @@ -510,8 +495,7 @@ public void testEquals() throws Exception { assertEquals(uugi.hashCode(), ugi3.hashCode()); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testEqualsWithRealUser() throws Exception { UserGroupInformation realUgi1 = UserGroupInformation.createUserForTesting( "RealUser", GROUP_NAMES); @@ -524,8 +508,7 @@ public void testEqualsWithRealUser() throws Exception { assertFalse(remoteUgi.equals(proxyUgi1)); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testGettingGroups() throws Exception { UserGroupInformation uugi = UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES); @@ -536,8 +519,8 @@ public void testGettingGroups() throws Exception { assertEquals(GROUP1_NAME, uugi.getPrimaryGroupName()); } - @SuppressWarnings("unchecked")@Test - @Timeout(value = 30) + @SuppressWarnings("unchecked") // from Mockito mocks + @Test (timeout = 30000) public void testAddToken() throws Exception { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("someone"); @@ -574,8 +557,8 @@ public void testAddToken() throws Exception { checkTokens(ugi, t1, t2, t3); } - @SuppressWarnings("unchecked")@Test - @Timeout(value = 30) + @SuppressWarnings("unchecked") // from Mockito mocks + @Test (timeout = 30000) public void testGetCreds() throws Exception { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("someone"); @@ -600,8 +583,8 @@ public void testGetCreds() throws Exception { checkTokens(ugi, t1, t2); } - @SuppressWarnings("unchecked")@Test - @Timeout(value = 30) + @SuppressWarnings("unchecked") // from Mockito mocks + @Test (timeout = 30000) public void testAddCreds() throws Exception { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("someone"); @@ -626,8 +609,7 @@ public void testAddCreds() throws Exception { assertSame(secret, ugi.getCredentials().getSecretKey(secretKey)); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testGetCredsNotSame() throws Exception { UserGroupInformation ugi = @@ -654,8 +636,8 @@ private void checkTokens(UserGroupInformation ugi, Token ... tokens) { assertEquals(tokens.length, ugiCreds.numberOfTokens()); } - @SuppressWarnings("unchecked")@Test - @Timeout(value = 30) + @SuppressWarnings("unchecked") // from Mockito mocks + @Test (timeout = 30000) public void testAddNamedToken() throws Exception { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("someone"); @@ -675,8 +657,8 @@ public void testAddNamedToken() throws Exception { assertSame(t1, ugi.getCredentials().getToken(service2)); } - @SuppressWarnings("unchecked")@Test - @Timeout(value = 30) + @SuppressWarnings("unchecked") // from Mockito mocks + @Test (timeout = 30000) public void testUGITokens() throws Exception { UserGroupInformation ugi = UserGroupInformation.createUserForTesting("TheDoctor", @@ -722,8 +704,7 @@ public Collection> run() throws IOException { assertTrue(otherSet.contains(t2)); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testTokenIdentifiers() throws Exception { UserGroupInformation ugi = UserGroupInformation.createUserForTesting( "TheDoctor", new String[] { "TheTARDIS" }); @@ -751,8 +732,7 @@ public Collection run() throws IOException { assertEquals(2, otherSet.size()); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testTestAuthMethod() throws Exception { UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); // verify the reverse mappings works @@ -764,42 +744,40 @@ public void testTestAuthMethod() throws Exception { } } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testUGIAuthMethod() throws Exception { final UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); final AuthenticationMethod am = AuthenticationMethod.KERBEROS; ugi.setAuthenticationMethod(am); - Assertions.assertEquals(am, ugi.getAuthenticationMethod()); + Assert.assertEquals(am, ugi.getAuthenticationMethod()); ugi.doAs(new PrivilegedExceptionAction() { @Override public Object run() throws IOException { - Assertions.assertEquals(am, UserGroupInformation.getCurrentUser() + Assert.assertEquals(am, UserGroupInformation.getCurrentUser() .getAuthenticationMethod()); return null; } }); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testUGIAuthMethodInRealUser() throws Exception { final UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); UserGroupInformation proxyUgi = UserGroupInformation.createProxyUser( "proxy", ugi); final AuthenticationMethod am = AuthenticationMethod.KERBEROS; ugi.setAuthenticationMethod(am); - Assertions.assertEquals(am, ugi.getAuthenticationMethod()); - Assertions.assertEquals(AuthenticationMethod.PROXY, + Assert.assertEquals(am, ugi.getAuthenticationMethod()); + Assert.assertEquals(AuthenticationMethod.PROXY, proxyUgi.getAuthenticationMethod()); - Assertions.assertEquals(am, UserGroupInformation + Assert.assertEquals(am, UserGroupInformation .getRealAuthenticationMethod(proxyUgi)); proxyUgi.doAs(new PrivilegedExceptionAction() { @Override public Object run() throws IOException { - Assertions.assertEquals(AuthenticationMethod.PROXY, UserGroupInformation + Assert.assertEquals(AuthenticationMethod.PROXY, UserGroupInformation .getCurrentUser().getAuthenticationMethod()); - Assertions.assertEquals(am, UserGroupInformation.getCurrentUser() + Assert.assertEquals(am, UserGroupInformation.getCurrentUser() .getRealUser().getAuthenticationMethod()); return null; } @@ -807,18 +785,17 @@ public Object run() throws IOException { UserGroupInformation proxyUgi2 = new UserGroupInformation(proxyUgi.getSubject()); proxyUgi2.setAuthenticationMethod(AuthenticationMethod.PROXY); - Assertions.assertEquals(proxyUgi, proxyUgi2); + Assert.assertEquals(proxyUgi, proxyUgi2); // Equality should work if authMethod is null UserGroupInformation realugi = UserGroupInformation.getCurrentUser(); UserGroupInformation proxyUgi3 = UserGroupInformation.createProxyUser( "proxyAnother", realugi); UserGroupInformation proxyUgi4 = new UserGroupInformation(proxyUgi3.getSubject()); - Assertions.assertEquals(proxyUgi3, proxyUgi4); + Assert.assertEquals(proxyUgi3, proxyUgi4); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testLoginObjectInSubject() throws Exception { UserGroupInformation loginUgi = UserGroupInformation.getLoginUser(); UserGroupInformation anotherUgi = new UserGroupInformation(loginUgi @@ -828,11 +805,10 @@ public void testLoginObjectInSubject() throws Exception { LoginContext login2 = anotherUgi.getSubject().getPrincipals(User.class) .iterator().next().getLogin(); //login1 and login2 must be same instances - Assertions.assertTrue(login1 == login2); + Assert.assertTrue(login1 == login2); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testLoginModuleCommit() throws Exception { UserGroupInformation loginUgi = UserGroupInformation.getLoginUser(); User user1 = loginUgi.getSubject().getPrincipals(User.class).iterator() @@ -843,7 +819,7 @@ public void testLoginModuleCommit() throws Exception { User user2 = loginUgi.getSubject().getPrincipals(User.class).iterator() .next(); // user1 and user2 must be same instances. - Assertions.assertTrue(user1 == user2); + Assert.assertTrue(user1 == user2); } public static void verifyLoginMetrics(long success, int failure) @@ -881,8 +857,7 @@ private static void verifyGroupMetrics( * with it, but that Subject was not created by Hadoop (ie it has no * associated User principal) */ - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testUGIUnderNonHadoopContext() throws Exception { Subject nonHadoopSubject = new Subject(); Subject.doAs(nonHadoopSubject, new PrivilegedExceptionAction() { @@ -895,8 +870,7 @@ public Void run() throws IOException { }); } - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testGetUGIFromSubject() throws Exception { KerberosPrincipal p = new KerberosPrincipal("guest"); Subject subject = new Subject(); @@ -907,8 +881,7 @@ public void testGetUGIFromSubject() throws Exception { } /** Test hasSufficientTimeElapsed method */ - @Test - @Timeout(value = 30) + @Test (timeout = 30000) public void testHasSufficientTimeElapsed() throws Exception { // Make hasSufficientTimeElapsed public Method method = UserGroupInformation.class @@ -942,8 +915,7 @@ public void testHasSufficientTimeElapsed() throws Exception { method.setAccessible(false); } - @Test - @Timeout(value = 10) + @Test(timeout=10000) public void testSetLoginUser() throws IOException { UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test-user"); UserGroupInformation.setLoginUser(ugi); @@ -1010,8 +982,8 @@ public Void run() throws Exception { Token t = mock(Token.class); when(t.getService()).thenReturn(new Text("t" + i)); UserGroupInformation.getCurrentUser().addToken(t); - assertNull( - thread.cme, "ConcurrentModificationException encountered"); + assertNull("ConcurrentModificationException encountered", + thread.cme); } } catch (ConcurrentModificationException cme) { cme.printStackTrace(); @@ -1167,7 +1139,7 @@ public void testGetNextRetryTime() throws Exception { String str = "5th retry, now:" + currentTime + ", retry:" + lastRetry; LOG.info(str); - assertEquals(endTime - reloginIntervalMs, lastRetry, str); + assertEquals(str, endTime - reloginIntervalMs, lastRetry); // make sure no more retries after (tgt endTime - login interval). UserGroupInformation.metrics.getRenewalFailures().incr(); @@ -1175,7 +1147,7 @@ public void testGetNextRetryTime() throws Exception { UserGroupInformation.getNextTgtRenewalTime(endTime, currentTime, rp); str = "overflow retry, now:" + currentTime + ", retry:" + lastRetry; LOG.info(str); - assertEquals(endTime - reloginIntervalMs, lastRetry, str); + assertEquals(str, endTime - reloginIntervalMs, lastRetry); } private void assertWithinBounds(final int numFailures, final long lastRetry, @@ -1188,13 +1160,12 @@ private void assertWithinBounds(final int numFailures, final long lastRetry, + ", lower bound:" + lower + ", upper bound:" + upper + ", retry:" + lastRetry); LOG.info(str); - assertTrue(lower <= lastRetry && lastRetry < upper, str); + assertTrue(str, lower <= lastRetry && lastRetry < upper); } // verify that getCurrentUser on the same and different subjects can be // concurrent. Ie. no synchronization. - @Test - @Timeout(value = 8) + @Test(timeout=8000) public void testConcurrentGetCurrentUser() throws Exception { final CyclicBarrier barrier = new CyclicBarrier(2); final CountDownLatch latch = new CountDownLatch(1); @@ -1335,8 +1306,8 @@ public void testImportTokensFromConfig() throws IOException { // Check if the tokens were loaded UserGroupInformation ugi = UserGroupInformation.getLoginUser(); Credentials outCred = ugi.getCredentials(); - assertEquals( - 2, outCred.getAllTokens().size(), "Tokens: " + outCred.getAllTokens()); + assertEquals("Tokens: " + outCred.getAllTokens(), + 2, outCred.getAllTokens().size()); boolean found0 = false; boolean found1 = false; for (Token token : outCred.getAllTokens()) { @@ -1350,10 +1321,10 @@ public void testImportTokensFromConfig() throws IOException { found1 = true; } } - assertTrue( - found0, "Expected token testTokenService0 not found: " + outCred); - assertTrue( - found1, "Expected token testTokenService1 not found: " + outCred); + assertTrue("Expected token testTokenService0 not found: " + outCred, + found0); + assertTrue("Expected token testTokenService1 not found: " + outCred, + found1); // Try to add the same token through configuration and file Credentials cred1 = new Credentials(); @@ -1365,8 +1336,8 @@ public void testImportTokensFromConfig() throws IOException { UserGroupInformation ugi1 = UserGroupInformation.getLoginUser(); Credentials outCred1 = ugi1.getCredentials(); - assertEquals( - 1, outCred1.getAllTokens().size(), "Tokens: " + outCred1.getAllTokens()); + assertEquals("Tokens: " + outCred1.getAllTokens(), + 1, outCred1.getAllTokens().size()); } @Test @@ -1385,8 +1356,8 @@ public void testImportTokensFromProperty() throws IOException { UserGroupInformation.reset(); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); Credentials creds = ugi.getCredentials(); - assertEquals( - 1, creds.getAllTokens().size(), "Tokens: " + creds.getAllTokens()); + assertEquals("Tokens: " + creds.getAllTokens(), + 1, creds.getAllTokens().size()); assertArrayEquals(creds.getToken(service).getIdentifier(), identity); // Cleanup diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java index a55322a849e02..4a6911035c6bd 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java @@ -22,7 +22,7 @@ import java.util.Map; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.TestFileBasedIPList; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/ServiceAssert.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/ServiceAssert.java index e341e5993a3d2..9f7250d129888 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/ServiceAssert.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/ServiceAssert.java @@ -19,12 +19,12 @@ package org.apache.hadoop.service; import org.apache.hadoop.service.Service; -import org.junit.jupiter.api.Assertions; +import org.junit.Assert; /** * A set of assertions about the state of any service */ -public class ServiceAssert extends Assertions { +public class ServiceAssert extends Assert { public static void assertServiceStateCreated(Service service) { assertServiceInState(service, Service.STATE.NOTINITED); @@ -43,9 +43,9 @@ public static void assertServiceStateStopped(Service service) { } public static void assertServiceInState(Service service, Service.STATE state) { - assertNotNull(service, "Null service"); - assertEquals(state -, service.getServiceState(), "Service in wrong state: " + service); + assertNotNull("Null service", service); + assertEquals("Service in wrong state: " + service, state, + service.getServiceState()); } /** @@ -58,7 +58,7 @@ public static void assertServiceInState(Service service, Service.STATE state) { public static void assertStateCount(BreakableService service, Service.STATE state, int expected) { - assertNotNull(service, "Null service"); + assertNotNull("Null service", service); int actual = service.getCount(state); if (expected != actual) { fail("Expected entry count for state [" + state +"] of " + service @@ -74,7 +74,7 @@ public static void assertStateCount(BreakableService service, */ public static void assertServiceConfigurationContains(Service service, String key) { - assertNotNull( - service.getConfig().get(key), "No option "+ key + " in service configuration"); + assertNotNull("No option "+ key + " in service configuration", + service.getConfig().get(key)); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestCompositeService.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestCompositeService.java index f518c669e108c..c75d8a08fba63 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestCompositeService.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestCompositeService.java @@ -66,8 +66,8 @@ public void testCallSequence() { CompositeServiceImpl[] services = serviceManager.getServices().toArray( new CompositeServiceImpl[0]); - assertEquals(NUM_OF_SERVICES -, services.length, "Number of registered services "); + assertEquals(NUM_OF_SERVICES, services.length, + "Number of registered services "); Configuration conf = new Configuration(); // Initialise the composite service @@ -78,9 +78,9 @@ public void testCallSequence() { // Verify the init() call sequence numbers for every service for (int i = 0; i < NUM_OF_SERVICES; i++) { - assertEquals(i -, services[i].getCallSequenceNumber(), "For " + services[i] - + " service, init() call sequence number should have been "); + assertEquals(i, services[i].getCallSequenceNumber(), + "For " + services[i] + + " service, init() call sequence number should have been "); } // Reset the call sequence numbers @@ -92,9 +92,9 @@ public void testCallSequence() { // Verify the start() call sequence numbers for every service for (int i = 0; i < NUM_OF_SERVICES; i++) { - assertEquals(i -, services[i].getCallSequenceNumber(), "For " + services[i] - + " service, start() call sequence number should have been "); + assertEquals(i, + services[i].getCallSequenceNumber(), "For " + services[i] + + " service, start() call sequence number should have been "); } resetServices(services); @@ -105,18 +105,18 @@ public void testCallSequence() { // Verify the stop() call sequence numbers for every service for (int i = 0; i < NUM_OF_SERVICES; i++) { - assertEquals( - ((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber(), "For " + services[i] - + " service, stop() call sequence number should have been "); + assertEquals(((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber(), + "For " + services[i] + + " service, stop() call sequence number should have been "); } // Try to stop again. This should be a no-op. serviceManager.stop(); // Verify that stop() call sequence numbers for every service don't change. for (int i = 0; i < NUM_OF_SERVICES; i++) { - assertEquals( - ((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber(), "For " + services[i] - + " service, stop() call sequence number should have been "); + assertEquals(((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber(), + "For " + services[i] + + " service, stop() call sequence number should have been "); } } @@ -156,11 +156,11 @@ public void testServiceStartup() { for (int i = 0; i < NUM_OF_SERVICES - 1; i++) { if (i >= FAILED_SERVICE_SEQ_NUMBER && STOP_ONLY_STARTED_SERVICES) { // Failed service state should be INITED - assertEquals(STATE.INITED -, services[NUM_OF_SERVICES - 1].getServiceState(), "Service state should have been "); + assertEquals(STATE.INITED, services[NUM_OF_SERVICES - 1].getServiceState(), + "Service state should have been "); } else { - assertEquals(STATE.STOPPED -, services[i].getServiceState(), "Service state should have been "); + assertEquals(STATE.STOPPED, services[i].getServiceState(), + "Service state should have been "); } } @@ -224,10 +224,8 @@ private void assertInState(STATE expected, } private void assertInState(STATE expected, Service service) { - assertEquals( - expected -, service.getServiceState(), "Service state should have been " + expected + " in " - + service); + assertEquals(expected, service.getServiceState(), + "Service state should have been " + expected + " in " + service); } /** @@ -322,8 +320,7 @@ public void testAddIfService() { @Override public void serviceInit(Configuration conf) { Integer notAService = new Integer(0); - assertFalse( - addIfService(notAService), "Added an integer as a service"); + assertFalse(addIfService(notAService), "Added an integer as a service"); service = new AbstractService("Service") {}; assertTrue(addIfService(service), "Unable to add a service"); @@ -331,8 +328,8 @@ public void serviceInit(Configuration conf) { }; testService.init(new Configuration()); - assertEquals( - 1, testService.getServices().size(), "Incorrect number of services"); + assertEquals(1, testService.getServices().size(), + "Incorrect number of services"); } @Test @@ -341,8 +338,7 @@ public void testRemoveService() { @Override public void serviceInit(Configuration conf) { Integer notAService = new Integer(0); - assertFalse( - addIfService(notAService), "Added an integer as a service"); + assertFalse(addIfService(notAService), "Added an integer as a service"); Service service1 = new AbstractService("Service1") {}; addIfService(service1); @@ -358,8 +354,8 @@ public void serviceInit(Configuration conf) { }; testService.init(new Configuration()); - assertEquals( - 2, testService.getServices().size(), "Incorrect number of services"); + assertEquals(2, testService.getServices().size(), + "Incorrect number of services"); } // @@ -397,8 +393,8 @@ public void testAddUninitedChildInInit() throws Throwable { assertInState(STATE.NOTINITED, child); parent.stop(); assertInState(STATE.NOTINITED, child); - assertEquals( - 1, parent.getServices().size(), "Incorrect number of services"); + assertEquals(1, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -497,8 +493,8 @@ public void testAddStartedChildBeforeInit() throws Throwable { //expected } parent.stop(); - assertEquals( - 1, parent.getServices().size(), "Incorrect number of services"); + assertEquals(1, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -561,8 +557,8 @@ public void testAddStoppedChildBeforeInit() throws Throwable { //expected } parent.stop(); - assertEquals( - 1, parent.getServices().size(), "Incorrect number of services"); + assertEquals(1, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -583,8 +579,8 @@ public void testAddStoppedChildInInit() throws Throwable { } assertInState(STATE.STOPPED, child); parent.stop(); - assertEquals( - 1, parent.getServices().size(), "Incorrect number of services"); + assertEquals(1, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -633,8 +629,8 @@ public void testAddUninitedSiblingBeforeInit() throws Throwable { assertInState(STATE.NOTINITED, sibling); parent.stop(); assertInState(STATE.NOTINITED, sibling); - assertEquals( - 1, parent.getServices().size(), "Incorrect number of services"); + assertEquals(1, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -653,8 +649,8 @@ public void testAddUninitedSiblingInInit() throws Throwable { //expected } parent.stop(); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -671,8 +667,8 @@ public void testAddUninitedSiblingInStart() throws Throwable { assertInState(STATE.NOTINITED, sibling); parent.stop(); assertInState(STATE.NOTINITED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -689,8 +685,8 @@ public void testAddUninitedSiblingInStop() throws Throwable { assertInState(STATE.NOTINITED, sibling); parent.stop(); assertInState(STATE.NOTINITED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -708,8 +704,8 @@ public void testAddInitedSiblingBeforeInit() throws Throwable { assertInState(STATE.INITED, sibling); parent.stop(); assertInState(STATE.INITED, sibling); - assertEquals( - 1, parent.getServices().size(), "Incorrect number of services"); + assertEquals(1, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -727,8 +723,8 @@ public void testAddInitedSiblingInInit() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -746,8 +742,8 @@ public void testAddInitedSiblingInStart() throws Throwable { assertInState(STATE.INITED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -778,8 +774,8 @@ public void testAddStartedSiblingBeforeInit() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STARTED, sibling); - assertEquals( - 1, parent.getServices().size(), "Incorrect number of services"); + assertEquals(1, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -798,8 +794,8 @@ public void testAddStartedSiblingInInit() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @@ -819,8 +815,8 @@ public void testAddStartedSiblingInStart() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -839,8 +835,8 @@ public void testAddStartedSiblingInStop() throws Throwable { assertInState(STATE.STARTED, sibling); parent.stop(); assertInState(STATE.STARTED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -860,8 +856,8 @@ public void testAddStoppedSiblingBeforeInit() throws Throwable { assertInState(STATE.STOPPED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals( - 1, parent.getServices().size(), "Incorrect number of services"); + assertEquals(1, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -885,8 +881,8 @@ public void testAddStoppedSiblingInInit() throws Throwable { } parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -906,8 +902,8 @@ public void testAddStoppedSiblingInStart() throws Throwable { assertInState(STATE.STOPPED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } @Test @@ -927,8 +923,8 @@ public void testAddStoppedSiblingInStop() throws Throwable { assertInState(STATE.STOPPED, sibling); parent.stop(); assertInState(STATE.STOPPED, sibling); - assertEquals( - 2, parent.getServices().size(), "Incorrect number of services"); + assertEquals(2, parent.getServices().size(), + "Incorrect number of services"); } public static class CompositeServiceAddingAChild extends CompositeService{ diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestGlobalStateChangeListener.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestGlobalStateChangeListener.java index 7d9b9e66d7b8e..7bee2d68c6d18 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestGlobalStateChangeListener.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestGlobalStateChangeListener.java @@ -23,8 +23,8 @@ import org.apache.hadoop.service.LoggingStateChangeListener; import org.apache.hadoop.service.Service; import org.apache.hadoop.service.ServiceStateChangeListener; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Test; +import org.junit.After; +import org.junit.Test; /** * Test global state changes. It is critical for all tests to clean up the @@ -58,7 +58,7 @@ private boolean unregister(ServiceStateChangeListener l) { /** * After every test case reset the list of global listeners. */ - @AfterEach + @After public void cleanup() { AbstractService.resetGlobalListeners(); } @@ -70,7 +70,7 @@ public void cleanup() { */ public void assertListenerState(BreakableStateChangeListener breakable, Service.STATE state) { - assertEquals(state, breakable.getLastState(), "Wrong state in " + breakable); + assertEquals("Wrong state in " + breakable, state, breakable.getLastState()); } /** @@ -80,8 +80,8 @@ public void assertListenerState(BreakableStateChangeListener breakable, */ public void assertListenerEventCount(BreakableStateChangeListener breakable, int count) { - assertEquals(count -, breakable.getEventCount(), "Wrong event count in " + breakable); + assertEquals("Wrong event count in " + breakable, count, + breakable.getEventCount()); } /** @@ -90,7 +90,7 @@ public void assertListenerEventCount(BreakableStateChangeListener breakable, @Test public void testRegisterListener() { register(); - assertTrue(unregister(), "listener not registered"); + assertTrue("listener not registered", unregister()); } /** @@ -100,9 +100,9 @@ public void testRegisterListener() { public void testRegisterListenerTwice() { register(); register(); - assertTrue(unregister(), "listener not registered"); + assertTrue("listener not registered", unregister()); //there should be no listener to unregister the second time - assertFalse(unregister(), "listener double registered"); + assertFalse("listener double registered", unregister()); } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceLifecycle.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceLifecycle.java index ccbc0a009fbf5..f72e130d75011 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceLifecycle.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceLifecycle.java @@ -25,7 +25,7 @@ import org.apache.hadoop.service.Service; import org.apache.hadoop.service.ServiceStateChangeListener; import org.apache.hadoop.service.ServiceStateException; -import org.junit.jupiter.api.Test; +import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -204,10 +204,10 @@ public void testStopFailingInitAndStop() throws Throwable { assertEquals(Service.STATE.INITED, svc.getFailureState()); Throwable failureCause = svc.getFailureCause(); - assertNotNull(failureCause, "Null failure cause in " + svc); + assertNotNull("Null failure cause in " + svc, failureCause); BreakableService.BrokenLifecycleEvent cause = (BreakableService.BrokenLifecycleEvent) failureCause; - assertNotNull(cause.state, "null state in " + cause + " raised by " + svc); + assertNotNull("null state in " + cause + " raised by " + svc, cause.state); assertEquals(Service.STATE.INITED, cause.state); } @@ -299,7 +299,7 @@ public synchronized void stateChanged(Service service) { private void assertEventCount(BreakableStateChangeListener listener, int expected) { - assertEquals(expected, listener.getEventCount(), listener.toString()); + assertEquals(listener.toString(), expected, listener.getEventCount()); } @Test @@ -343,7 +343,7 @@ public void testListenerWithNotifications() throws Throwable { long duration = System.currentTimeMillis() - start; assertEquals(Service.STATE.STOPPED, listener.notifyingState); assertServiceInState(service, Service.STATE.STOPPED); - assertTrue(duration < 10000, "Duration of " + duration + " too long"); + assertTrue("Duration of " + duration + " too long", duration < 10000); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceOperations.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceOperations.java index e7596f2a45a3a..558f93e197630 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceOperations.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/TestServiceOperations.java @@ -20,10 +20,10 @@ import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.junit.jupiter.api.Test; -import org.junit.runner.RunWith; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.Mock; import org.mockito.Mockito; -import org.mockito.junit.MockitoJUnitRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,7 +38,7 @@ /** * Test miscellaneous service operations through mocked failures. */ -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(MockitoExtension.class) public class TestServiceOperations { @Mock diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java index 6ac07a6942767..6570366423475 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java @@ -25,13 +25,11 @@ import static org.apache.hadoop.test.GenericTestUtils.*; import org.apache.hadoop.util.ExitCodeProvider; import org.apache.hadoop.util.ExitUtil; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeAll; -import org.junit.Rule; -import org.junit.rules.TestName; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,8 +39,8 @@ import java.io.OutputStream; import java.util.Arrays; import java.util.List; -import java.util.concurrent.TimeUnit; +@Timeout(15) public class AbstractServiceLauncherTestBase extends Assertions implements LauncherExitCodes { private static final Logger LOG = LoggerFactory.getLogger( @@ -54,18 +52,6 @@ public class AbstractServiceLauncherTestBase extends Assertions implements */ private Service serviceToTeardown; - /** - * All tests have a short life. - */ - @Rule - public Timeout testTimeout = new Timeout(15000, TimeUnit.MILLISECONDS); - - /** - * Rule to provide the method name. - */ - @Rule - public TestName methodName = new TestName(); - /** * Turn off the exit util JVM exits, downgrading them to exception throws. */ diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java index 668a523530d14..dd3603b261d8b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java @@ -24,6 +24,7 @@ import org.apache.hadoop.service.launcher.testservices.RunningService; import static org.apache.hadoop.service.launcher.LauncherArguments.*; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import java.io.File; import java.io.FileWriter; @@ -97,8 +98,7 @@ public void testConfExtraction() throws Throwable { List args = launcher.extractCommandOptions(extracted, argsList); if (!args.isEmpty()) { - assertEquals( - 0, args.size(), "args beginning with " + args.get(0)); + assertEquals(0, args.size(), "args beginning with " + args.get(0)); } assertEquals("true", extracted.get("propagated", "unset")); } @@ -121,17 +121,16 @@ ARG_CONF_PREFIXED, configFile(conf1), List args = launcher.extractCommandOptions(extracted, argsList); if (!args.isEmpty()) { - assertEquals( - 0, args.size(), "args beginning with " + args.get(0)); + assertEquals(0, args.size(), "args beginning with " + args.get(0)); } assertTrue(extracted.getBoolean(key1, false)); assertEquals(7, extracted.getInt(key2, -1)); } @Test - public void testConfArgWrongFiletype() throws Throwable { + public void testConfArgWrongFiletype(TestInfo testInfo) throws Throwable { new File(CONF_FILE_DIR).mkdirs(); - File file = new File(CONF_FILE_DIR, methodName.getMethodName()); + File file = new File(CONF_FILE_DIR, testInfo.getDisplayName()); try (FileWriter fileWriter = new FileWriter(file)) { fileWriter.write("not-a-conf-file"); fileWriter.close(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java index b25dc46e1ae45..16fb47a1cdaa7 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java @@ -68,10 +68,10 @@ public void testInterruptEscalationShutdown() throws Throwable { } //the service is now stopped assertStopped(service); - assertTrue( - escalator.isSignalAlreadyReceived(), "isSignalAlreadyReceived() == false in " + escalator); - assertFalse( - escalator.isForcedShutdownTimedOut(), "isForcedShutdownTimedOut() == true in " + escalator); + assertTrue(escalator.isSignalAlreadyReceived(), + "isSignalAlreadyReceived() == false in " + escalator); + assertFalse(escalator.isForcedShutdownTimedOut(), + "isForcedShutdownTimedOut() == true in " + escalator); // now interrupt it a second time and expect it to escalate to a halt try { @@ -99,8 +99,8 @@ public void testBlockingShutdownTimeouts() throws Throwable { assertExceptionDetails(EXIT_INTERRUPTED, "", e); } - assertTrue( - escalator.isForcedShutdownTimedOut(), "isForcedShutdownTimedOut() == false in " + escalator); + assertTrue(escalator.isForcedShutdownTimedOut(), + "isForcedShutdownTimedOut() == false in " + escalator); } private static class InterruptCatcher implements IrqHandler.Interrupted { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestJUnitSetup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestJUnitSetup.java index a41618665b4da..2c33d383bc93f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestJUnitSetup.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestJUnitSetup.java @@ -18,11 +18,12 @@ package org.apache.hadoop.test; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.junit.jupiter.api.Assertions.fail; + public class TestJUnitSetup { public static final Logger LOG = LoggerFactory.getLogger(TestJUnitSetup.class); @@ -35,6 +36,6 @@ public void testJavaAssert() { LOG.info("The AssertionError is expected.", ae); return; } - Assertions.fail("Java assert does not work."); + fail("Java assert does not work."); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java index 15159129a7cea..390d8ded967a4 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java @@ -124,8 +124,8 @@ protected void assertRetryCount(int expected) { * @param minCount minimum value */ protected void assertMinRetryCount(int minCount) { - assertTrue( - minCount <= retry.getInvocationCount(), "retry count of " + retry + " is not >= " + minCount); + assertTrue(minCount <= retry.getInvocationCount(), + "retry count of " + retry + " is not >= " + minCount); } /** @@ -181,8 +181,8 @@ public void testAwaitLinearRetry() throws Throwable { TIMEOUT_FAILURE_HANDLER); fail("should not have got here"); } catch (TimeoutException e) { - assertEquals( - 2, linearRetry.getInvocationCount(), linearRetry.toString()); + assertEquals(2, linearRetry.getInvocationCount(), + linearRetry.toString()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java index 7d87070abed19..cd0a94022857c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java @@ -17,7 +17,9 @@ */ package org.apache.hadoop.test; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; @@ -56,8 +58,7 @@ public void doWork() throws Exception { assertEquals(3, threadsRun.get()); // Test shouldn't have waited the full 30 seconds, since // the threads exited faster than that. - assertTrue( - et - st < 5000, "Test took " + (et - st) + "ms"); + assertTrue(et - st < 5000, "Test took " + (et - st) + "ms"); } @Test @@ -81,8 +82,7 @@ public void doWork() throws Exception { long et = Time.now(); // Test shouldn't have waited the full 30 seconds, since // the thread throws faster than that - assertTrue( - et - st < 5000, "Test took " + (et - st) + "ms"); + assertTrue(et - st < 5000, "Test took " + (et - st) + "ms"); } @Test @@ -106,8 +106,7 @@ public void doWork() throws Exception { long et = Time.now(); // Test shouldn't have waited the full 30 seconds, since // the thread throws faster than that - assertTrue( - et - st < 5000, "Test took " + (et - st) + "ms"); + assertTrue(et - st < 5000, "Test took " + (et - st) + "ms"); } @Test @@ -129,11 +128,11 @@ public void doAnAction() throws Exception { long elapsed = et - st; // Test should have waited just about 3 seconds - assertTrue( - Math.abs(elapsed - 3000) < 500, "Test took " + (et - st) + "ms"); + assertTrue(Math.abs(elapsed - 3000) < 500, + "Test took " + (et - st) + "ms"); // Counter should have been incremented lots of times in 3 full seconds - assertTrue( - counter.get() > 1000, "Counter value = " + counter.get()); + assertTrue(counter.get() > 1000, + "Counter value = " + counter.get()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java index 5478390c9dab5..6805dcd2fd4b3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java @@ -23,10 +23,11 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import org.junit.runner.notification.Failure; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestTimedOutTestsListener { @@ -156,16 +157,16 @@ public void testThreadDumpAndDeadlocks() throws Exception { Thread.sleep(100); } - Assertions.assertEquals(3, countStringOccurrences(s, "BLOCKED")); + assertEquals(3, countStringOccurrences(s, "BLOCKED")); - Failure failure = new Failure( - null, new Exception(TimedOutTestsListener.TEST_TIMED_OUT_PREFIX)); + RuntimeException failure = + new RuntimeException(TimedOutTestsListener.TEST_TIMED_OUT_PREFIX); StringWriter writer = new StringWriter(); new TimedOutTestsListener(new PrintWriter(writer)).testFailure(failure); String out = writer.toString(); - Assertions.assertTrue(out.contains("THREAD DUMP")); - Assertions.assertTrue(out.contains("DEADLOCKS DETECTED")); + assertTrue(out.contains("THREAD DUMP")); + assertTrue(out.contains("DEADLOCKS DETECTED")); System.out.println(out); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TimedOutTestsListener.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TimedOutTestsListener.java index 1bdeddb57ae89..8a61020ca1ac7 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TimedOutTestsListener.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TimedOutTestsListener.java @@ -30,14 +30,12 @@ import java.util.Map; import org.apache.hadoop.util.StringUtils; -import org.junit.runner.notification.Failure; -import org.junit.runner.notification.RunListener; /** * JUnit run listener which prints full thread dump into System.err * in case a test is failed due to timeout. */ -public class TimedOutTestsListener extends RunListener { +public class TimedOutTestsListener { static final String TEST_TIMED_OUT_PREFIX = "test timed out after"; @@ -53,8 +51,7 @@ public TimedOutTestsListener(PrintWriter output) { this.output = output; } - @Override - public void testFailure(Failure failure) throws Exception { + public void testFailure(RuntimeException failure) throws Exception { if (failure != null && failure.getMessage() != null && failure.getMessage().startsWith(TEST_TIMED_OUT_PREFIX)) { output.println("====> TEST TIMED OUT. PRINTING THREAD DUMP. <===="); From 74e9935e214474205a54d03af6afdafdf06d74e5 Mon Sep 17 00:00:00 2001 From: fanshilun Date: Fri, 7 Feb 2025 06:58:16 +0800 Subject: [PATCH 3/7] HADOOP-19415. Fix JunitTest & CheckStyle. --- .../hadoop/net/TestClusterTopology.java | 6 +- .../java/org/apache/hadoop/net/TestDNS.java | 5 +- .../hadoop/net/TestDNSDomainNameResolver.java | 2 +- .../org/apache/hadoop/net/TestNetUtils.java | 6 +- .../hadoop/oncrpc/TestRpcAcceptedReply.java | 2 +- .../org/apache/hadoop/oncrpc/TestXDR.java | 2 +- .../hadoop/security/alias/TestCredShell.java | 12 +- .../alias/TestCredentialProviderFactory.java | 18 +- .../authorize/TestAccessControlList.java | 15 +- .../security/authorize/TestProxyUsers.java | 2 +- .../security/http/TestCrossOriginFilter.java | 157 +++++++++--------- .../http/TestXFrameOptionsFilter.java | 22 ++- .../hadoop/security/ssl/TestSSLFactory.java | 42 +++-- .../web/TestWebDelegationToken.java | 157 +++++++++--------- 14 files changed, 229 insertions(+), 219 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java index c0b9e21f459bb..8732ce006042c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java @@ -106,13 +106,15 @@ public void testCountNumNodes() throws Exception { excludedNodes.add(node4); assertEquals(3, cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes), "excluded nodes with ROOT scope should be considered"); - assertEquals(2, cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), excludedNodes), + assertEquals(2, + cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), excludedNodes), "excluded nodes without ~ scope should be considered"); assertEquals(1, cluster.countNumOfAvailableNodes(deadNode.getNetworkLocation(), excludedNodes), "excluded nodes with rack scope should be considered"); // adding the node in excluded scope to excluded list excludedNodes.add(node2); - assertEquals(2, cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), excludedNodes), + assertEquals(2, + cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(), excludedNodes), "excluded nodes with ~ scope should be considered"); // getting count with non-exist scope. assertEquals(0, cluster.countNumOfAvailableNodes("/non-exist", excludedNodes), diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java index c2ead4b509993..66b7918605c36 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java @@ -81,9 +81,8 @@ public void testGetLocalHostIsFast() throws Exception { assertEquals(hostname3, hostname2); assertEquals(hostname2, hostname1); long interval = t2 - t1; - assertTrue( - - interval < 20000, "Took too long to determine local host - caching is not working"); + assertTrue(interval < 20000, + "Took too long to determine local host - caching is not working"); } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java index c260680f5df5d..f49f4a9bfa928 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNSDomainNameResolver.java @@ -35,7 +35,7 @@ public class TestDNSDomainNameResolver { public void testGetHostNameByIP() throws UnknownHostException { InetAddress localhost = InetAddress.getLocalHost(); assumeFalse(Objects.equals(localhost.getCanonicalHostName(), localhost.getHostAddress()), - "IP lookup support required"); + "IP lookup support required"); // Precondition: host name and canonical host name for unresolved returns an IP address. InetAddress unresolved = InetAddress.getByAddress(localhost.getHostAddress(), diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java index c332a8842be54..0b2d0b8abc821 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java @@ -745,17 +745,17 @@ public void testNormalizeHostName() { String summary = "original [" + StringUtils.join(hosts, ", ") + "]" + " normalized [" + StringUtils.join(normalizedHosts, ", ") + "]"; // when ipaddress is normalized, same address is expected in return - assertEquals(summary, hosts.get(0), normalizedHosts.get(0)); + assertEquals(hosts.get(0), normalizedHosts.get(0), summary); // for normalizing a resolvable hostname, resolved ipaddress is expected in return assertFalse(normalizedHosts.get(1).equals(hosts.get(1)), "Element 1 equal "+ summary); - assertEquals(summary, hosts.get(0), normalizedHosts.get(1)); + assertEquals(hosts.get(0), normalizedHosts.get(1), summary); // this address HADOOP-8372: when normalizing a valid resolvable hostname start with numeric, // its ipaddress is expected to return assertFalse(normalizedHosts.get(2).equals(hosts.get(2)), "Element 2 equal " + summary); // return the same hostname after normalizing a irresolvable hostname. - assertEquals(summary, hosts.get(3), normalizedHosts.get(3)); + assertEquals(hosts.get(3), normalizedHosts.get(3), summary); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java index 89651c64d122c..03e12ae069f2f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java @@ -43,7 +43,7 @@ public void testAcceptState() { @Test public void testAcceptStateFromInvalidValue() { assertThrows(IndexOutOfBoundsException.class, () -> - AcceptState.fromValue(6)); + AcceptState.fromValue(6)); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java index 7132fb949e703..00727c07bc710 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java @@ -30,7 +30,7 @@ private void serializeInt(int times) { XDR r = w.asReadOnlyWrap(); for (int i = 0; i < times; ++i) - assertEquals(WRITE_VALUE,r.readInt()); + assertEquals(WRITE_VALUE, r.readInt()); } private void serializeLong(int times) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java index 2a7b84d2112ed..c02763e16a8c2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java @@ -249,8 +249,8 @@ public void testCommandHelpExitsNormally() throws Exception { for (String cmd : Arrays.asList("create", "list", "delete")) { CredentialShell shell = new CredentialShell(); shell.setConf(new Configuration()); - assertEquals( - 0, shell.init(new String[] {cmd, "-help"}), "Expected help argument on " + cmd + " to return 0"); + assertEquals(0, shell.init(new String[] {cmd, "-help"}), + "Expected help argument on " + cmd + " to return 0"); } } @@ -258,13 +258,13 @@ public void testCommandHelpExitsNormally() throws Exception { public void testEmptyArgForCommands() throws Exception { CredentialShell shell = new CredentialShell(); String[] command = { "list", "-provider" }; - assertEquals(1 -, shell.init(command), "Expected empty argument on " + command + " to return 1"); + assertEquals(1, + shell.init(command), "Expected empty argument on " + command + " to return 1"); for (String cmd : Arrays.asList("create", "delete")) { shell.setConf(new Configuration()); - assertEquals(1 -, shell.init(new String[] { cmd }), "Expected empty argument on " + cmd + " to return 1"); + assertEquals(1, shell.init(new String[] { cmd }), + "Expected empty argument on " + cmd + " to return 1"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java index d5d762e6c167e..839c03eb52bca 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java @@ -34,9 +34,8 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; import org.junit.jupiter.api.BeforeEach; -import org.junit.Rule; import org.junit.jupiter.api.Test; -import org.junit.rules.TestName; +import org.junit.jupiter.api.TestInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,12 +48,9 @@ public class TestCredentialProviderFactory { public static final Logger LOG = LoggerFactory.getLogger(TestCredentialProviderFactory.class); - @Rule - public final TestName test = new TestName(); - @BeforeEach - public void announce() { - LOG.info("Running test " + test.getMethodName()); + public void announce(TestInfo testInfo) { + LOG.info("Running test " + testInfo.getDisplayName()); } private static char[] chars = { 'a', 'b', 'c', 'd', 'e', 'f', 'g', @@ -237,8 +233,8 @@ public void testLocalJksProvider() throws Exception { Path path = ProviderUtils.unnestUri(new URI(ourUrl)); FileSystem fs = path.getFileSystem(conf); FileStatus s = fs.getFileStatus(path); - assertEquals("Unexpected permissions: " + s.getPermission().toString(), - "rw-------", s.getPermission().toString()); + assertEquals("rw-------", s.getPermission().toString(), + "Unexpected permissions: " + s.getPermission().toString()); assertTrue(file.isFile(), file + " should exist"); // check permission retention after explicit change @@ -280,7 +276,7 @@ public void checkPermissionRetention(Configuration conf, String ourUrl, FileSystem fs = path.getFileSystem(conf); FileStatus s = fs.getFileStatus(path); - assertEquals("Permissions should have been retained from the preexisting " + - "keystore.", "rwxrwxrwx", s.getPermission().toString()); + assertEquals("rwxrwxrwx", s.getPermission().toString(), + "Permissions should have been retained from the preexisting keystore."); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java index 5b94c6d8a045b..87e3d2e791f54 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestAccessControlList.java @@ -461,14 +461,13 @@ public void testIsUserAllowed() { private void assertUserAllowed(UserGroupInformation ugi, AccessControlList acl) { - assertTrue( - acl.isUserAllowed(ugi), "User " + ugi + " is not granted the access-control!!"); + assertTrue(acl.isUserAllowed(ugi), + "User " + ugi + " is not granted the access-control!!"); } private void assertUserNotAllowed(UserGroupInformation ugi, AccessControlList acl) { - assertFalse( - acl.isUserAllowed(ugi), "User " + ugi + assertFalse(acl.isUserAllowed(ugi), "User " + ugi + " is incorrectly granted the access-control!!"); } @@ -481,12 +480,12 @@ public void testUseRealUserAclsForProxiedUser() { UserGroupInformation user1 = UserGroupInformation.createProxyUserForTesting("regularJane", realUserUgi, new String [] {"group1"}); - assertFalse( - acl.isUserAllowed(user1), "User " + user1 + " should not have been granted access."); + assertFalse(acl.isUserAllowed(user1), + "User " + user1 + " should not have been granted access."); acl = new AccessControlList(AccessControlList.USE_REAL_ACLS + realUser); - assertTrue( - acl.isUserAllowed(user1), "User " + user1 + " should have access but was denied."); + assertTrue(acl.isUserAllowed(user1), + "User " + user1 + " should have access but was denied."); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java index 45fe04514c0fd..8f99715427c7c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java @@ -479,7 +479,7 @@ public void testProxyUsersWithNullPrefix() throws Exception { @Test public void testProxyUsersWithEmptyPrefix() throws Exception { - assertThrows(IllegalArgumentException.class,() -> { + assertThrows(IllegalArgumentException.class, () -> { ProxyUsers.refreshSuperUserGroupsConfiguration(new Configuration(false), ""); }); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java index 6d5c36ee96bf5..4613dce12aff1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java @@ -32,9 +32,14 @@ import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.security.http.CrossOriginFilter; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.mockito.Mockito; + +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.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import static org.apache.hadoop.test.MockitoUtil.verifyZeroInteractions; @@ -49,12 +54,12 @@ public void testSameOrigin() throws ServletException, IOException { FilterConfig filterConfig = new FilterConfigTest(conf); // Origin is not specified for same origin requests - HttpServletRequest mockReq = Mockito.mock(HttpServletRequest.class); - Mockito.when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn(null); + HttpServletRequest mockReq = mock(HttpServletRequest.class); + when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn(null); // Objects to verify interactions based on request - HttpServletResponse mockRes = Mockito.mock(HttpServletResponse.class); - FilterChain mockChain = Mockito.mock(FilterChain.class); + HttpServletResponse mockRes = mock(HttpServletResponse.class); + FilterChain mockChain = mock(FilterChain.class); // Object under test CrossOriginFilter filter = new CrossOriginFilter(); @@ -62,7 +67,7 @@ public void testSameOrigin() throws ServletException, IOException { filter.doFilter(mockReq, mockRes, mockChain); verifyZeroInteractions(mockRes); - Mockito.verify(mockChain).doFilter(mockReq, mockRes); + verify(mockChain).doFilter(mockReq, mockRes); } @Test @@ -76,27 +81,27 @@ public void testAllowAllOrigins() throws ServletException, IOException { // Object under test CrossOriginFilter filter = new CrossOriginFilter(); filter.init(filterConfig); - Assertions.assertTrue(filter.areOriginsAllowed("example.com")); + assertTrue(filter.areOriginsAllowed("example.com")); } @Test public void testEncodeHeaders() { String validOrigin = "http://localhost:12345"; String encodedValidOrigin = CrossOriginFilter.encodeHeader(validOrigin); - Assertions.assertEquals( + assertEquals( validOrigin, encodedValidOrigin, "Valid origin encoding should match exactly"); String httpResponseSplitOrigin = validOrigin + " \nSecondHeader: value"; String encodedResponseSplitOrigin = CrossOriginFilter.encodeHeader(httpResponseSplitOrigin); - Assertions.assertEquals( + assertEquals( validOrigin, encodedResponseSplitOrigin, "Http response split origin should be protected against"); // Test Origin List String validOriginList = "http://foo.example.com:12345 http://bar.example.com:12345"; String encodedValidOriginList = CrossOriginFilter .encodeHeader(validOriginList); - Assertions.assertEquals( + assertEquals( validOriginList, encodedValidOriginList, "Valid origin list encoding should match exactly"); } @@ -113,17 +118,17 @@ public void testPatternMatchingOrigins() throws ServletException, IOException { filter.init(filterConfig); // match multiple sub-domains - Assertions.assertFalse(filter.areOriginsAllowed("example.com")); - Assertions.assertFalse(filter.areOriginsAllowed("foo:example.com")); - Assertions.assertTrue(filter.areOriginsAllowed("foo.example.com")); - Assertions.assertTrue(filter.areOriginsAllowed("foo.bar.example.com")); + assertFalse(filter.areOriginsAllowed("example.com")); + assertFalse(filter.areOriginsAllowed("foo:example.com")); + assertTrue(filter.areOriginsAllowed("foo.example.com")); + assertTrue(filter.areOriginsAllowed("foo.bar.example.com")); // First origin is allowed - Assertions.assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com")); + assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com")); // Second origin is allowed - Assertions.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com")); + assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com")); // No origin in list is allowed - Assertions.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); + assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); } @Test @@ -139,17 +144,17 @@ public void testRegexPatternMatchingOrigins() throws ServletException, IOExcepti filter.init(filterConfig); // match multiple sub-domains - Assertions.assertFalse(filter.areOriginsAllowed("example.com")); - Assertions.assertFalse(filter.areOriginsAllowed("foo:example.com")); - Assertions.assertTrue(filter.areOriginsAllowed("foo.example.com")); - Assertions.assertTrue(filter.areOriginsAllowed("foo.bar.example.com")); + assertFalse(filter.areOriginsAllowed("example.com")); + assertFalse(filter.areOriginsAllowed("foo:example.com")); + assertTrue(filter.areOriginsAllowed("foo.example.com")); + assertTrue(filter.areOriginsAllowed("foo.bar.example.com")); // First origin is allowed - Assertions.assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com")); + assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com")); // Second origin is allowed - Assertions.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com")); + assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com")); // No origin in list is allowed - Assertions.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); + assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); } @Test @@ -164,13 +169,13 @@ public void testComplexRegexPatternMatchingOrigins() throws ServletException, IO CrossOriginFilter filter = new CrossOriginFilter(); filter.init(filterConfig); - Assertions.assertTrue(filter.areOriginsAllowed("http://sub1.example.com")); - Assertions.assertTrue(filter.areOriginsAllowed("https://sub1.example.com")); - Assertions.assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234")); - Assertions.assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080")); + assertTrue(filter.areOriginsAllowed("http://sub1.example.com")); + assertTrue(filter.areOriginsAllowed("https://sub1.example.com")); + assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234")); + assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080")); // No origin in list is allowed - Assertions.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); + assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); } @Test @@ -186,23 +191,23 @@ public void testMixedRegexPatternMatchingOrigins() throws ServletException, IOEx CrossOriginFilter filter = new CrossOriginFilter(); filter.init(filterConfig); - Assertions.assertTrue(filter.areOriginsAllowed("http://sub1.example.com")); - Assertions.assertTrue(filter.areOriginsAllowed("https://sub1.example.com")); - Assertions.assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234")); - Assertions.assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080")); + assertTrue(filter.areOriginsAllowed("http://sub1.example.com")); + assertTrue(filter.areOriginsAllowed("https://sub1.example.com")); + assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234")); + assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080")); // match multiple sub-domains - Assertions.assertFalse(filter.areOriginsAllowed("example2.com")); - Assertions.assertFalse(filter.areOriginsAllowed("foo:example2.com")); - Assertions.assertTrue(filter.areOriginsAllowed("foo.example2.com")); - Assertions.assertTrue(filter.areOriginsAllowed("foo.bar.example2.com")); + assertFalse(filter.areOriginsAllowed("example2.com")); + assertFalse(filter.areOriginsAllowed("foo:example2.com")); + assertTrue(filter.areOriginsAllowed("foo.example2.com")); + assertTrue(filter.areOriginsAllowed("foo.bar.example2.com")); // First origin is allowed - Assertions.assertTrue(filter.areOriginsAllowed("foo.example2.com foo.nomatch.com")); + assertTrue(filter.areOriginsAllowed("foo.example2.com foo.nomatch.com")); // Second origin is allowed - Assertions.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example2.com")); + assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example2.com")); // No origin in list is allowed - Assertions.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); + assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com")); } @Test @@ -214,12 +219,12 @@ public void testDisallowedOrigin() throws ServletException, IOException { FilterConfig filterConfig = new FilterConfigTest(conf); // Origin is not specified for same origin requests - HttpServletRequest mockReq = Mockito.mock(HttpServletRequest.class); - Mockito.when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn("example.org"); + HttpServletRequest mockReq = mock(HttpServletRequest.class); + when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn("example.org"); // Objects to verify interactions based on request - HttpServletResponse mockRes = Mockito.mock(HttpServletResponse.class); - FilterChain mockChain = Mockito.mock(FilterChain.class); + HttpServletResponse mockRes = mock(HttpServletResponse.class); + FilterChain mockChain = mock(FilterChain.class); // Object under test CrossOriginFilter filter = new CrossOriginFilter(); @@ -227,7 +232,7 @@ public void testDisallowedOrigin() throws ServletException, IOException { filter.doFilter(mockReq, mockRes, mockChain); verifyZeroInteractions(mockRes); - Mockito.verify(mockChain).doFilter(mockReq, mockRes); + verify(mockChain).doFilter(mockReq, mockRes); } @Test @@ -239,15 +244,15 @@ public void testDisallowedMethod() throws ServletException, IOException { FilterConfig filterConfig = new FilterConfigTest(conf); // Origin is not specified for same origin requests - HttpServletRequest mockReq = Mockito.mock(HttpServletRequest.class); - Mockito.when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn("example.com"); - Mockito.when( + HttpServletRequest mockReq = mock(HttpServletRequest.class); + when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn("example.com"); + when( mockReq.getHeader(CrossOriginFilter.ACCESS_CONTROL_REQUEST_METHOD)) .thenReturn("DISALLOWED_METHOD"); // Objects to verify interactions based on request - HttpServletResponse mockRes = Mockito.mock(HttpServletResponse.class); - FilterChain mockChain = Mockito.mock(FilterChain.class); + HttpServletResponse mockRes = mock(HttpServletResponse.class); + FilterChain mockChain = mock(FilterChain.class); // Object under test CrossOriginFilter filter = new CrossOriginFilter(); @@ -255,7 +260,7 @@ public void testDisallowedMethod() throws ServletException, IOException { filter.doFilter(mockReq, mockRes, mockChain); verifyZeroInteractions(mockRes); - Mockito.verify(mockChain).doFilter(mockReq, mockRes); + verify(mockChain).doFilter(mockReq, mockRes); } @Test @@ -267,18 +272,18 @@ public void testDisallowedHeader() throws ServletException, IOException { FilterConfig filterConfig = new FilterConfigTest(conf); // Origin is not specified for same origin requests - HttpServletRequest mockReq = Mockito.mock(HttpServletRequest.class); - Mockito.when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn("example.com"); - Mockito.when( + HttpServletRequest mockReq = mock(HttpServletRequest.class); + when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn("example.com"); + when( mockReq.getHeader(CrossOriginFilter.ACCESS_CONTROL_REQUEST_METHOD)) .thenReturn("GET"); - Mockito.when( + when( mockReq.getHeader(CrossOriginFilter.ACCESS_CONTROL_REQUEST_HEADERS)) .thenReturn("Disallowed-Header"); // Objects to verify interactions based on request - HttpServletResponse mockRes = Mockito.mock(HttpServletResponse.class); - FilterChain mockChain = Mockito.mock(FilterChain.class); + HttpServletResponse mockRes = mock(HttpServletResponse.class); + FilterChain mockChain = mock(FilterChain.class); // Object under test CrossOriginFilter filter = new CrossOriginFilter(); @@ -286,7 +291,7 @@ public void testDisallowedHeader() throws ServletException, IOException { filter.doFilter(mockReq, mockRes, mockChain); verifyZeroInteractions(mockRes); - Mockito.verify(mockChain).doFilter(mockReq, mockRes); + verify(mockChain).doFilter(mockReq, mockRes); } @Test @@ -298,34 +303,34 @@ public void testCrossOriginFilter() throws ServletException, IOException { FilterConfig filterConfig = new FilterConfigTest(conf); // Origin is not specified for same origin requests - HttpServletRequest mockReq = Mockito.mock(HttpServletRequest.class); - Mockito.when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn("example.com"); - Mockito.when( + HttpServletRequest mockReq = mock(HttpServletRequest.class); + when(mockReq.getHeader(CrossOriginFilter.ORIGIN)).thenReturn("example.com"); + when( mockReq.getHeader(CrossOriginFilter.ACCESS_CONTROL_REQUEST_METHOD)) .thenReturn("GET"); - Mockito.when( + when( mockReq.getHeader(CrossOriginFilter.ACCESS_CONTROL_REQUEST_HEADERS)) .thenReturn("X-Requested-With"); // Objects to verify interactions based on request - HttpServletResponse mockRes = Mockito.mock(HttpServletResponse.class); - FilterChain mockChain = Mockito.mock(FilterChain.class); + HttpServletResponse mockRes = mock(HttpServletResponse.class); + FilterChain mockChain = mock(FilterChain.class); // Object under test CrossOriginFilter filter = new CrossOriginFilter(); filter.init(filterConfig); filter.doFilter(mockReq, mockRes, mockChain); - Mockito.verify(mockRes).setHeader(CrossOriginFilter.ACCESS_CONTROL_ALLOW_ORIGIN, + verify(mockRes).setHeader(CrossOriginFilter.ACCESS_CONTROL_ALLOW_ORIGIN, "example.com"); - Mockito.verify(mockRes).setHeader( + verify(mockRes).setHeader( CrossOriginFilter.ACCESS_CONTROL_ALLOW_CREDENTIALS, Boolean.TRUE.toString()); - Mockito.verify(mockRes).setHeader(CrossOriginFilter.ACCESS_CONTROL_ALLOW_METHODS, + verify(mockRes).setHeader(CrossOriginFilter.ACCESS_CONTROL_ALLOW_METHODS, filter.getAllowedMethodsHeader()); - Mockito.verify(mockRes).setHeader(CrossOriginFilter.ACCESS_CONTROL_ALLOW_HEADERS, + verify(mockRes).setHeader(CrossOriginFilter.ACCESS_CONTROL_ALLOW_HEADERS, filter.getAllowedHeadersHeader()); - Mockito.verify(mockChain).doFilter(mockReq, mockRes); + verify(mockChain).doFilter(mockReq, mockRes); } @Test @@ -343,13 +348,13 @@ public void testCrossOriginFilterAfterRestart() throws ServletException { filter.init(filterConfig); //verify filter values - Assertions.assertTrue( + assertTrue( filter.getAllowedHeadersHeader() .compareTo("X-Requested-With,Accept") == 0, "Allowed headers do not match"); - Assertions.assertTrue( + assertTrue( filter.getAllowedMethodsHeader() .compareTo("GET,POST") == 0, "Allowed methods do not match"); - Assertions.assertTrue(filter.areOriginsAllowed("example.com")); + assertTrue(filter.areOriginsAllowed("example.com")); //destroy filter values and clear conf filter.destroy(); @@ -365,13 +370,13 @@ public void testCrossOriginFilterAfterRestart() throws ServletException { filter.init(filterConfig); //verify filter values - Assertions.assertTrue( + assertTrue( filter.getAllowedHeadersHeader() .compareTo("Content-Type,Origin") == 0, "Allowed headers do not match"); - Assertions.assertTrue( + assertTrue( filter.getAllowedMethodsHeader() .compareTo("GET,HEAD") == 0, "Allowed methods do not match"); - Assertions.assertTrue(filter.areOriginsAllowed("newexample.com")); + assertTrue(filter.areOriginsAllowed("newexample.com")); //destroy filter values filter.destroy(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java index 497d8d150c25c..5d6c2790aebf0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java @@ -24,7 +24,6 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -32,7 +31,9 @@ import org.mockito.stubbing.Answer; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.any; /** * Test the default and customized behaviors of XFrameOptionsFilter. @@ -57,10 +58,8 @@ public void testDefaultOptionsValue() throws Exception { @Override public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); - Assertions.assertTrue( - - ((HttpServletResponse)args[1]). - containsHeader(X_FRAME_OPTIONS), "header should be visible inside chain and filters."); + assertTrue(((HttpServletResponse)args[1]).containsHeader(X_FRAME_OPTIONS), + "header should be visible inside chain and filters."); return null; } } @@ -71,7 +70,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { @Override public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); - Assertions.assertTrue( + assertTrue( "DENY".equals(args[1]), "Options value incorrect should be DENY but is: " + args[1]); headers.add((String)args[1]); @@ -108,12 +107,11 @@ public void testCustomOptionsValueAndNoOverrides() throws Exception { public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); HttpServletResponse resp = (HttpServletResponse) args[1]; - Assertions.assertTrue( - - resp.containsHeader(X_FRAME_OPTIONS), "Header should be visible inside chain and filters."); + assertTrue(resp.containsHeader(X_FRAME_OPTIONS), + "Header should be visible inside chain and filters."); // let's try and set another value for the header and make // sure that it doesn't overwrite the configured value - Assertions.assertTrue(resp instanceof + assertTrue(resp instanceof XFrameOptionsFilter.XFrameOptionsResponseWrapper); resp.setHeader(X_FRAME_OPTIONS, "LJM"); return null; @@ -126,7 +124,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { @Override public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); - Assertions.assertEquals("SAMEORIGIN", args[1], + assertEquals("SAMEORIGIN", args[1], "Options value incorrect should be SAMEORIGIN but is: " + args[1]); headers.add((String)args[1]); return null; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java index 89846a04305ea..e5f1a4b043ae8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java @@ -24,7 +24,14 @@ import static org.apache.hadoop.security.ssl.SSLFactory.Mode.CLIENT; import static org.apache.hadoop.security.ssl.SSLFactory.SSL_CLIENT_CONF_KEY; import static org.apache.hadoop.security.ssl.SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY; -import static org.junit.jupiter.api.Assertions.*; +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.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; @@ -34,7 +41,6 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.StringUtils; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -151,13 +157,13 @@ public void testSslConfClassPathFirst() throws Exception { @Test public void clientMode() throws Exception { - assertThrows(IllegalArgumentException.class, ()->{ + assertThrows(IllegalStateException.class, () -> { Configuration conf = createConfiguration(false, true); SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); try { sslFactory.init(); - Assertions.assertNotNull(sslFactory.createSSLSocketFactory()); - Assertions.assertNotNull(sslFactory.getHostnameVerifier()); + assertNotNull(sslFactory.createSSLSocketFactory()); + assertNotNull(sslFactory.getHostnameVerifier()); sslFactory.createSSLServerSocketFactory(); } finally { sslFactory.destroy(); @@ -170,8 +176,8 @@ private void serverMode(boolean clientCert, boolean socket) throws Exception { SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); try { sslFactory.init(); - Assertions.assertNotNull(sslFactory.createSSLServerSocketFactory()); - Assertions.assertEquals(clientCert, sslFactory.isClientCertRequired()); + assertNotNull(sslFactory.createSSLServerSocketFactory()); + assertEquals(clientCert, sslFactory.isClientCertRequired()); if (socket) { sslFactory.createSSLSocketFactory(); } else { @@ -185,13 +191,13 @@ private void serverMode(boolean clientCert, boolean socket) throws Exception { @Test public void serverModeWithoutClientCertsSocket() throws Exception { - assertThrows(IllegalArgumentException.class, + assertThrows(IllegalStateException.class, () -> serverMode(false, true)); } @Test public void serverModeWithClientCertsSocket() throws Exception { - assertThrows(IllegalArgumentException.class, () -> serverMode(true, true)); + assertThrows(IllegalStateException.class, () -> serverMode(true, true)); } @Test @@ -201,7 +207,7 @@ public void serverModeWithoutClientCertsVerifier() throws Exception { @Test public void serverModeWithClientCertsVerifier() throws Exception { - assertThrows(IllegalArgumentException.class, ()-> serverMode(true, false)); + assertThrows(IllegalStateException.class, ()-> serverMode(true, false)); } private void runDelegatedTasks(SSLEngineResult result, SSLEngine engine) @@ -299,7 +305,7 @@ public void testServerWeakCiphers() throws Exception { dataDone = true; } } - Assertions.fail("The exception was not thrown"); + fail("The exception was not thrown"); } catch (SSLHandshakeException e) { GenericTestUtils.assertExceptionContains("no cipher suites in common", e); } @@ -326,33 +332,33 @@ public void validHostnameVerifier() throws Exception { SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assertions.assertEquals("DEFAULT", sslFactory.getHostnameVerifier().toString()); + assertEquals("DEFAULT", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "ALLOW_ALL"); sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assertions.assertEquals("ALLOW_ALL", + assertEquals("ALLOW_ALL", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "DEFAULT_AND_LOCALHOST"); sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assertions.assertEquals("DEFAULT_AND_LOCALHOST", + assertEquals("DEFAULT_AND_LOCALHOST", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "STRICT"); sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assertions.assertEquals("STRICT", sslFactory.getHostnameVerifier().toString()); + assertEquals("STRICT", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "STRICT_IE6"); sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); sslFactory.init(); - Assertions.assertEquals("STRICT_IE6", + assertEquals("STRICT_IE6", sslFactory.getHostnameVerifier().toString()); sslFactory.destroy(); } @@ -394,10 +400,10 @@ public void testConnectionConfigurator() throws Exception { sslFactory.init(); HttpsURLConnection sslConn = (HttpsURLConnection) new URL("https://foo").openConnection(); - Assertions.assertNotSame("STRICT_IE6", + assertNotSame("STRICT_IE6", sslConn.getHostnameVerifier().toString()); sslFactory.configure(sslConn); - Assertions.assertEquals("STRICT_IE6", + assertEquals("STRICT_IE6", sslConn.getHostnameVerifier().toString()); } finally { sslFactory.destroy(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java index f40aa867b1403..f838cbce75b03 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java @@ -37,7 +37,6 @@ import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.eclipse.jetty.servlet.FilterHolder; @@ -78,6 +77,12 @@ import java.util.UUID; import java.util.concurrent.Callable; +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.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + public class TestWebDelegationToken { private static final String OK_USER = "ok-user"; private static final String FAIL_USER = "fail-user"; @@ -235,45 +240,45 @@ public void testRawHttpCalls() throws Exception { // unauthenticated access to URL HttpURLConnection conn = (HttpURLConnection) nonAuthURL.openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // authenticated access to URL conn = (HttpURLConnection) authURL.openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // unauthenticated access to get delegation token URL url = new URL(nonAuthURL.toExternalForm() + "?op=GETDELEGATIONTOKEN"); conn = (HttpURLConnection) url.openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // authenticated access to get delegation token url = new URL(authURL.toExternalForm() + "&op=GETDELEGATIONTOKEN&renewer=foo"); conn = (HttpURLConnection) url.openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ObjectMapper mapper = new ObjectMapper(); Map map = mapper.readValue(conn.getInputStream(), Map.class); String dt = (String) ((Map) map.get("Token")).get("urlString"); - Assertions.assertNotNull(dt); + assertNotNull(dt); // delegation token access to URL url = new URL(nonAuthURL.toExternalForm() + "?delegation=" + dt); conn = (HttpURLConnection) url.openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // delegation token and authenticated access to URL url = new URL(authURL.toExternalForm() + "&delegation=" + dt); conn = (HttpURLConnection) url.openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // renewew delegation token, unauthenticated access to URL url = new URL(nonAuthURL.toExternalForm() + "?op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assertions.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // renewew delegation token, authenticated access to URL @@ -281,14 +286,14 @@ public void testRawHttpCalls() throws Exception { "&op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // renewew delegation token, authenticated access to URL, not renewer url = new URL(getJettyURL() + "/foo/bar?authenticated=bar&op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assertions.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); // cancel delegation token, nonauthenticated access to URL @@ -296,32 +301,32 @@ public void testRawHttpCalls() throws Exception { "?op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // cancel canceled delegation token, nonauthenticated access to URL url = new URL(nonAuthURL.toExternalForm() + "?op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assertions.assertEquals(HttpURLConnection.HTTP_NOT_FOUND, + assertEquals(HttpURLConnection.HTTP_NOT_FOUND, conn.getResponseCode()); // get new delegation token url = new URL(authURL.toExternalForm() + "&op=GETDELEGATIONTOKEN&renewer=foo"); conn = (HttpURLConnection) url.openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); mapper = new ObjectMapper(); map = mapper.readValue(conn.getInputStream(), Map.class); dt = (String) ((Map) map.get("Token")).get("urlString"); - Assertions.assertNotNull(dt); + assertNotNull(dt); // cancel delegation token, authenticated access to URL url = new URL(authURL.toExternalForm() + "&op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); } finally { jetty.stop(); } @@ -364,32 +369,32 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) try { aUrl.getDelegationToken(nonAuthURL, token, FOO_USER); - Assertions.fail(); + fail(); } catch (Exception ex) { - Assertions.assertTrue(ex.getCause().getMessage().contains("401")); + assertTrue(ex.getCause().getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, FOO_USER); - Assertions.assertNotNull(token.getDelegationToken()); - Assertions.assertEquals(new Text("token-kind"), + assertNotNull(token.getDelegationToken()); + assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); aUrl.renewDelegationToken(authURL, token); try { aUrl.renewDelegationToken(nonAuthURL, token); - Assertions.fail(); + fail(); } catch (Exception ex) { - Assertions.assertTrue(ex.getMessage().contains("401")); + assertTrue(ex.getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, FOO_USER); try { aUrl.renewDelegationToken(authURL2, token); - Assertions.fail(); + fail(); } catch (Exception ex) { - Assertions.assertTrue(ex.getMessage().contains("403")); + assertTrue(ex.getMessage().contains("403")); } aUrl.getDelegationToken(authURL, token, FOO_USER); @@ -405,7 +410,7 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) try { aUrl.renewDelegationToken(nonAuthURL, token); } catch (Exception ex) { - Assertions.assertTrue(ex.getMessage().contains("401")); + assertTrue(ex.getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, "foo"); @@ -416,13 +421,13 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) @Override public Void run() throws Exception { HttpURLConnection conn = aUrl.openConnection(nonAuthURL, new DelegationTokenAuthenticatedURL.Token()); - Assertions.assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); + assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); if (useQS) { - Assertions.assertNull(conn.getHeaderField("UsingHeader")); - Assertions.assertNotNull(conn.getHeaderField("UsingQueryString")); + assertNull(conn.getHeaderField("UsingHeader")); + assertNotNull(conn.getHeaderField("UsingQueryString")); } else { - Assertions.assertNotNull(conn.getHeaderField("UsingHeader")); - Assertions.assertNull(conn.getHeaderField("UsingQueryString")); + assertNotNull(conn.getHeaderField("UsingHeader")); + assertNull(conn.getHeaderField("UsingQueryString")); } return null; } @@ -472,8 +477,8 @@ public void testExternalDelegationTokenSecretManager() throws Exception { new DelegationTokenAuthenticatedURL(); aUrl.getDelegationToken(authURL, token, FOO_USER); - Assertions.assertNotNull(token.getDelegationToken()); - Assertions.assertEquals(new Text("fooKind"), + assertNotNull(token.getDelegationToken()); + assertEquals(new Text("fooKind"), token.getDelegationToken().getKind()); } finally { @@ -553,17 +558,17 @@ public Void run() throws Exception { DelegationTokenAuthenticatedURL aUrl = new DelegationTokenAuthenticatedURL(); HttpURLConnection conn = aUrl.openConnection(url, token); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals(FOO_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(FOO_USER, ret.get(0)); try { aUrl.getDelegationToken(url, token, FOO_USER); - Assertions.fail(); + fail(); } catch (AuthenticationException ex) { - Assertions.assertTrue(ex.getMessage().contains( + assertTrue(ex.getMessage().contains( "delegation token operation")); } return null; @@ -623,15 +628,15 @@ public Void run() throws Exception { DelegationTokenAuthenticatedURL aUrl = new DelegationTokenAuthenticatedURL(); HttpURLConnection conn = aUrl.openConnection(url, token); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals(FOO_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(FOO_USER, ret.get(0)); aUrl.getDelegationToken(url, token, FOO_USER); - Assertions.assertNotNull(token.getDelegationToken()); - Assertions.assertEquals(new Text("token-kind"), + assertNotNull(token.getDelegationToken()); + assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); return null; } @@ -746,7 +751,7 @@ private void testKerberosDelegationTokenAuthenticator( final String doAsUser = doAs ? OK_USER : null; File testDir = new File("target/" + UUID.randomUUID().toString()); - Assertions.assertTrue(testDir.mkdirs()); + assertTrue(testDir.mkdirs()); MiniKdc kdc = new MiniKdc(MiniKdc.createConf(), testDir); final Server jetty = createJettyServer(); ServletContextHandler context = new ServletContextHandler(); @@ -774,9 +779,9 @@ private void testKerberosDelegationTokenAuthenticator( try { aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); - Assertions.fail(); + fail(); } catch (AuthenticationException ex) { - Assertions.assertTrue(ex.getCause().getMessage().contains("GSSException")); + assertTrue(ex.getCause().getMessage().contains("GSSException")); } doAsKerberosUser("client", keytabFile.getAbsolutePath(), @@ -785,8 +790,8 @@ private void testKerberosDelegationTokenAuthenticator( public Void call() throws Exception { aUrl.getDelegationToken( url, token, doAs ? doAsUser : "client", doAsUser); - Assertions.assertNotNull(token.getDelegationToken()); - Assertions.assertEquals(new Text("token-kind"), + assertNotNull(token.getDelegationToken()); + assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); // Make sure the token belongs to the right owner ByteArrayInputStream buf = new ByteArrayInputStream( @@ -796,29 +801,29 @@ public Void call() throws Exception { new DelegationTokenIdentifier(new Text("token-kind")); id.readFields(dis); dis.close(); - Assertions.assertEquals( + assertEquals( doAs ? new Text(OK_USER) : new Text("client"), id.getOwner()); if (doAs) { - Assertions.assertEquals(new Text("client"), id.getRealUser()); + assertEquals(new Text("client"), id.getRealUser()); } aUrl.renewDelegationToken(url, token, doAsUser); - Assertions.assertNotNull(token.getDelegationToken()); + assertNotNull(token.getDelegationToken()); aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); - Assertions.assertNotNull(token.getDelegationToken()); + assertNotNull(token.getDelegationToken()); try { aUrl.renewDelegationToken(url, token, doAsUser); - Assertions.fail(); + fail(); } catch (Exception ex) { - Assertions.assertTrue(ex.getMessage().contains("403")); + assertTrue(ex.getMessage().contains("403")); } aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); aUrl.cancelDelegationToken(url, token, doAsUser); - Assertions.assertNull(token.getDelegationToken()); + assertNull(token.getDelegationToken()); return null; } @@ -848,17 +853,17 @@ public void testProxyUser() throws Exception { url.toExternalForm(), FOO_USER, OK_USER); HttpURLConnection conn = (HttpURLConnection) new URL(strUrl).openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals(OK_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(OK_USER, ret.get(0)); strUrl = String.format("%s?user.name=%s&DOAS=%s", url.toExternalForm(), FOO_USER, OK_USER); conn = (HttpURLConnection) new URL(strUrl).openConnection(); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals(OK_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(OK_USER, ret.get(0)); UserGroupInformation ugi = UserGroupInformation.createRemoteUser(FOO_USER); ugi.doAs(new PrivilegedExceptionAction() { @@ -871,16 +876,16 @@ public Void run() throws Exception { // proxyuser using authentication handler authentication HttpURLConnection conn = aUrl.openConnection(url, token, OK_USER); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals(OK_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(OK_USER, ret.get(0)); // unauthorized proxy user using authentication handler authentication conn = aUrl.openConnection(url, token, FAIL_USER); - Assertions.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); // proxy using delegation token authentication @@ -892,12 +897,12 @@ public Void run() throws Exception { // requests using delegation token as auth do not honor doAs conn = aUrl.openConnection(url, token, OK_USER); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals(FOO_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(FOO_USER, ret.get(0)); return null; } @@ -954,21 +959,21 @@ public Void run() throws Exception { // user foo HttpURLConnection conn = aUrl.openConnection(url, token); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals("remoteuser=" + FOO_USER+ ":ugi=" + FOO_USER, + assertEquals(1, ret.size()); + assertEquals("remoteuser=" + FOO_USER+ ":ugi=" + FOO_USER, ret.get(0)); // user ok-user via proxyuser foo conn = aUrl.openConnection(url, token, OK_USER); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + + assertEquals(1, ret.size()); + assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + ":ugi=" + OK_USER, ret.get(0)); return null; @@ -1016,11 +1021,11 @@ public Void run() throws Exception { // user ok-user via proxyuser foo HttpURLConnection conn = aUrl.openConnection(url, token, OK_USER); - Assertions.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - Assertions.assertEquals(1, ret.size()); - Assertions.assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + + assertEquals(1, ret.size()); + assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + ":ugi=" + OK_USER, ret.get(0)); return null; From 34e9edca7fcaf4b3109b78cd6afb3141dbbbc959 Mon Sep 17 00:00:00 2001 From: fanshilun Date: Fri, 7 Feb 2025 12:14:39 +0800 Subject: [PATCH 4/7] HADOOP-19415. Fix JunitTest & CheckStyle. --- .../TestZKDelegationTokenSecretManager.java | 78 ++++++++-------- ...onTokenAuthenticationHandlerWithMocks.java | 88 +++++++++---------- 2 files changed, 83 insertions(+), 83 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java index c7ab5164edc62..25dae7e4fd5cc 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java @@ -53,16 +53,16 @@ import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.auth.DigestAuthenticationProvider; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; import org.junit.Rule; -import org.junit.jupiter.api.Test; +import org.junit.Test; import org.junit.rules.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.Assert.fail; public class TestZKDelegationTokenSecretManager { @@ -82,13 +82,13 @@ public class TestZKDelegationTokenSecretManager { @Rule public Timeout globalTimeout = new Timeout(300000, TimeUnit.MILLISECONDS); - @BeforeEach + @Before public void setup() throws Exception { zkServer = new TestingServer(); zkServer.start(); } - @AfterEach + @After public void tearDown() throws Exception { if (zkServer != null) { zkServer.close(); @@ -136,7 +136,7 @@ public void testMultiNodeOperationsImpl(boolean setZeroRetry) throws Exception { Token token = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token); + Assert.assertNotNull(token); tm2.verifyToken(token); tm2.renewToken(token, "foo"); tm1.verifyToken(token); @@ -150,7 +150,7 @@ public void testMultiNodeOperationsImpl(boolean setZeroRetry) throws Exception { token = (Token) tm2.createToken( UserGroupInformation.getCurrentUser(), "bar"); - Assertions.assertNotNull(token); + Assert.assertNotNull(token); tm1.verifyToken(token); tm1.renewToken(token, "bar"); tm2.verifyToken(token); @@ -177,15 +177,15 @@ public void testNodeUpAferAWhile() throws Exception { Token token1 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token1); + Assert.assertNotNull(token1); Token token2 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "bar"); - Assertions.assertNotNull(token2); + Assert.assertNotNull(token2); Token token3 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "boo"); - Assertions.assertNotNull(token3); + Assert.assertNotNull(token3); tm1.verifyToken(token1); tm1.verifyToken(token2); @@ -212,7 +212,7 @@ public void testNodeUpAferAWhile() throws Exception { Token token4 = (Token) tm2.createToken( UserGroupInformation.getCurrentUser(), "xyz"); - Assertions.assertNotNull(token4); + Assert.assertNotNull(token4); tm2.verifyToken(token4); tm1.verifyToken(token4); @@ -253,19 +253,19 @@ public void testMultiNodeCompeteForSeqNum() throws Exception { Token token1 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token1); + Assert.assertNotNull(token1); AbstractDelegationTokenIdentifier id1 = tm1.getDelegationTokenSecretManager().decodeTokenIdentifier(token1); - Assertions.assertEquals( - 1, id1.getSequenceNumber(), "Token seq should be the same"); + Assert.assertEquals( + "Token seq should be the same", 1, id1.getSequenceNumber()); Token token2 = (Token) tm1.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token2); + Assert.assertNotNull(token2); AbstractDelegationTokenIdentifier id2 = tm1.getDelegationTokenSecretManager().decodeTokenIdentifier(token2); - Assertions.assertEquals( - 2, id2.getSequenceNumber(), "Token seq should be the same"); + Assert.assertEquals( + "Token seq should be the same", 2, id2.getSequenceNumber()); tm2 = new DelegationTokenManager(conf, new Text("bla")); tm2.init(); @@ -273,19 +273,19 @@ public void testMultiNodeCompeteForSeqNum() throws Exception { Token token3 = (Token) tm2.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token3); + Assert.assertNotNull(token3); AbstractDelegationTokenIdentifier id3 = tm2.getDelegationTokenSecretManager().decodeTokenIdentifier(token3); - Assertions.assertEquals( - 1001, id3.getSequenceNumber(), "Token seq should be the same"); + Assert.assertEquals( + "Token seq should be the same", 1001, id3.getSequenceNumber()); Token token4 = (Token) tm2.createToken( UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token4); + Assert.assertNotNull(token4); AbstractDelegationTokenIdentifier id4 = tm2.getDelegationTokenSecretManager().decodeTokenIdentifier(token4); - Assertions.assertEquals( - 1002, id4.getSequenceNumber(), "Token seq should be the same"); + Assert.assertEquals( + "Token seq should be the same", 1002, id4.getSequenceNumber()); verifyDestroy(tm1, conf); verifyDestroy(tm2, conf); @@ -304,7 +304,7 @@ public void testRenewTokenSingleManager() throws Exception { Token token = (Token) tm1.createToken(UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token); + Assert.assertNotNull(token); tm1.renewToken(token, "foo"); tm1.verifyToken(token); verifyDestroy(tm1, conf); @@ -324,7 +324,7 @@ public void testCancelTokenSingleManager() throws Exception { Token token = (Token) tm1.createToken(UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token); + Assert.assertNotNull(token); tm1.cancelToken(token, "foo"); try { verifyTokenFail(tm1, token); @@ -371,7 +371,7 @@ public void testStopThreads() throws Exception { Token token = (Token) tm1.createToken(UserGroupInformation.getCurrentUser(), "foo"); - Assertions.assertNotNull(token); + Assert.assertNotNull(token); tm1.destroy(); } @@ -420,8 +420,8 @@ public List getDefaultAcl() { private void verifyACL(CuratorFramework curatorFramework, String path, ACL expectedACL) throws Exception { List acls = curatorFramework.getACL().forPath(path); - Assertions.assertEquals(1, acls.size()); - Assertions.assertEquals(expectedACL, acls.get(0)); + Assert.assertEquals(1, acls.size()); + Assert.assertEquals(expectedACL, acls.get(0)); } // Since it is possible that there can be a delay for the cancel token message @@ -470,11 +470,11 @@ public void testNodesLoadedAfterRestart() throws Exception { Token token = (Token) tm .createToken(UserGroupInformation.getCurrentUser(), "good"); - Assertions.assertNotNull(token); + Assert.assertNotNull(token); Token cancelled = (Token) tm .createToken(UserGroupInformation.getCurrentUser(), "cancelled"); - Assertions.assertNotNull(cancelled); + Assert.assertNotNull(cancelled); tm.verifyToken(token); tm.verifyToken(cancelled); @@ -511,12 +511,12 @@ public Boolean get() { smNew.decodeTokenIdentifier(cancelled); AbstractDelegationTokenSecretManager.DelegationTokenInformation dtinfo = zksmNew.getTokenInfo(id); - Assertions.assertNull(dtinfo, "canceled dt should be gone!"); + Assert.assertNull("canceled dt should be gone!", dtinfo); // The good token should be loaded on startup, and removed after expiry. id = smNew.decodeTokenIdentifier(token); dtinfo = zksmNew.getTokenInfoFromMemory(id); - Assertions.assertNotNull(dtinfo, "good dt should be in memory!"); + Assert.assertNotNull("good dt should be in memory!", dtinfo); // Wait for the good token to expire. Thread.sleep(5000); @@ -556,7 +556,7 @@ public void testCreatingParentContainersIfNeeded() throws Exception { // Check if the created NameSpace exists. Stat stat = curatorFramework.checkExists().forPath(workingPath); - Assertions.assertNotNull(stat); + Assert.assertNotNull(stat); tm1.destroy(); curatorFramework.close(); @@ -584,7 +584,7 @@ public void testCreateNameSpaceRepeatedly() throws Exception { // Check if the created NameSpace exists. Stat stat = curatorFramework.checkExists().forPath(workingPath); - Assertions.assertNotNull(stat); + Assert.assertNotNull(stat); // Repeated creation will throw NodeExists exception LambdaTestUtils.intercept(KeeperException.class, @@ -624,10 +624,10 @@ public void testMultipleInit() throws Exception { List> futures = executorService.invokeAll( Arrays.asList(tm1Callable, tm2Callable)); for(Future future : futures) { - Assertions.assertTrue(future.get()); + Assert.assertTrue(future.get()); } executorService.shutdownNow(); - Assertions.assertTrue(executorService.awaitTermination(1, TimeUnit.SECONDS)); + Assert.assertTrue(executorService.awaitTermination(1, TimeUnit.SECONDS)); tm1.destroy(); tm2.destroy(); @@ -636,7 +636,7 @@ public void testMultipleInit() throws Exception { // Check if the created NameSpace exists. Stat stat = curatorFramework.checkExists().forPath(workingPath); - Assertions.assertNotNull(stat); + Assert.assertNotNull(stat); curatorFramework.close(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java index 4965d5041f92d..bc140fa7b1075 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java @@ -30,11 +30,11 @@ import org.apache.hadoop.security.authentication.server.AuthenticationToken; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.Token; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; import org.junit.Rule; -import org.junit.jupiter.api.Test; +import org.junit.Test; import org.junit.rules.Timeout; import org.mockito.Mockito; @@ -96,7 +96,7 @@ public AuthenticationToken authenticate(HttpServletRequest request, @Rule public Timeout testTimeout = new Timeout(120000, TimeUnit.MILLISECONDS); - @BeforeEach + @Before public void setUp() throws Exception { Properties conf = new Properties(); @@ -105,7 +105,7 @@ public void setUp() throws Exception { handler.initTokenManager(conf); } - @AfterEach + @After public void cleanUp() { handler.destroy(); } @@ -133,10 +133,10 @@ private void testNonManagementOperation() throws Exception { HttpServletRequest request = Mockito.mock(HttpServletRequest.class); Mockito.when(request.getParameter( DelegationTokenAuthenticator.OP_PARAM)).thenReturn(null); - Assertions.assertTrue(handler.managementOperation(null, request, null)); + Assert.assertTrue(handler.managementOperation(null, request, null)); Mockito.when(request.getParameter( DelegationTokenAuthenticator.OP_PARAM)).thenReturn("CREATE"); - Assertions.assertTrue(handler.managementOperation(null, request, null)); + Assert.assertTrue(handler.managementOperation(null, request, null)); } private void testManagementOperationErrors() throws Exception { @@ -148,7 +148,7 @@ private void testManagementOperationErrors() throws Exception { GETDELEGATIONTOKEN.toString() ); Mockito.when(request.getMethod()).thenReturn("FOO"); - Assertions.assertFalse(handler.managementOperation(null, request, response)); + Assert.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).sendError( Mockito.eq(HttpServletResponse.SC_BAD_REQUEST), Mockito.startsWith("Wrong HTTP method")); @@ -158,7 +158,7 @@ private void testManagementOperationErrors() throws Exception { DelegationTokenAuthenticator.DelegationTokenOperation. GETDELEGATIONTOKEN.getHttpMethod() ); - Assertions.assertFalse(handler.managementOperation(null, request, response)); + Assert.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus( Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED)); Mockito.verify(response).setHeader( @@ -181,7 +181,7 @@ private Token testGetToken(String renewer, Mockito.when(token.getUserName()).thenReturn("user"); Mockito.when(response.getWriter()).thenReturn(new PrintWriter( new StringWriter())); - Assertions.assertFalse(handler.managementOperation(token, request, response)); + Assert.assertFalse(handler.managementOperation(token, request, response)); String queryString = DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" + @@ -197,7 +197,7 @@ private Token testGetToken(String renewer, StringWriter writer = new StringWriter(); PrintWriter pwriter = new PrintWriter(writer); Mockito.when(response.getWriter()).thenReturn(pwriter); - Assertions.assertFalse(handler.managementOperation(token, request, response)); + Assert.assertFalse(handler.managementOperation(token, request, response)); if (renewer == null) { Mockito.verify(token).getUserName(); } else { @@ -209,8 +209,8 @@ private Token testGetToken(String renewer, String responseOutput = writer.toString(); String tokenLabel = DelegationTokenAuthenticator. DELEGATION_TOKEN_JSON; - Assertions.assertTrue(responseOutput.contains(tokenLabel)); - Assertions.assertTrue(responseOutput.contains( + Assert.assertTrue(responseOutput.contains(tokenLabel)); + Assert.assertTrue(responseOutput.contains( DelegationTokenAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON)); ObjectMapper jsonMapper = new ObjectMapper(); Map json = jsonMapper.readValue(responseOutput, Map.class); @@ -221,11 +221,11 @@ private Token testGetToken(String renewer, Token dt = new Token(); dt.decodeFromUrlString(tokenStr); handler.getTokenManager().verifyToken(dt); - Assertions.assertEquals(expectedTokenKind, dt.getKind()); + Assert.assertEquals(expectedTokenKind, dt.getKind()); if (service != null) { - Assertions.assertEquals(service, dt.getService().toString()); + Assert.assertEquals(service, dt.getService().toString()); } else { - Assertions.assertEquals(0, dt.getService().getLength()); + Assert.assertEquals(0, dt.getService().getLength()); } return dt; } @@ -251,7 +251,7 @@ private void testCancelToken(Token token) Mockito.when(request.getMethod()). thenReturn(op.getHttpMethod()); - Assertions.assertFalse(handler.managementOperation(null, request, response)); + Assert.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).sendError( Mockito.eq(HttpServletResponse.SC_BAD_REQUEST), Mockito.contains("requires the parameter [token]")); @@ -262,15 +262,15 @@ private void testCancelToken(Token token) DelegationTokenAuthenticator.TOKEN_PARAM + "=" + token.encodeToUrlString() ); - Assertions.assertFalse(handler.managementOperation(null, request, response)); + Assert.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_OK); try { handler.getTokenManager().verifyToken(token); - Assertions.fail(); + Assert.fail(); } catch (SecretManager.InvalidToken ex) { //NOP } catch (Throwable ex) { - Assertions.fail(); + Assert.fail(); } } @@ -295,7 +295,7 @@ private void testRenewToken(Token dToken, Mockito.when(request.getMethod()). thenReturn(op.getHttpMethod()); - Assertions.assertFalse(handler.managementOperation(null, request, response)); + Assert.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus( Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED)); Mockito.verify(response).setHeader(Mockito.eq( @@ -306,7 +306,7 @@ private void testRenewToken(Token dToken, Mockito.reset(response); AuthenticationToken token = Mockito.mock(AuthenticationToken.class); Mockito.when(token.getUserName()).thenReturn(testRenewer); - Assertions.assertFalse(handler.managementOperation(token, request, response)); + Assert.assertFalse(handler.managementOperation(token, request, response)); Mockito.verify(response).sendError( Mockito.eq(HttpServletResponse.SC_BAD_REQUEST), Mockito.contains("requires the parameter [token]")); @@ -320,10 +320,10 @@ private void testRenewToken(Token dToken, thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" + dToken.encodeToUrlString()); - Assertions.assertFalse(handler.managementOperation(token, request, response)); + Assert.assertFalse(handler.managementOperation(token, request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_OK); pwriter.close(); - Assertions.assertTrue(writer.toString().contains("long")); + Assert.assertTrue(writer.toString().contains("long")); handler.getTokenManager().verifyToken(dToken); } @@ -347,12 +347,12 @@ private void testValidDelegationTokenQueryString() throws Exception { dToken.encodeToUrlString()); AuthenticationToken token = handler.authenticate(request, response); - Assertions.assertEquals(UserGroupInformation.getCurrentUser(). + Assert.assertEquals(UserGroupInformation.getCurrentUser(). getShortUserName(), token.getUserName()); - Assertions.assertEquals(0, token.getExpires()); - Assertions.assertEquals(handler.getType(), + Assert.assertEquals(0, token.getExpires()); + Assert.assertEquals(handler.getType(), token.getType()); - Assertions.assertTrue(token.isExpired()); + Assert.assertTrue(token.isExpired()); } @SuppressWarnings("unchecked") @@ -367,12 +367,12 @@ private void testValidDelegationTokenHeader() throws Exception { dToken.encodeToUrlString()); AuthenticationToken token = handler.authenticate(request, response); - Assertions.assertEquals(UserGroupInformation.getCurrentUser(). + Assert.assertEquals(UserGroupInformation.getCurrentUser(). getShortUserName(), token.getUserName()); - Assertions.assertEquals(0, token.getExpires()); - Assertions.assertEquals(handler.getType(), + Assert.assertEquals(0, token.getExpires()); + Assert.assertEquals(handler.getType(), token.getType()); - Assertions.assertTrue(token.isExpired()); + Assert.assertTrue(token.isExpired()); } private void testInvalidDelegationTokenQueryString() throws Exception { @@ -382,9 +382,9 @@ private void testInvalidDelegationTokenQueryString() throws Exception { DelegationTokenAuthenticator.DELEGATION_PARAM + "=invalid"); StringWriter writer = new StringWriter(); Mockito.when(response.getWriter()).thenReturn(new PrintWriter(writer)); - Assertions.assertNull(handler.authenticate(request, response)); + Assert.assertNull(handler.authenticate(request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_FORBIDDEN); - Assertions.assertTrue(writer.toString().contains("AuthenticationException")); + Assert.assertTrue(writer.toString().contains("AuthenticationException")); } private void testInvalidDelegationTokenHeader() throws Exception { @@ -395,8 +395,8 @@ private void testInvalidDelegationTokenHeader() throws Exception { "invalid"); StringWriter writer = new StringWriter(); Mockito.when(response.getWriter()).thenReturn(new PrintWriter(writer)); - Assertions.assertNull(handler.authenticate(request, response)); - Assertions.assertTrue(writer.toString().contains("AuthenticationException")); + Assert.assertNull(handler.authenticate(request, response)); + Assert.assertTrue(writer.toString().contains("AuthenticationException")); } private String getToken() throws Exception { @@ -413,7 +413,7 @@ private String getToken() throws Exception { Mockito.when(token.getUserName()).thenReturn("user"); Mockito.when(response.getWriter()).thenReturn(new PrintWriter( new StringWriter())); - Assertions.assertFalse(handler.managementOperation(token, request, response)); + Assert.assertFalse(handler.managementOperation(token, request, response)); Mockito.when(request.getQueryString()). thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + @@ -425,7 +425,7 @@ private String getToken() throws Exception { StringWriter writer = new StringWriter(); PrintWriter pwriter = new PrintWriter(writer); Mockito.when(response.getWriter()).thenReturn(pwriter); - Assertions.assertFalse(handler.managementOperation(token, request, response)); + Assert.assertFalse(handler.managementOperation(token, request, response)); Mockito.verify(token).getUserName(); Mockito.verify(response).setStatus(HttpServletResponse.SC_OK); Mockito.verify(response).setContentType(MediaType.APPLICATION_JSON); @@ -433,8 +433,8 @@ private String getToken() throws Exception { String responseOutput = writer.toString(); String tokenLabel = DelegationTokenAuthenticator. DELEGATION_TOKEN_JSON; - Assertions.assertTrue(responseOutput.contains(tokenLabel)); - Assertions.assertTrue(responseOutput.contains( + Assert.assertTrue(responseOutput.contains(tokenLabel)); + Assert.assertTrue(responseOutput.contains( DelegationTokenAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON)); ObjectMapper jsonMapper = new ObjectMapper(); Map json = jsonMapper.readValue(responseOutput, Map.class); @@ -468,7 +468,7 @@ public void testCannotGetTokenUsingToken() throws Exception { StringWriter writer = new StringWriter(); PrintWriter pwriter = new PrintWriter(writer); Mockito.when(response.getWriter()).thenReturn(pwriter); - Assertions.assertFalse(handler.managementOperation(null, request, response)); + Assert.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_UNAUTHORIZED); } @@ -492,7 +492,7 @@ public void testCannotRenewTokenUsingToken() throws Exception { StringWriter writer = new StringWriter(); PrintWriter pwriter = new PrintWriter(writer); Mockito.when(response.getWriter()).thenReturn(pwriter); - Assertions.assertFalse(handler.managementOperation(null, request, response)); + Assert.assertFalse(handler.managementOperation(null, request, response)); Mockito.verify(response).setStatus(HttpServletResponse.SC_UNAUTHORIZED); } @@ -536,7 +536,7 @@ public void write(String str) { }; Mockito.when(response.getWriter()).thenReturn(printWriterCloseCount); - Assertions.assertFalse(noAuthCloseHandler.managementOperation(token, request, + Assert.assertFalse(noAuthCloseHandler.managementOperation(token, request, response)); } finally { noAuthCloseHandler.destroy(); From 466a778f7a1d81f5618c52861d4e07921f317e42 Mon Sep 17 00:00:00 2001 From: fanshilun Date: Fri, 7 Feb 2025 15:46:56 +0800 Subject: [PATCH 5/7] HADOOP-19415. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-common Part4. --- .../security/TestLdapGroupsMapping.java | 2 +- .../TestShellBasedUnixGroupsMapping.java | 8 +- .../security/TestUGILoginFromKeytab.java | 53 ++- .../hadoop/security/alias/TestCredShell.java | 2 +- .../alias/TestCredentialProviderFactory.java | 2 +- .../security/http/TestCrossOriginFilter.java | 32 +- .../http/TestXFrameOptionsFilter.java | 5 +- .../ssl/TestReloadingX509KeyManager.java | 320 +++++++++--------- .../security/token/TestDtUtilShell.java | 116 +++---- .../web/TestDelegationTokenManager.java | 19 +- .../web/TestWebDelegationToken.java | 141 ++++---- 11 files changed, 343 insertions(+), 357 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java index c3cc7cf54393e..1bb43ffcd5eb5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java @@ -353,7 +353,7 @@ public void testConfGetPassword() throws Exception { // let's make sure that a password that doesn't exist returns an // empty string as currently expected and used to trigger a call to // extract password - assertEquals("", mapping.getPassword(conf,"invalid-alias", "")); + assertEquals("", mapping.getPassword(conf, "invalid-alias", "")); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java index a23ea2413abee..ccade918177c9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedUnixGroupsMapping.java @@ -337,12 +337,12 @@ public void testFiniteGroupResolutionTime() throws Exception { ReflectionUtils.newInstance(TestDelayedGroupCommand.class, conf); executor = mapping.createGroupExecutor(userName); - assertEquals(defaultTimeout, - executor.getTimeoutInterval(), "Expected the group names executor to carry the default timeout"); + assertEquals(defaultTimeout, executor.getTimeoutInterval(), + "Expected the group names executor to carry the default timeout"); executor = mapping.createGroupIDExecutor(userName); - assertEquals(defaultTimeout, - executor.getTimeoutInterval(), "Expected the group ID executor to carry the default timeout"); + assertEquals(defaultTimeout, executor.getTimeoutInterval(), + "Expected the group ID executor to carry the default timeout"); mapping.getGroups(userName); assertFalse(shellMappingLog.getOutput().contains(commandTimeoutMessage), diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java index 9af75e209b421..f5eff7065c865 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java @@ -116,14 +116,14 @@ public void testUGILoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - assertTrue( - ugi.isFromKeytab(), "UGI should be configured to login from keytab"); + assertTrue(ugi.isFromKeytab(), + "UGI should be configured to login from keytab"); User user = getUser(ugi.getSubject()); assertNotNull(user.getLogin()); - assertTrue( - user.getLastLogin() > beforeLogin, "User login time is less than before login time, " + assertTrue(user.getLastLogin() > beforeLogin, + "User login time is less than before login time, " + "beforeLoginTime:" + beforeLogin + " userLoginTime:" + user.getLastLogin()); } @@ -139,8 +139,7 @@ public void testUGIReLoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - assertTrue( - ugi.isFromKeytab(), "UGI should be configured to login from keytab"); + assertTrue(ugi.isFromKeytab(), "UGI should be configured to login from keytab"); // Verify relogin from keytab. User user = getUser(ugi.getSubject()); @@ -154,8 +153,8 @@ public void testUGIReLoginFromKeytab() throws Exception { ugi.reloginFromKeytab(); final long secondLogin = user.getLastLogin(); final LoginContext login2 = user.getLogin(); - assertTrue( - secondLogin > firstLogin, "User should have been able to relogin from keytab"); + assertTrue(secondLogin > firstLogin, + "User should have been able to relogin from keytab"); assertNotNull(login2); assertNotSame(login1, login2); } @@ -174,8 +173,7 @@ public void testUGIForceReLoginFromKeytab() throws Exception { UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath()); UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - assertTrue( - ugi.isFromKeytab(), "UGI should be configured to login from keytab"); + assertTrue(ugi.isFromKeytab(), "UGI should be configured to login from keytab"); // Verify relogin from keytab. User user = getUser(ugi.getSubject()); @@ -190,8 +188,8 @@ public void testUGIForceReLoginFromKeytab() throws Exception { ugi.forceReloginFromKeytab(); final long secondLogin = user.getLastLogin(); final LoginContext login2 = user.getLogin(); - assertTrue( - secondLogin > firstLogin, "User should have been able to relogin from keytab"); + assertTrue(secondLogin > firstLogin, + "User should have been able to relogin from keytab"); assertNotNull(login2); assertNotSame(login1, login2); } @@ -287,11 +285,9 @@ public void testUGIRefreshFromKeytab() throws Exception { assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); assertTrue(ugi.isFromKeytab()); - assertTrue( - UserGroupInformation.isKerberosKeyTabLoginRenewalEnabled()); - assertTrue( - UserGroupInformation.getKerberosLoginRenewalExecutor() - .isPresent()); + assertTrue(UserGroupInformation.isKerberosKeyTabLoginRenewalEnabled()); + assertTrue(UserGroupInformation.getKerberosLoginRenewalExecutor() + .isPresent()); } @Test @@ -312,13 +308,11 @@ public void testUGIRefreshFromKeytabDisabled() throws Exception { UserGroupInformation ugi = UserGroupInformation.getLoginUser(); assertEquals(UserGroupInformation.AuthenticationMethod.KERBEROS, - ugi.getAuthenticationMethod()); + ugi.getAuthenticationMethod()); assertTrue(ugi.isFromKeytab()); - assertFalse( - UserGroupInformation.isKerberosKeyTabLoginRenewalEnabled()); - assertFalse( - UserGroupInformation.getKerberosLoginRenewalExecutor() - .isPresent()); + assertFalse(UserGroupInformation.isKerberosKeyTabLoginRenewalEnabled()); + assertFalse(UserGroupInformation.getKerberosLoginRenewalExecutor() + .isPresent()); } private static KerberosTicket getTicket(UserGroupInformation ugi) { @@ -331,10 +325,8 @@ private static KerberosTicket getTicket(UserGroupInformation ugi) { // the expected principal. private static KerberosTicket checkTicketAndKeytab(UserGroupInformation ugi, KerberosPrincipal principal, boolean expectIsKeytab) { - assertEquals( - principal.getName(), ugi.getUserName(), "wrong principal"); - assertEquals( - expectIsKeytab, ugi.isFromKeytab(), "is not keytab"); + assertEquals(principal.getName(), ugi.getUserName(), "wrong principal"); + assertEquals(expectIsKeytab, ugi.isFromKeytab(), "is not keytab"); KerberosTicket ticket = getTicket(ugi); assertNotNull(ticket, "no ticket"); assertEquals(principal, ticket.getClient(), "wrong principal"); @@ -438,9 +430,9 @@ public Void run() throws IOException { // original login user not affected. extLoginUser.reloginFromKeytab(); assertSame(extLoginUserTicket, - checkTicketAndKeytab(extLoginUser, principal2, false)); + checkTicketAndKeytab(extLoginUser, principal2, false)); assertSame(originalLoginUserTicket, - checkTicketAndKeytab(originalLoginUser, principal1, true)); + checkTicketAndKeytab(originalLoginUser, principal1, true)); // verify original login user gets a new ticket, new login user // not affected. @@ -540,8 +532,7 @@ public Void call() throws Exception { }); // wait for the thread to block on the barrier in the logout of the // relogin. - assertTrue( - latch.await(2, TimeUnit.SECONDS), "first relogin didn't block"); + assertTrue(latch.await(2, TimeUnit.SECONDS), "first relogin didn't block"); // although the logout removed the keytab instance, verify the ugi // knows from its login params that it is supposed to be from a keytab. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java index c02763e16a8c2..00c69f9d63e0e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java @@ -263,7 +263,7 @@ public void testEmptyArgForCommands() throws Exception { for (String cmd : Arrays.asList("create", "delete")) { shell.setConf(new Configuration()); - assertEquals(1, shell.init(new String[] { cmd }), + assertEquals(1, shell.init(new String[]{cmd}), "Expected empty argument on " + cmd + " to return 1"); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java index 839c03eb52bca..d5d7e521f603f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java @@ -277,6 +277,6 @@ public void checkPermissionRetention(Configuration conf, String ourUrl, FileSystem fs = path.getFileSystem(conf); FileStatus s = fs.getFileStatus(path); assertEquals("rwxrwxrwx", s.getPermission().toString(), - "Permissions should have been retained from the preexisting keystore."); + "Permissions should have been retained from the preexisting keystore."); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java index 4613dce12aff1..f7d70172e5c97 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java @@ -88,21 +88,21 @@ public void testAllowAllOrigins() throws ServletException, IOException { public void testEncodeHeaders() { String validOrigin = "http://localhost:12345"; String encodedValidOrigin = CrossOriginFilter.encodeHeader(validOrigin); - assertEquals( - validOrigin, encodedValidOrigin, "Valid origin encoding should match exactly"); + assertEquals(validOrigin, encodedValidOrigin, + "Valid origin encoding should match exactly"); String httpResponseSplitOrigin = validOrigin + " \nSecondHeader: value"; String encodedResponseSplitOrigin = CrossOriginFilter.encodeHeader(httpResponseSplitOrigin); - assertEquals( - validOrigin, encodedResponseSplitOrigin, "Http response split origin should be protected against"); + assertEquals(validOrigin, encodedResponseSplitOrigin, + "Http response split origin should be protected against"); // Test Origin List String validOriginList = "http://foo.example.com:12345 http://bar.example.com:12345"; String encodedValidOriginList = CrossOriginFilter .encodeHeader(validOriginList); - assertEquals( - validOriginList, encodedValidOriginList, "Valid origin list encoding should match exactly"); + assertEquals(validOriginList, encodedValidOriginList, + "Valid origin list encoding should match exactly"); } @Test @@ -348,12 +348,10 @@ public void testCrossOriginFilterAfterRestart() throws ServletException { filter.init(filterConfig); //verify filter values - assertTrue( - filter.getAllowedHeadersHeader() - .compareTo("X-Requested-With,Accept") == 0, "Allowed headers do not match"); - assertTrue( - filter.getAllowedMethodsHeader() - .compareTo("GET,POST") == 0, "Allowed methods do not match"); + assertTrue(filter.getAllowedHeadersHeader(). + compareTo("X-Requested-With,Accept") == 0, "Allowed headers do not match"); + assertTrue(filter.getAllowedMethodsHeader(). + compareTo("GET,POST") == 0, "Allowed methods do not match"); assertTrue(filter.areOriginsAllowed("example.com")); //destroy filter values and clear conf @@ -370,12 +368,10 @@ public void testCrossOriginFilterAfterRestart() throws ServletException { filter.init(filterConfig); //verify filter values - assertTrue( - filter.getAllowedHeadersHeader() - .compareTo("Content-Type,Origin") == 0, "Allowed headers do not match"); - assertTrue( - filter.getAllowedMethodsHeader() - .compareTo("GET,HEAD") == 0, "Allowed methods do not match"); + assertTrue(filter.getAllowedHeadersHeader().compareTo("Content-Type,Origin") == 0, + "Allowed headers do not match"); + assertTrue(filter.getAllowedMethodsHeader().compareTo("GET,HEAD") == 0, + "Allowed methods do not match"); assertTrue(filter.areOriginsAllowed("newexample.com")); //destroy filter values diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java index 5d6c2790aebf0..50cd009a11782 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestXFrameOptionsFilter.java @@ -70,9 +70,8 @@ public Object answer(InvocationOnMock invocation) throws Throwable { @Override public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); - assertTrue( - "DENY".equals(args[1]), "Options value incorrect should be DENY but is: " - + args[1]); + assertTrue("DENY".equals(args[1]), + "Options value incorrect should be DENY but is: " + args[1]); headers.add((String)args[1]); return null; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java index 768b88f9dd832..b3a74b745e0de 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java @@ -35,177 +35,179 @@ import java.util.function.Supplier; import static org.apache.hadoop.security.ssl.KeyStoreTestUtil.*; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; public class TestReloadingX509KeyManager { - private static final String BASEDIR = GenericTestUtils.getTempPath( - TestReloadingX509TrustManager.class.getSimpleName()); + private static final String BASEDIR = GenericTestUtils.getTempPath( + TestReloadingX509TrustManager.class.getSimpleName()); + + private final GenericTestUtils.LogCapturer reloaderLog = GenericTestUtils.LogCapturer.captureLogs( + FileMonitoringTimerTask.LOG); + + @BeforeAll + public static void setUp() throws Exception { + File base = new File(BASEDIR); + FileUtil.fullyDelete(base); + base.mkdirs(); + } + + @Test + public void testLoadMissingKeyStore() throws Exception { + assertThrows(IOException.class, () -> { + String keystoreLocation = BASEDIR + "/testmissing.jks"; + + ReloadingX509KeystoreManager tm = + new ReloadingX509KeystoreManager("jks", keystoreLocation, + "password", + "password"); + }); + } + + @Test + public void testLoadCorruptKeyStore() throws Exception { + assertThrows(IOException.class, () -> { + String keystoreLocation = BASEDIR + "/testcorrupt.jks"; + OutputStream os = new FileOutputStream(keystoreLocation); + os.write(1); + os.close(); + + ReloadingX509KeystoreManager tm = + new ReloadingX509KeystoreManager("jks", keystoreLocation, + "password", "password"); + }); + } + + @Test + @Timeout(value = 3000) + public void testReload() throws Exception { + KeyPair kp = generateKeyPair("RSA"); + X509Certificate sCert = generateCertificate("CN=localhost, O=server", kp, 30, + "SHA1withRSA"); + String keystoreLocation = BASEDIR + "/testreload.jks"; + createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), sCert); + + long reloadInterval = 10; + Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); + ReloadingX509KeystoreManager tm = + new ReloadingX509KeystoreManager("jks", keystoreLocation, + "password", "password"); + try { + fileMonitoringTimer.schedule(new FileMonitoringTimerTask( + Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); + assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); + + // Wait so that the file modification time is different + Thread.sleep((reloadInterval+ 1000)); + + // Change the certificate with a new keypair + final KeyPair anotherKP = generateKeyPair("RSA"); + sCert = KeyStoreTestUtil.generateCertificate("CN=localhost, O=server", anotherKP, 30, + "SHA1withRSA"); + createKeyStore(keystoreLocation, "password", "cert1", anotherKP.getPrivate(), sCert); - private final GenericTestUtils.LogCapturer reloaderLog = GenericTestUtils.LogCapturer.captureLogs( - FileMonitoringTimerTask.LOG); - - @BeforeAll - public static void setUp() throws Exception { - File base = new File(BASEDIR); - FileUtil.fullyDelete(base); - base.mkdirs(); + GenericTestUtils.waitFor(new Supplier() { + @Override + public Boolean get() { + return tm.getPrivateKey("cert1").equals(kp.getPrivate()); + } + }, (int) reloadInterval, 100000); + } finally { + fileMonitoringTimer.cancel(); } + } - @Test - public void testLoadMissingKeyStore() throws Exception { - assertThrows(IOException.class, () -> { - String keystoreLocation = BASEDIR + "/testmissing.jks"; + @Test + @Timeout(value = 30) + public void testReloadMissingTrustStore() throws Exception { + KeyPair kp = generateKeyPair("RSA"); + X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); + String keystoreLocation = BASEDIR + "/testmissing.jks"; + createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), cert1); - ReloadingX509KeystoreManager tm = - new ReloadingX509KeystoreManager("jks", keystoreLocation, - "password", - "password"); - }); - } + long reloadInterval = 10; + Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); + ReloadingX509KeystoreManager tm = + new ReloadingX509KeystoreManager("jks", keystoreLocation, + "password", + "password"); - @Test - public void testLoadCorruptKeyStore() throws Exception { - assertThrows(IOException.class, () -> { - String keystoreLocation = BASEDIR + "/testcorrupt.jks"; - OutputStream os = new FileOutputStream(keystoreLocation); - os.write(1); - os.close(); - - ReloadingX509KeystoreManager tm = - new ReloadingX509KeystoreManager("jks", keystoreLocation, - "password", "password"); - }); - } + try { + fileMonitoringTimer.schedule(new FileMonitoringTimerTask( + Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); + assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); - @Test - @Timeout(value = 3000) - public void testReload() throws Exception { - KeyPair kp = generateKeyPair("RSA"); - X509Certificate sCert = generateCertificate("CN=localhost, O=server", kp, 30, - "SHA1withRSA"); - String keystoreLocation = BASEDIR + "/testreload.jks"; - createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), sCert); - - long reloadInterval = 10; - Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); - ReloadingX509KeystoreManager tm = - new ReloadingX509KeystoreManager("jks", keystoreLocation, - "password", - "password"); - try { - fileMonitoringTimer.schedule(new FileMonitoringTimerTask( - Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); - assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); - - // Wait so that the file modification time is different - Thread.sleep((reloadInterval+ 1000)); - - // Change the certificate with a new keypair - final KeyPair anotherKP = generateKeyPair("RSA"); - sCert = KeyStoreTestUtil.generateCertificate("CN=localhost, O=server", anotherKP, 30, - "SHA1withRSA"); - createKeyStore(keystoreLocation, "password", "cert1", anotherKP.getPrivate(), sCert); - - GenericTestUtils.waitFor(new Supplier() { - @Override - public Boolean get() { - return tm.getPrivateKey("cert1").equals(kp.getPrivate()); - } - }, (int) reloadInterval, 100000); - } finally { - fileMonitoringTimer.cancel(); - } - } + assertFalse(reloaderLog.getOutput().contains( + FileMonitoringTimerTask.PROCESS_ERROR_MESSAGE)); - @Test - @Timeout(value = 30) - public void testReloadMissingTrustStore() throws Exception { - KeyPair kp = generateKeyPair("RSA"); - X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); - String keystoreLocation = BASEDIR + "/testmissing.jks"; - createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), cert1); - - long reloadInterval = 10; - Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); - ReloadingX509KeystoreManager tm = - new ReloadingX509KeystoreManager("jks", keystoreLocation, - "password", - "password"); - try { - fileMonitoringTimer.schedule(new FileMonitoringTimerTask( - Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); - assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); - - assertFalse(reloaderLog.getOutput().contains( - FileMonitoringTimerTask.PROCESS_ERROR_MESSAGE)); - - // Wait for the first reload to happen so we actually detect a change after the delete - Thread.sleep((reloadInterval+ 1000)); - - new File(keystoreLocation).delete(); - - // Wait for the reload to happen and log to get written to - Thread.sleep((reloadInterval+ 1000)); - - waitForFailedReloadAtLeastOnce((int) reloadInterval); - - assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); - } finally { - reloaderLog.stopCapturing(); - fileMonitoringTimer.cancel(); - } - } + // Wait for the first reload to happen so we actually detect a change after the delete + Thread.sleep((reloadInterval+ 1000)); + new File(keystoreLocation).delete(); - @Test - @Timeout(value = 30) - public void testReloadCorruptTrustStore() throws Exception { - KeyPair kp = generateKeyPair("RSA"); - X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); - String keystoreLocation = BASEDIR + "/testmissing.jks"; - createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), cert1); - - long reloadInterval = 10; - Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); - ReloadingX509KeystoreManager tm = - new ReloadingX509KeystoreManager("jks", keystoreLocation, - "password", - "password"); - try { - fileMonitoringTimer.schedule(new FileMonitoringTimerTask( - Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); - assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); - - // Wait so that the file modification time is different - Thread.sleep((reloadInterval + 1000)); - - assertFalse(reloaderLog.getOutput().contains( - FileMonitoringTimerTask.PROCESS_ERROR_MESSAGE)); - OutputStream os = new FileOutputStream(keystoreLocation); - os.write(1); - os.close(); - - waitForFailedReloadAtLeastOnce((int) reloadInterval); - - assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); - } finally { - reloaderLog.stopCapturing(); - fileMonitoringTimer.cancel(); - } - } + // Wait for the reload to happen and log to get written to + Thread.sleep((reloadInterval+ 1000)); - /**Wait for the reloader thread to load the configurations at least once - * by probing the log of the thread if the reload fails. - */ - private void waitForFailedReloadAtLeastOnce(int reloadInterval) - throws InterruptedException, TimeoutException { - GenericTestUtils.waitFor(new Supplier() { - @Override - public Boolean get() { - return reloaderLog.getOutput().contains( - FileMonitoringTimerTask.PROCESS_ERROR_MESSAGE); - } - }, reloadInterval, 10 * 1000); + waitForFailedReloadAtLeastOnce((int) reloadInterval); + + assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); + } finally { + reloaderLog.stopCapturing(); + fileMonitoringTimer.cancel(); } + } + + @Test + @Timeout(value = 30) + public void testReloadCorruptTrustStore() throws Exception { + KeyPair kp = generateKeyPair("RSA"); + X509Certificate cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA"); + String keystoreLocation = BASEDIR + "/testmissing.jks"; + createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), cert1); + + long reloadInterval = 10; + Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); + ReloadingX509KeystoreManager tm = + new ReloadingX509KeystoreManager("jks", keystoreLocation, + "password", + "password"); + + try { + fileMonitoringTimer.schedule(new FileMonitoringTimerTask( + Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); + assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); + + // Wait so that the file modification time is different + Thread.sleep((reloadInterval + 1000)); + + assertFalse(reloaderLog.getOutput().contains( + FileMonitoringTimerTask.PROCESS_ERROR_MESSAGE)); + OutputStream os = new FileOutputStream(keystoreLocation); + os.write(1); + os.close(); + + waitForFailedReloadAtLeastOnce((int) reloadInterval); + + assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); + } finally { + reloaderLog.stopCapturing(); + fileMonitoringTimer.cancel(); + } + } + + /**Wait for the reloader thread to load the configurations at least once + * by probing the log of the thread if the reload fails. + */ + private void waitForFailedReloadAtLeastOnce(int reloadInterval) + throws InterruptedException, TimeoutException { + GenericTestUtils.waitFor(new Supplier() { + @Override + public Boolean get() { + return reloaderLog.getOutput().contains( + FileMonitoringTimerTask.PROCESS_ERROR_MESSAGE); + } + }, reloadInterval, 10 * 1000); + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java index eb68d226bb47e..7ada6f1376731 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java @@ -129,39 +129,39 @@ public void testPrint() throws Exception { args = new String[] {"print", tokenFilename}; rc = dt.run(args); assertEquals(0, rc, "test simple print exit code"); - assertTrue( - outContent.toString().contains(KIND.toString()), "test simple print output kind:\n" + outContent.toString()); - assertTrue( - outContent.toString().contains(SERVICE.toString()), "test simple print output service:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(KIND.toString()), + "test simple print output kind:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(SERVICE.toString()), + "test simple print output service:\n" + outContent.toString()); outContent.reset(); args = new String[] {"print", tokenLegacyFile.toString()}; rc = dt.run(args); assertEquals(0, rc, "test legacy print exit code"); - assertTrue( - outContent.toString().contains(KIND.toString()), "test simple print output kind:\n" + outContent.toString()); - assertTrue( - outContent.toString().contains(SERVICE.toString()), "test simple print output service:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(KIND.toString()), + "test simple print output kind:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(SERVICE.toString()), + "test simple print output service:\n" + outContent.toString()); outContent.reset(); args = new String[] { "print", "-alias", SERVICE.toString(), tokenFilename}; rc = dt.run(args); assertEquals(0, rc, "test alias print exit code"); - assertTrue( - outContent.toString().contains(KIND.toString()), "test simple print output kind:\n" + outContent.toString()); - assertTrue( - outContent.toString().contains(SERVICE.toString()), "test simple print output service:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(KIND.toString()), + "test simple print output kind:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(SERVICE.toString()), + "test simple print output service:\n" + outContent.toString()); outContent.reset(); args = new String[] { "print", "-alias", "not-a-serivce", tokenFilename}; rc = dt.run(args); assertEquals(0, rc, "test no alias print exit code"); - assertFalse( - outContent.toString().contains(KIND.toString()), "test no alias print output kind:\n" + outContent.toString()); - assertFalse( - outContent.toString().contains(SERVICE.toString()), "test no alias print output service:\n" + outContent.toString()); + assertFalse(outContent.toString().contains(KIND.toString()), + "test no alias print output kind:\n" + outContent.toString()); + assertFalse(outContent.toString().contains(SERVICE.toString()), + "test no alias print output service:\n" + outContent.toString()); } @Test @@ -175,17 +175,17 @@ public void testEdit() throws Exception { args = new String[] {"print", "-alias", oldService, tokenFilename2}; rc = dt.run(args); assertEquals(0, rc, "test simple edit print old exit code"); - assertTrue( - outContent.toString().contains(KIND.toString()), "test simple edit output kind old:\n" + outContent.toString()); - assertTrue( - outContent.toString().contains(oldService), "test simple edit output service old:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(KIND.toString()), + "test simple edit output kind old:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(oldService), + "test simple edit output service old:\n" + outContent.toString()); args = new String[] {"print", "-alias", newAlias, tokenFilename2}; rc = dt.run(args); assertEquals(0, rc, "test simple edit print new exit code"); - assertTrue( - outContent.toString().contains(KIND.toString()), "test simple edit output kind new:\n" + outContent.toString()); - assertTrue( - outContent.toString().contains(newAlias), "test simple edit output service new:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(KIND.toString()), + "test simple edit output kind new:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(newAlias), + "test simple edit output service new:\n" + outContent.toString()); } @Test @@ -196,12 +196,12 @@ public void testAppend() throws Exception { args = new String[] {"print", tokenFilename2}; rc = dt.run(args); assertEquals(0, rc, "test simple append print exit code"); - assertTrue( - outContent.toString().contains(KIND.toString()), "test simple append output kind:\n" + outContent.toString()); - assertTrue( - outContent.toString().contains(SERVICE.toString()), "test simple append output service:\n" + outContent.toString()); - assertTrue( - outContent.toString().contains(SERVICE2.toString()), "test simple append output service:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(KIND.toString()), + "test simple append output kind:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(SERVICE.toString()), + "test simple append output service:\n" + outContent.toString()); + assertTrue(outContent.toString().contains(SERVICE2.toString()), + "test simple append output service:\n" + outContent.toString()); } @Test @@ -212,10 +212,10 @@ public void testRemove() throws Exception { args = new String[] {"print", tokenFilename}; rc = dt.run(args); assertEquals(0, rc, "test simple remove print exit code"); - assertFalse( - outContent.toString().contains(KIND.toString()), "test simple remove output kind:\n" + outContent.toString()); - assertFalse( - outContent.toString().contains(SERVICE.toString()), "test simple remove output service:\n" + outContent.toString()); + assertFalse(outContent.toString().contains(KIND.toString()), + "test simple remove output kind:\n" + outContent.toString()); + assertFalse(outContent.toString().contains(SERVICE.toString()), + "test simple remove output service:\n" + outContent.toString()); } @Test @@ -227,10 +227,10 @@ public void testGet() throws Exception { rc = dt.run(args); String oc = outContent.toString(); assertEquals(0, rc, "test print after get exit code"); - assertTrue( - oc.contains(KIND_GET.toString()), "test print after get output kind:\n" + oc); - assertTrue( - oc.contains(SERVICE_GET.toString()), "test print after get output service:\n" + oc); + assertTrue(oc.contains(KIND_GET.toString()), + "test print after get output kind:\n" + oc); + assertTrue(oc.contains(SERVICE_GET.toString()), + "test print after get output service:\n" + oc); } @Test @@ -243,10 +243,10 @@ public void testGetWithServiceFlag() throws Exception { rc = dt.run(args); String oc = outContent.toString(); assertEquals(0, rc, "test print after get with service flag exit code"); - assertTrue( - oc.contains(KIND_GET.toString()), "test print after get with service flag output kind:\n" + oc); - assertTrue( - oc.contains(SERVICE_GET.toString()), "test print after get with service flag output service:\n" + oc); + assertTrue(oc.contains(KIND_GET.toString()), + "test print after get with service flag output kind:\n" + oc); + assertTrue(oc.contains(SERVICE_GET.toString()), + "test print after get with service flag output service:\n" + oc); } @Test @@ -258,12 +258,12 @@ public void testGetWithAliasFlag() throws Exception { rc = dt.run(args); String oc = outContent.toString(); assertEquals(0, rc, "test print after get with alias flag exit code"); - assertTrue( - oc.contains(KIND_GET.toString()), "test print after get with alias flag output kind:\n" + oc); - assertTrue( - oc.contains(alias), "test print after get with alias flag output alias:\n" + oc); - assertFalse(oc.contains(SERVICE_GET.toString()), "test print after get with alias flag output old service:\n" + - oc); + assertTrue(oc.contains(KIND_GET.toString()), + "test print after get with alias flag output kind:\n" + oc); + assertTrue(oc.contains(alias), + "test print after get with alias flag output alias:\n" + oc); + assertFalse(oc.contains(SERVICE_GET.toString()), + "test print after get with alias flag output old service:\n" + oc); } @Test @@ -303,12 +303,12 @@ public void testImport() throws Exception { args = new String[] {"print", tokenFilenameImport}; rc = dt.run(args); assertEquals(0, rc, "test simple import print old exit code"); - assertTrue( - outContent.toString().contains(KIND_IMPORT.toString()), "test print after import output:\n" + outContent); - assertTrue( - outContent.toString().contains(SERVICE_IMPORT.toString()), "test print after import output:\n" + outContent); - assertTrue( - outContent.toString().contains(base64), "test print after simple import output:\n" + outContent); + assertTrue(outContent.toString().contains(KIND_IMPORT.toString()), + "test print after import output:\n" + outContent); + assertTrue(outContent.toString().contains(SERVICE_IMPORT.toString()), + "test print after import output:\n" + outContent); + assertTrue(outContent.toString().contains(base64), + "test print after simple import output:\n" + outContent); } @Test @@ -322,9 +322,9 @@ public void testImportWithAliasFlag() throws Exception { args = new String[] {"print", tokenFilenameImport}; rc = dt.run(args); assertEquals(0, rc, "test simple import print old exit code"); - assertTrue( - outContent.toString().contains(KIND_IMPORT.toString()), "test print after import output:\n" + outContent); - assertTrue( - outContent.toString().contains(alias), "test print after import with alias output:\n" + outContent); + assertTrue(outContent.toString().contains(KIND_IMPORT.toString()), + "test print after import output:\n" + outContent); + assertTrue(outContent.toString().contains(alias), + "test print after import with alias output:\n" + outContent); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenManager.java index 8836749b2e6a5..ad69f5d1d5c42 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenManager.java @@ -26,29 +26,28 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; -@RunWith(Parameterized.class) public class TestDelegationTokenManager { private static final long DAY_IN_SECS = 86400; - @Parameterized.Parameters public static Collection headers() { - return Arrays.asList(new Object[][] { { false }, { true } }); + return Arrays.asList(new Object[][]{{false}, {true}}); } private boolean enableZKKey; - public TestDelegationTokenManager(boolean enableZKKey) { - this.enableZKKey = enableZKKey; + public void initTestDelegationTokenManager(boolean pEnableZKKey) { + this.enableZKKey = pEnableZKKey; } @SuppressWarnings("unchecked") - @Test - public void testDTManager() throws Exception { + @ParameterizedTest + @MethodSource("headers") + public void testDTManager(boolean pEnableZKKey) throws Exception { + initTestDelegationTokenManager(pEnableZKKey); Configuration conf = new Configuration(false); conf.setLong(DelegationTokenManager.UPDATE_INTERVAL, DAY_IN_SECS); conf.setLong(DelegationTokenManager.MAX_LIFETIME, DAY_IN_SECS); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java index f838cbce75b03..b6f60c4e5b3bc 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java @@ -240,45 +240,45 @@ public void testRawHttpCalls() throws Exception { // unauthenticated access to URL HttpURLConnection conn = (HttpURLConnection) nonAuthURL.openConnection(); - assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // authenticated access to URL conn = (HttpURLConnection) authURL.openConnection(); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // unauthenticated access to get delegation token URL url = new URL(nonAuthURL.toExternalForm() + "?op=GETDELEGATIONTOKEN"); conn = (HttpURLConnection) url.openConnection(); - assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // authenticated access to get delegation token url = new URL(authURL.toExternalForm() + "&op=GETDELEGATIONTOKEN&renewer=foo"); conn = (HttpURLConnection) url.openConnection(); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ObjectMapper mapper = new ObjectMapper(); Map map = mapper.readValue(conn.getInputStream(), Map.class); String dt = (String) ((Map) map.get("Token")).get("urlString"); - assertNotNull(dt); + assertNotNull(dt); // delegation token access to URL url = new URL(nonAuthURL.toExternalForm() + "?delegation=" + dt); conn = (HttpURLConnection) url.openConnection(); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // delegation token and authenticated access to URL url = new URL(authURL.toExternalForm() + "&delegation=" + dt); conn = (HttpURLConnection) url.openConnection(); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // renewew delegation token, unauthenticated access to URL url = new URL(nonAuthURL.toExternalForm() + "?op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); // renewew delegation token, authenticated access to URL @@ -286,14 +286,14 @@ public void testRawHttpCalls() throws Exception { "&op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // renewew delegation token, authenticated access to URL, not renewer url = new URL(getJettyURL() + "/foo/bar?authenticated=bar&op=RENEWDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - assertEquals(HttpURLConnection.HTTP_FORBIDDEN, + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); // cancel delegation token, nonauthenticated access to URL @@ -301,32 +301,32 @@ public void testRawHttpCalls() throws Exception { "?op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); // cancel canceled delegation token, nonauthenticated access to URL url = new URL(nonAuthURL.toExternalForm() + "?op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - assertEquals(HttpURLConnection.HTTP_NOT_FOUND, + assertEquals(HttpURLConnection.HTTP_NOT_FOUND, conn.getResponseCode()); // get new delegation token url = new URL(authURL.toExternalForm() + "&op=GETDELEGATIONTOKEN&renewer=foo"); conn = (HttpURLConnection) url.openConnection(); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); mapper = new ObjectMapper(); map = mapper.readValue(conn.getInputStream(), Map.class); dt = (String) ((Map) map.get("Token")).get("urlString"); - assertNotNull(dt); + assertNotNull(dt); // cancel delegation token, authenticated access to URL url = new URL(authURL.toExternalForm() + "&op=CANCELDELEGATIONTOKEN&token=" + dt); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); } finally { jetty.stop(); } @@ -369,32 +369,32 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) try { aUrl.getDelegationToken(nonAuthURL, token, FOO_USER); - fail(); + fail(); } catch (Exception ex) { - assertTrue(ex.getCause().getMessage().contains("401")); + assertTrue(ex.getCause().getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, FOO_USER); - assertNotNull(token.getDelegationToken()); - assertEquals(new Text("token-kind"), + assertNotNull(token.getDelegationToken()); + assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); aUrl.renewDelegationToken(authURL, token); try { aUrl.renewDelegationToken(nonAuthURL, token); - fail(); + fail(); } catch (Exception ex) { - assertTrue(ex.getMessage().contains("401")); + assertTrue(ex.getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, FOO_USER); try { aUrl.renewDelegationToken(authURL2, token); - fail(); + fail(); } catch (Exception ex) { - assertTrue(ex.getMessage().contains("403")); + assertTrue(ex.getMessage().contains("403")); } aUrl.getDelegationToken(authURL, token, FOO_USER); @@ -410,7 +410,7 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) try { aUrl.renewDelegationToken(nonAuthURL, token); } catch (Exception ex) { - assertTrue(ex.getMessage().contains("401")); + assertTrue(ex.getMessage().contains("401")); } aUrl.getDelegationToken(authURL, token, "foo"); @@ -421,13 +421,13 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) @Override public Void run() throws Exception { HttpURLConnection conn = aUrl.openConnection(nonAuthURL, new DelegationTokenAuthenticatedURL.Token()); - assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); + assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); if (useQS) { - assertNull(conn.getHeaderField("UsingHeader")); - assertNotNull(conn.getHeaderField("UsingQueryString")); + assertNull(conn.getHeaderField("UsingHeader")); + assertNotNull(conn.getHeaderField("UsingQueryString")); } else { - assertNotNull(conn.getHeaderField("UsingHeader")); - assertNull(conn.getHeaderField("UsingQueryString")); + assertNotNull(conn.getHeaderField("UsingHeader")); + assertNull(conn.getHeaderField("UsingQueryString")); } return null; } @@ -477,8 +477,8 @@ public void testExternalDelegationTokenSecretManager() throws Exception { new DelegationTokenAuthenticatedURL(); aUrl.getDelegationToken(authURL, token, FOO_USER); - assertNotNull(token.getDelegationToken()); - assertEquals(new Text("fooKind"), + assertNotNull(token.getDelegationToken()); + assertEquals(new Text("fooKind"), token.getDelegationToken().getKind()); } finally { @@ -628,15 +628,15 @@ public Void run() throws Exception { DelegationTokenAuthenticatedURL aUrl = new DelegationTokenAuthenticatedURL(); HttpURLConnection conn = aUrl.openConnection(url, token); - assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals(FOO_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(FOO_USER, ret.get(0)); aUrl.getDelegationToken(url, token, FOO_USER); - assertNotNull(token.getDelegationToken()); - assertEquals(new Text("token-kind"), + assertNotNull(token.getDelegationToken()); + assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); return null; } @@ -751,7 +751,7 @@ private void testKerberosDelegationTokenAuthenticator( final String doAsUser = doAs ? OK_USER : null; File testDir = new File("target/" + UUID.randomUUID().toString()); - assertTrue(testDir.mkdirs()); + assertTrue(testDir.mkdirs()); MiniKdc kdc = new MiniKdc(MiniKdc.createConf(), testDir); final Server jetty = createJettyServer(); ServletContextHandler context = new ServletContextHandler(); @@ -781,7 +781,7 @@ private void testKerberosDelegationTokenAuthenticator( aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); fail(); } catch (AuthenticationException ex) { - assertTrue(ex.getCause().getMessage().contains("GSSException")); + assertTrue(ex.getCause().getMessage().contains("GSSException")); } doAsKerberosUser("client", keytabFile.getAbsolutePath(), @@ -790,8 +790,8 @@ private void testKerberosDelegationTokenAuthenticator( public Void call() throws Exception { aUrl.getDelegationToken( url, token, doAs ? doAsUser : "client", doAsUser); - assertNotNull(token.getDelegationToken()); - assertEquals(new Text("token-kind"), + assertNotNull(token.getDelegationToken()); + assertEquals(new Text("token-kind"), token.getDelegationToken().getKind()); // Make sure the token belongs to the right owner ByteArrayInputStream buf = new ByteArrayInputStream( @@ -801,29 +801,28 @@ public Void call() throws Exception { new DelegationTokenIdentifier(new Text("token-kind")); id.readFields(dis); dis.close(); - assertEquals( - doAs ? new Text(OK_USER) : new Text("client"), id.getOwner()); + assertEquals(doAs ? new Text(OK_USER) : new Text("client"), id.getOwner()); if (doAs) { - assertEquals(new Text("client"), id.getRealUser()); + assertEquals(new Text("client"), id.getRealUser()); } aUrl.renewDelegationToken(url, token, doAsUser); - assertNotNull(token.getDelegationToken()); + assertNotNull(token.getDelegationToken()); aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); - assertNotNull(token.getDelegationToken()); + assertNotNull(token.getDelegationToken()); try { aUrl.renewDelegationToken(url, token, doAsUser); - fail(); + fail(); } catch (Exception ex) { - assertTrue(ex.getMessage().contains("403")); + assertTrue(ex.getMessage().contains("403")); } aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); aUrl.cancelDelegationToken(url, token, doAsUser); - assertNull(token.getDelegationToken()); + assertNull(token.getDelegationToken()); return null; } @@ -853,17 +852,17 @@ public void testProxyUser() throws Exception { url.toExternalForm(), FOO_USER, OK_USER); HttpURLConnection conn = (HttpURLConnection) new URL(strUrl).openConnection(); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals(OK_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(OK_USER, ret.get(0)); strUrl = String.format("%s?user.name=%s&DOAS=%s", url.toExternalForm(), FOO_USER, OK_USER); conn = (HttpURLConnection) new URL(strUrl).openConnection(); - assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals(OK_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(OK_USER, ret.get(0)); UserGroupInformation ugi = UserGroupInformation.createRemoteUser(FOO_USER); ugi.doAs(new PrivilegedExceptionAction() { @@ -876,16 +875,16 @@ public Void run() throws Exception { // proxyuser using authentication handler authentication HttpURLConnection conn = aUrl.openConnection(url, token, OK_USER); - assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals(OK_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(OK_USER, ret.get(0)); // unauthorized proxy user using authentication handler authentication conn = aUrl.openConnection(url, token, FAIL_USER); - assertEquals(HttpURLConnection.HTTP_FORBIDDEN, + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); // proxy using delegation token authentication @@ -897,12 +896,12 @@ public Void run() throws Exception { // requests using delegation token as auth do not honor doAs conn = aUrl.openConnection(url, token, OK_USER); - assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals(FOO_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(FOO_USER, ret.get(0)); return null; } @@ -959,22 +958,22 @@ public Void run() throws Exception { // user foo HttpURLConnection conn = aUrl.openConnection(url, token); - assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils .readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals("remoteuser=" + FOO_USER+ ":ugi=" + FOO_USER, + assertEquals(1, ret.size()); + assertEquals("remoteuser=" + FOO_USER+ ":ugi=" + FOO_USER, ret.get(0)); // user ok-user via proxyuser foo conn = aUrl.openConnection(url, token, OK_USER); - assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + - ":ugi=" + OK_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals("realugi=" + FOO_USER + ":remoteuser=" + OK_USER + + ":ugi=" + OK_USER, ret.get(0)); return null; } @@ -1021,11 +1020,11 @@ public Void run() throws Exception { // user ok-user via proxyuser foo HttpURLConnection conn = aUrl.openConnection(url, token, OK_USER); - assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + + assertEquals(1, ret.size()); + assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + ":ugi=" + OK_USER, ret.get(0)); return null; From 7612bf98fb472cb0b68f284a592a88cdf083e2d6 Mon Sep 17 00:00:00 2001 From: fanshilun Date: Fri, 7 Feb 2025 18:01:46 +0800 Subject: [PATCH 6/7] HADOOP-19415. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-common Part4. --- .../ssl/TestReloadingX509KeyManager.java | 78 ++++++++++--------- .../security/token/TestDtUtilShell.java | 2 +- .../web/TestWebDelegationToken.java | 40 +++++----- 3 files changed, 61 insertions(+), 59 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java index b3a74b745e0de..5edbc4e7aa3c9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java @@ -42,10 +42,10 @@ public class TestReloadingX509KeyManager { private static final String BASEDIR = GenericTestUtils.getTempPath( - TestReloadingX509TrustManager.class.getSimpleName()); + TestReloadingX509TrustManager.class.getSimpleName()); private final GenericTestUtils.LogCapturer reloaderLog = GenericTestUtils.LogCapturer.captureLogs( - FileMonitoringTimerTask.LOG); + FileMonitoringTimerTask.LOG); @BeforeAll public static void setUp() throws Exception { @@ -75,8 +75,8 @@ public void testLoadCorruptKeyStore() throws Exception { os.close(); ReloadingX509KeystoreManager tm = - new ReloadingX509KeystoreManager("jks", keystoreLocation, - "password", "password"); + new ReloadingX509KeystoreManager("jks", keystoreLocation, + "password", "password"); }); } @@ -90,32 +90,33 @@ public void testReload() throws Exception { createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), sCert); long reloadInterval = 10; - Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); + Timer fileMonitoringTimer = + new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); ReloadingX509KeystoreManager tm = new ReloadingX509KeystoreManager("jks", keystoreLocation, "password", "password"); try { - fileMonitoringTimer.schedule(new FileMonitoringTimerTask( - Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); - assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); - - // Wait so that the file modification time is different - Thread.sleep((reloadInterval+ 1000)); - - // Change the certificate with a new keypair - final KeyPair anotherKP = generateKeyPair("RSA"); - sCert = KeyStoreTestUtil.generateCertificate("CN=localhost, O=server", anotherKP, 30, - "SHA1withRSA"); - createKeyStore(keystoreLocation, "password", "cert1", anotherKP.getPrivate(), sCert); - - GenericTestUtils.waitFor(new Supplier() { - @Override - public Boolean get() { + fileMonitoringTimer.schedule(new FileMonitoringTimerTask( + Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); + assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); + + // Wait so that the file modification time is different + Thread.sleep((reloadInterval+ 1000)); + + // Change the certificate with a new keypair + final KeyPair anotherKP = generateKeyPair("RSA"); + sCert = KeyStoreTestUtil.generateCertificate("CN=localhost, O=server", anotherKP, 30, + "SHA1withRSA"); + createKeyStore(keystoreLocation, "password", "cert1", anotherKP.getPrivate(), sCert); + + GenericTestUtils.waitFor(new Supplier() { + @Override + public Boolean get() { return tm.getPrivateKey("cert1").equals(kp.getPrivate()); } - }, (int) reloadInterval, 100000); + }, (int) reloadInterval, 100000); } finally { - fileMonitoringTimer.cancel(); + fileMonitoringTimer.cancel(); } } @@ -128,34 +129,35 @@ public void testReloadMissingTrustStore() throws Exception { createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), cert1); long reloadInterval = 10; - Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); + Timer fileMonitoringTimer = + new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); ReloadingX509KeystoreManager tm = new ReloadingX509KeystoreManager("jks", keystoreLocation, "password", "password"); try { - fileMonitoringTimer.schedule(new FileMonitoringTimerTask( - Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); - assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); + fileMonitoringTimer.schedule(new FileMonitoringTimerTask( + Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); + assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); - assertFalse(reloaderLog.getOutput().contains( - FileMonitoringTimerTask.PROCESS_ERROR_MESSAGE)); + assertFalse(reloaderLog.getOutput().contains( + FileMonitoringTimerTask.PROCESS_ERROR_MESSAGE)); - // Wait for the first reload to happen so we actually detect a change after the delete - Thread.sleep((reloadInterval+ 1000)); + // Wait for the first reload to happen so we actually detect a change after the delete + Thread.sleep((reloadInterval+ 1000)); - new File(keystoreLocation).delete(); + new File(keystoreLocation).delete(); - // Wait for the reload to happen and log to get written to - Thread.sleep((reloadInterval+ 1000)); + // Wait for the reload to happen and log to get written to + Thread.sleep((reloadInterval+ 1000)); - waitForFailedReloadAtLeastOnce((int) reloadInterval); + waitForFailedReloadAtLeastOnce((int) reloadInterval); - assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); + assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); } finally { - reloaderLog.stopCapturing(); - fileMonitoringTimer.cancel(); + reloaderLog.stopCapturing(); + fileMonitoringTimer.cancel(); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java index 7ada6f1376731..7162d7e2cb45a 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java @@ -325,6 +325,6 @@ public void testImportWithAliasFlag() throws Exception { assertTrue(outContent.toString().contains(KIND_IMPORT.toString()), "test print after import output:\n" + outContent); assertTrue(outContent.toString().contains(alias), - "test print after import with alias output:\n" + outContent); + "test print after import with alias output:\n" + outContent); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java index b6f60c4e5b3bc..1bb010afe6c7d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java @@ -418,20 +418,20 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); ugi.addToken(token.getDelegationToken()); ugi.doAs(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - HttpURLConnection conn = aUrl.openConnection(nonAuthURL, new DelegationTokenAuthenticatedURL.Token()); - assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); - if (useQS) { - assertNull(conn.getHeaderField("UsingHeader")); - assertNotNull(conn.getHeaderField("UsingQueryString")); - } else { - assertNotNull(conn.getHeaderField("UsingHeader")); - assertNull(conn.getHeaderField("UsingQueryString")); - } - return null; - } - }); + @Override + public Void run() throws Exception { + HttpURLConnection conn = aUrl.openConnection(nonAuthURL, new DelegationTokenAuthenticatedURL.Token()); + assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); + if (useQS) { + assertNull(conn.getHeaderField("UsingHeader")); + assertNotNull(conn.getHeaderField("UsingQueryString")); + } else { + assertNotNull(conn.getHeaderField("UsingHeader")); + assertNull(conn.getHeaderField("UsingQueryString")); + } + return null; + } + }); } finally { @@ -558,17 +558,17 @@ public Void run() throws Exception { DelegationTokenAuthenticatedURL aUrl = new DelegationTokenAuthenticatedURL(); HttpURLConnection conn = aUrl.openConnection(url, token); - assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); List ret = IOUtils.readLines(conn.getInputStream(), StandardCharsets.UTF_8); - assertEquals(1, ret.size()); - assertEquals(FOO_USER, ret.get(0)); + assertEquals(1, ret.size()); + assertEquals(FOO_USER, ret.get(0)); try { aUrl.getDelegationToken(url, token, FOO_USER); - fail(); + fail(); } catch (AuthenticationException ex) { - assertTrue(ex.getMessage().contains( + assertTrue(ex.getMessage().contains( "delegation token operation")); } return null; @@ -779,7 +779,7 @@ private void testKerberosDelegationTokenAuthenticator( try { aUrl.getDelegationToken(url, token, FOO_USER, doAsUser); - fail(); + fail(); } catch (AuthenticationException ex) { assertTrue(ex.getCause().getMessage().contains("GSSException")); } From 2107f188826396862add96695f764bcafe7ce8a2 Mon Sep 17 00:00:00 2001 From: fanshilun Date: Sat, 8 Feb 2025 08:04:10 +0800 Subject: [PATCH 7/7] HADOOP-19415. Fix CheckStyle. --- .../security/ssl/TestReloadingX509KeyManager.java | 13 +++++++------ .../delegation/web/TestWebDelegationToken.java | 3 ++- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java index 5edbc4e7aa3c9..20e327991ca09 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509KeyManager.java @@ -97,7 +97,7 @@ public void testReload() throws Exception { "password", "password"); try { fileMonitoringTimer.schedule(new FileMonitoringTimerTask( - Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); + Paths.get(keystoreLocation), tm::loadFrom, null), reloadInterval, reloadInterval); assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); // Wait so that the file modification time is different @@ -112,8 +112,8 @@ public void testReload() throws Exception { GenericTestUtils.waitFor(new Supplier() { @Override public Boolean get() { - return tm.getPrivateKey("cert1").equals(kp.getPrivate()); - } + return tm.getPrivateKey("cert1").equals(kp.getPrivate()); + } }, (int) reloadInterval, 100000); } finally { fileMonitoringTimer.cancel(); @@ -138,7 +138,7 @@ public void testReloadMissingTrustStore() throws Exception { try { fileMonitoringTimer.schedule(new FileMonitoringTimerTask( - Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); + Paths.get(keystoreLocation), tm::loadFrom, null), reloadInterval, reloadInterval); assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); assertFalse(reloaderLog.getOutput().contains( @@ -170,7 +170,8 @@ public void testReloadCorruptTrustStore() throws Exception { createKeyStore(keystoreLocation, "password", "cert1", kp.getPrivate(), cert1); long reloadInterval = 10; - Timer fileMonitoringTimer = new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); + Timer fileMonitoringTimer = + new Timer(FileBasedKeyStoresFactory.SSL_MONITORING_THREAD_NAME, true); ReloadingX509KeystoreManager tm = new ReloadingX509KeystoreManager("jks", keystoreLocation, "password", @@ -178,7 +179,7 @@ public void testReloadCorruptTrustStore() throws Exception { try { fileMonitoringTimer.schedule(new FileMonitoringTimerTask( - Paths.get(keystoreLocation), tm::loadFrom,null), reloadInterval, reloadInterval); + Paths.get(keystoreLocation), tm::loadFrom, null), reloadInterval, reloadInterval); assertEquals(kp.getPrivate(), tm.getPrivateKey("cert1")); // Wait so that the file modification time is different diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java index 1bb010afe6c7d..25756dd427794 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java @@ -420,7 +420,8 @@ private void testDelegationTokenAuthenticatorCalls(final boolean useQS) ugi.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HttpURLConnection conn = aUrl.openConnection(nonAuthURL, new DelegationTokenAuthenticatedURL.Token()); + HttpURLConnection conn = + aUrl.openConnection(nonAuthURL, new DelegationTokenAuthenticatedURL.Token()); assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); if (useQS) { assertNull(conn.getHeaderField("UsingHeader"));