ソースを参照

HADOOP-19415. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-common Part10. (#7670)

* HADOOP-19415. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-common Part10.

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Co-authored-by: He Xiaoqiao <hexiaoqiao@apache.org>
Co-authored-by: Hualong Zhang <hualong.z@hotmail.com>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: He Xiaoqiao <hexiaoqiao@apache.org>
Reviewed-by: Hualong Zhang <hualong.z@hotmail.com>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 3 週間 前
コミット
fd380ef565
59 ファイル変更522 行追加518 行削除
  1. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java
  2. 24 24
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestFlagSet.java
  3. 5 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestLeakReporter.java
  4. 6 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockCache.java
  5. 4 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockData.java
  6. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockOperations.java
  7. 6 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBoundedResourcePool.java
  8. 6 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java
  9. 5 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java
  10. 8 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestExecutorServiceFuturePool.java
  11. 4 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestFilePosition.java
  12. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestRetryer.java
  13. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestValidate.java
  14. 15 13
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestFsShellConcat.java
  15. 5 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java
  16. 4 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java
  17. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java
  18. 38 28
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java
  19. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java
  20. 12 12
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java
  21. 23 23
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java
  22. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/store/TestFSBuilderSupport.java
  23. 31 31
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/wrappedio/impl/TestWrappedStatistics.java
  24. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestShadedProtobufHelper.java
  25. 14 18
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/AbstractHadoopTestBase.java
  26. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEnclosingRoot.java
  27. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestXMLUtils.java
  28. 55 55
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
  29. 16 16
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestBucketConfiguration.java
  30. 2 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInstrumentationLifecycle.java
  31. 31 25
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java
  32. 4 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java
  33. 4 5
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AccessGrantConfiguration.java
  34. 16 18
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSSEConfiguration.java
  35. 3 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java
  36. 4 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java
  37. 2 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java
  38. 3 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java
  39. 5 6
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java
  40. 3 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java
  41. 16 16
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/impl/TestActiveAuditManagerThreadLeakage.java
  42. 4 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestIAMInstanceCredentialsProvider.java
  43. 30 30
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java
  44. 12 12
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestAwsClientConfig.java
  45. 18 18
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestClientManager.java
  46. 3 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestErrorTranslation.java
  47. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
  48. 3 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestS3ExpressStorage.java
  49. 11 11
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/logging/TestLogControllerFactory.java
  50. 9 10
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java
  51. 2 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java
  52. 9 9
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java
  53. 2 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java
  54. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java
  55. 2 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java
  56. 5 5
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestMetastoreChecking.java
  57. 13 15
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/TestErrorCodeMapping.java
  58. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/sdk/TestAWSV2SDK.java
  59. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsErrorTranslation.java

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java

@@ -25,7 +25,7 @@ import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
 import org.assertj.core.api.AbstractStringAssert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 24 - 24
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestFlagSet.java

@@ -20,8 +20,7 @@ package org.apache.hadoop.fs.impl;
 
 import java.util.EnumSet;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
@@ -31,6 +30,7 @@ import static java.util.EnumSet.noneOf;
 import static org.apache.hadoop.fs.impl.FlagSet.buildFlagSet;
 import static org.apache.hadoop.fs.impl.FlagSet.createFlagSet;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Unit tests for {@link FlagSet} class.
@@ -68,7 +68,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
    */
   @Test
   public void testEntryEnableDisable() {
-    Assertions.assertThat(flagSet.flags()).isEmpty();
+    assertThat(flagSet.flags()).isEmpty();
     assertDisabled(SimpleEnum.a);
     flagSet.enable(SimpleEnum.a);
     assertEnabled(SimpleEnum.a);
@@ -81,7 +81,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
    */
   @Test
   public void testSetMethod() {
-    Assertions.assertThat(flagSet.flags()).isEmpty();
+    assertThat(flagSet.flags()).isEmpty();
     flagSet.set(SimpleEnum.a, true);
     assertEnabled(SimpleEnum.a);
     flagSet.set(SimpleEnum.a, false);
@@ -136,7 +136,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
    * @param expected expected value
    */
   private void assertStringValue(final String expected) {
-    Assertions.assertThat(flagSet.toString())
+    assertThat(flagSet.toString())
         .isEqualTo(expected);
   }
 
@@ -144,7 +144,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
    * Assert the configuration string form matches that expected.
    */
   public void assertConfigurationStringMatches(final String expected) {
-    Assertions.assertThat(flagSet.toConfigurationString())
+    assertThat(flagSet.toConfigurationString())
         .describedAs("Configuration string of %s", flagSet)
         .isEqualTo(expected);
   }
@@ -224,7 +224,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
    * @param capability capability to probe for
    */
   private void assertHasCapability(final String capability) {
-    Assertions.assertThat(flagSet.hasCapability(capability))
+    assertThat(flagSet.hasCapability(capability))
         .describedAs("Capability of %s on %s", capability, flagSet)
         .isTrue();
   }
@@ -234,7 +234,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
    * @param capability capability to probe for
    */
   private void assertLacksCapability(final String capability) {
-    Assertions.assertThat(flagSet.hasCapability(capability))
+    assertThat(flagSet.hasCapability(capability))
         .describedAs("Capability of %s on %s", capability, flagSet)
         .isFalse();
   }
@@ -248,7 +248,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
     assertFlags(SimpleEnum.a, SimpleEnum.b, SimpleEnum.c);
     assertHasCapability(CAPABILITY_A);
     assertHasCapability(CAPABILITY_B);
-    Assertions.assertThat(flagSet.pathCapabilities())
+    assertThat(flagSet.pathCapabilities())
         .describedAs("path capabilities of %s", flagSet)
         .containsExactlyInAnyOrder(CAPABILITY_A, CAPABILITY_B, CAPABILITY_C);
   }
@@ -259,7 +259,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
         KEYDOT,
         allOf(SimpleEnum.class));
     final FlagSet<SimpleEnum> s2 = roundTrip(s1);
-    Assertions.assertThat(s1.flags()).isEqualTo(s2.flags());
+    assertThat(s1.flags()).isEqualTo(s2.flags());
     assertFlagSetMatches(s2, SimpleEnum.a, SimpleEnum.b, SimpleEnum.c);
   }
 
@@ -268,13 +268,13 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
     final FlagSet<SimpleEnum> s1 = createFlagSet(SimpleEnum.class, KEYDOT,
         noneOf(SimpleEnum.class));
     final FlagSet<SimpleEnum> s2 = roundTrip(s1);
-    Assertions.assertThat(s1.flags())
+    assertThat(s1.flags())
         .isEqualTo(s2.flags());
-    Assertions.assertThat(s2.isEmpty())
+    assertThat(s2.isEmpty())
         .describedAs("empty flagset %s", s2)
         .isTrue();
     assertFlagSetMatches(flagSet);
-    Assertions.assertThat(flagSet.pathCapabilities())
+    assertThat(flagSet.pathCapabilities())
         .describedAs("path capabilities of %s", flagSet)
         .isEmpty();
   }
@@ -298,10 +298,10 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
     final FlagSet<SimpleEnum> s2 = createFlagSet(SimpleEnum.class, KEYDOT, SimpleEnum.a);
     // make one of them immutable
     s2.makeImmutable();
-    Assertions.assertThat(s1)
+    assertThat(s1)
         .describedAs("s1 == s2")
         .isEqualTo(s2);
-    Assertions.assertThat(s1.hashCode())
+    assertThat(s1.hashCode())
         .describedAs("hashcode of s1 == hashcode of s2")
         .isEqualTo(s2.hashCode());
   }
@@ -312,7 +312,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
         createFlagSet(SimpleEnum.class, KEYDOT, noneOf(SimpleEnum.class));
     final FlagSet<SimpleEnum> s2 =
         createFlagSet(SimpleEnum.class, KEYDOT, SimpleEnum.a, SimpleEnum.b);
-    Assertions.assertThat(s1)
+    assertThat(s1)
         .describedAs("s1 == s2")
         .isNotEqualTo(s2);
   }
@@ -323,7 +323,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
         createFlagSet(SimpleEnum.class, KEYDOT, noneOf(SimpleEnum.class));
     final FlagSet<?> s2 =
         createFlagSet(OtherEnum.class, KEYDOT, OtherEnum.a);
-    Assertions.assertThat(s1)
+    assertThat(s1)
         .describedAs("s1 == s2")
         .isNotEqualTo(s2);
   }
@@ -338,13 +338,13 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
             createFlagSet(SimpleEnum.class, KEYDOT, SimpleEnum.a, SimpleEnum.b);
     s1.makeImmutable();
     FlagSet<SimpleEnum> s2 = s1.copy();
-    Assertions.assertThat(s2)
+    assertThat(s2)
         .describedAs("copy of %s", s1)
         .isNotSameAs(s1);
-    Assertions.assertThat(!s2.isImmutable())
+    assertThat(!s2.isImmutable())
         .describedAs("set %s is immutable", s2)
         .isTrue();
-    Assertions.assertThat(s1)
+    assertThat(s1)
         .describedAs("s1 == s2")
         .isEqualTo(s2);
   }
@@ -377,7 +377,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
    * @param flag flag to check
    */
   private void assertEnabled(final SimpleEnum flag) {
-    Assertions.assertThat(flagSet.enabled(flag))
+    assertThat(flagSet.enabled(flag))
         .describedAs("status of flag %s in %s", flag, flagSet)
         .isTrue();
   }
@@ -387,7 +387,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
    * @param flag flag to check
    */
   private void assertDisabled(final SimpleEnum flag) {
-    Assertions.assertThat(flagSet.enabled(flag))
+    assertThat(flagSet.enabled(flag))
         .describedAs("status of flag %s in %s", flag, flagSet)
         .isFalse();
   }
@@ -409,7 +409,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
   private void assertFlagSetMatches(
       FlagSet<SimpleEnum> fs,
       SimpleEnum... flags) {
-    Assertions.assertThat(fs.flags())
+    assertThat(fs.flags())
         .describedAs("path capabilities of %s", fs)
         .containsExactly(flags);
   }
@@ -424,7 +424,7 @@ public final class TestFlagSet extends AbstractHadoopTestBase {
   private void assertPathCapabilitiesMatch(
       FlagSet<SimpleEnum> fs,
       String... capabilities) {
-    Assertions.assertThat(fs.pathCapabilities())
+    assertThat(fs.pathCapabilities())
         .describedAs("path capabilities of %s", fs)
         .containsExactlyInAnyOrder(capabilities);
   }

+ 5 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestLeakReporter.java

@@ -20,8 +20,7 @@ package org.apache.hadoop.fs.impl;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -30,6 +29,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 
 import static org.apache.hadoop.fs.impl.LeakReporter.THREAD_FORMAT;
 import static org.apache.hadoop.test.GenericTestUtils.LogCapturer.captureLogs;
+import static org.assertj.core.api.Assertions.assertThat;
 
 public final class TestLeakReporter extends AbstractHadoopTestBase {
 
@@ -74,7 +74,7 @@ public final class TestLeakReporter extends AbstractHadoopTestBase {
         oldName,
         Thread.currentThread().getId());
     // log auditing
-    Assertions.assertThat(output)
+    assertThat(output)
         .describedAs("output from the logs")
         .contains("WARN")
         .contains(message)
@@ -140,7 +140,7 @@ public final class TestLeakReporter extends AbstractHadoopTestBase {
         this::raiseNPE);
     reporter.close();
 
-    Assertions.assertThat(reporter.isClosed())
+    assertThat(reporter.isClosed())
         .describedAs("reporter closed)")
         .isTrue();
   }
@@ -158,7 +158,7 @@ public final class TestLeakReporter extends AbstractHadoopTestBase {
    * @param ex expected.
    */
   private void assertCloseCount(final int ex) {
-    Assertions.assertThat(closeCount.get())
+    assertThat(closeCount.get())
         .describedAs("close count")
         .isEqualTo(ex);
   }

+ 6 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockCache.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.fs.impl.prefetch;
 
 import java.nio.ByteBuffer;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalDirAllocator;
@@ -29,11 +29,11 @@ import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_TMP_DIR;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
-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.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestBlockCache extends AbstractHadoopTestBase {
 

+ 4 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockData.java

@@ -19,14 +19,14 @@
 
 package org.apache.hadoop.fs.impl.prefetch;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-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;
 
 public class TestBlockData extends AbstractHadoopTestBase {
 

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockOperations.java

@@ -21,12 +21,12 @@ package org.apache.hadoop.fs.impl.prefetch;
 
 import java.lang.reflect.Method;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestBlockOperations extends AbstractHadoopTestBase {
 

+ 6 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBoundedResourcePool.java

@@ -24,16 +24,16 @@ import java.util.Collections;
 import java.util.IdentityHashMap;
 import java.util.Set;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+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 TestBoundedResourcePool extends AbstractHadoopTestBase {
 

+ 6 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java

@@ -26,16 +26,16 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertSame;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
 
 public class TestBufferData extends AbstractHadoopTestBase {
 

+ 5 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java

@@ -19,15 +19,15 @@
 
 package org.apache.hadoop.fs.impl.prefetch;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
+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.assertSame;
 
 public class TestBufferPool extends AbstractHadoopTestBase {
 

+ 8 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestExecutorServiceFuturePool.java

@@ -25,25 +25,25 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-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.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
 
   private ExecutorService executorService;
 
-  @Before
+  @BeforeEach
   public void setUp() {
     executorService = Executors.newFixedThreadPool(3);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (executorService != null) {
       executorService.shutdownNow();
@@ -58,7 +58,7 @@ public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
     Future<Void> future =
         futurePool.executeRunnable(() -> atomicBoolean.set(true));
     future.get(30, TimeUnit.SECONDS);
-    assertTrue("atomicBoolean set to true?", atomicBoolean.get());
+    assertTrue(atomicBoolean.get(), "atomicBoolean set to true?");
   }
 
   @Test
@@ -71,7 +71,7 @@ public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
       return null;
     });
     future.get(30, TimeUnit.SECONDS);
-    assertTrue("atomicBoolean set to true?", atomicBoolean.get());
+    assertTrue(atomicBoolean.get(), "atomicBoolean set to true?");
   }
 
   @Test

+ 4 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestFilePosition.java

@@ -21,15 +21,15 @@ package org.apache.hadoop.fs.impl.prefetch;
 
 import java.nio.ByteBuffer;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.assertj.core.api.Assertions.assertThat;
-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;
 
 public class TestFilePosition extends AbstractHadoopTestBase {
 

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestRetryer.java

@@ -19,13 +19,13 @@
 
 package org.apache.hadoop.fs.impl.prefetch;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 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;
 
 public class TestRetryer extends AbstractHadoopTestBase {
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestValidate.java

@@ -24,7 +24,7 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.Arrays;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 

+ 15 - 13
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestFsShellConcat.java

@@ -25,10 +25,8 @@ import java.io.PrintStream;
 import java.net.URI;
 import java.util.Random;
 
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -40,8 +38,12 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
-import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
 
 /**
  * Test Concat.
@@ -54,7 +56,7 @@ public class TestFsShellConcat extends AbstractHadoopTestBase {
   private static Path testRootDir;
   private static Path dstPath;
 
-  @Before
+  @BeforeEach
   public void before() throws IOException {
     conf = new Configuration();
     shell = new FsShell(conf);
@@ -90,8 +92,8 @@ public class TestFsShellConcat extends AbstractHadoopTestBase {
     byte[] expectContent = out.toByteArray();
 
     // Do concat.
-    FileSystem mockFs = Mockito.mock(FileSystem.class);
-    Mockito.doAnswer(invocation -> {
+    FileSystem mockFs = mock(FileSystem.class);
+    doAnswer(invocation -> {
       Object[] args = invocation.getArguments();
       Path target = (Path)args[0];
       Path[] src = (Path[]) args[1];
@@ -104,7 +106,7 @@ public class TestFsShellConcat extends AbstractHadoopTestBase {
     // Verify concat result.
     ContractTestUtils
         .assertPathExists(lfs, "The target file doesn't exist.", dstPath);
-    Assertions.assertThat(lfs.listStatus(testRootDir).length).isEqualTo(1);
+    assertThat(lfs.listStatus(testRootDir).length).isEqualTo(1);
     assertEquals(expectContent.length, lfs.getFileStatus(dstPath).getLen());
     out = new ByteArrayOutputStream();
     try (InputStream in = lfs.open(dstPath)) {
@@ -119,11 +121,11 @@ public class TestFsShellConcat extends AbstractHadoopTestBase {
 
   @Test
   public void testUnsupportedFs() throws Exception {
-    FileSystem mockFs = Mockito.mock(FileSystem.class);
-    Mockito.doThrow(
+    FileSystem mockFs = mock(FileSystem.class);
+    doThrow(
         new UnsupportedOperationException("Mock unsupported exception."))
         .when(mockFs).concat(any(Path.class), any(Path[].class));
-    Mockito.doAnswer(invocationOnMock -> new URI("mockfs:///")).when(mockFs)
+    doAnswer(invocationOnMock -> new URI("mockfs:///")).when(mockFs)
         .getUri();
     Concat.setTestFs(mockFs);
     final ByteArrayOutputStream err = new ByteArrayOutputStream();
@@ -136,7 +138,7 @@ public class TestFsShellConcat extends AbstractHadoopTestBase {
     }
     System.err.print(err.toString());
     String expectedErrMsg = "Dest filesystem 'mockfs' doesn't support concat";
-    Assertions.assertThat(err.toString().contains(expectedErrMsg))
+    assertThat(err.toString().contains(expectedErrMsg))
         .withFailMessage("The err message should contain \"" + expectedErrMsg
             + "\" message.").isTrue();
   }

+ 5 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java

@@ -24,9 +24,9 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
 
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,14 +59,14 @@ public class TestDurationTracking extends AbstractHadoopTestBase {
 
   private final AtomicInteger invocationCounter = new AtomicInteger(0);
 
-  @Before
+  @BeforeEach
   public void setup() {
     stats = iostatisticsStore()
         .withDurationTracking(REQUESTS)
         .build();
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     LOG.info("stats {}", stats);
   }

+ 4 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java

@@ -23,9 +23,8 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-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;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -85,7 +84,7 @@ public class TestDynamicIOStatistics extends AbstractHadoopTestBase {
 
   private static final String[] KEYS = new String[]{ALONG, AINT, COUNT, EVAL};
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     statistics = dynamicIOStatistics()
         .withAtomicLongCounter(ALONG, aLong)
@@ -141,7 +140,7 @@ public class TestDynamicIOStatistics extends AbstractHadoopTestBase {
    */
   @Test
   public void testKeys() throws Throwable {
-    Assertions.assertThat(statistics.counters().keySet())
+    assertThat(statistics.counters().keySet())
         .describedAs("statistic keys of %s", statistics)
         .containsExactlyInAnyOrder(KEYS);
   }

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.statistics;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
 import org.apache.hadoop.test.AbstractHadoopTestBase;

+ 38 - 28
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java

@@ -21,14 +21,11 @@ package org.apache.hadoop.fs.statistics;
 import java.util.Arrays;
 import java.util.Collection;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.fs.statistics.impl.ForwardingIOStatisticsStore;
 import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticGauge;
@@ -36,6 +33,7 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatSt
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMean;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMinimum;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Test the {@link IOStatisticsSetters} interface implementations through
@@ -46,8 +44,6 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatist
  * is set, to verify it is harmless.
  */
 
-@RunWith(Parameterized.class)
-
 public class TestIOStatisticsSetters extends AbstractHadoopTestBase {
 
   public static final String COUNTER = "counter";
@@ -60,11 +56,10 @@ public class TestIOStatisticsSetters extends AbstractHadoopTestBase {
 
   public static final String MEAN = "mean";
 
-  private final IOStatisticsSetters ioStatistics;
+  private IOStatisticsSetters ioStatistics;
 
-  private final boolean createsNewEntries;
+  private boolean createsNewEntries;
 
-  @Parameterized.Parameters(name="{0}")
   public static Collection<Object[]> params() {
     return Arrays.asList(new Object[][]{
         {"IOStatisticsSnapshot", new IOStatisticsSnapshot(), true},
@@ -87,17 +82,20 @@ public class TestIOStatisticsSetters extends AbstractHadoopTestBase {
         .build();
   }
 
-  public TestIOStatisticsSetters(
-      String source,
-      IOStatisticsSetters ioStatisticsSetters,
-      boolean createsNewEntries) {
-    this.ioStatistics = ioStatisticsSetters;
-
-    this.createsNewEntries = createsNewEntries;
+  public void initTestIOStatisticsSetters(String source,
+      IOStatisticsSetters pIoStatisticsSetters,
+      boolean pCreatesNewEntries) {
+    this.ioStatistics = pIoStatisticsSetters;
+    this.createsNewEntries = pCreatesNewEntries;
   }
 
-  @Test
-  public void testCounter() throws Throwable {
+  @ParameterizedTest(name="{0}")
+  @MethodSource("params")
+  public void testCounter(String source,
+      IOStatisticsSetters pIoStatisticsSetters, boolean pCreatesNewEntries)
+      throws Throwable {
+    initTestIOStatisticsSetters(source, pIoStatisticsSetters, pCreatesNewEntries);
+
     // write
     ioStatistics.setCounter(COUNTER, 1);
     assertThatStatisticCounter(ioStatistics, COUNTER)
@@ -115,14 +113,17 @@ public class TestIOStatisticsSetters extends AbstractHadoopTestBase {
       assertThatStatisticCounter(ioStatistics, unknown)
           .isEqualTo(3);
     } else {
-      Assertions.assertThat(ioStatistics.counters())
+      assertThat(ioStatistics.counters())
           .describedAs("Counter map in {}", ioStatistics)
           .doesNotContainKey(unknown);
     }
   }
 
-  @Test
-  public void testMaximum() throws Throwable {
+  @ParameterizedTest(name="{0}")
+  @MethodSource("params")
+  public void testMaximum(String source,
+      IOStatisticsSetters pIoStatisticsSetters, boolean pCreatesNewEntries) throws Throwable {
+    initTestIOStatisticsSetters(source, pIoStatisticsSetters, pCreatesNewEntries);
     // write
     ioStatistics.setMaximum(MAXIMUM, 1);
     assertThatStatisticMaximum(ioStatistics, MAXIMUM)
@@ -137,8 +138,11 @@ public class TestIOStatisticsSetters extends AbstractHadoopTestBase {
     ioStatistics.setMaximum("mm2", 3);
   }
 
-  @Test
-  public void testMinimum() throws Throwable {
+  @ParameterizedTest(name="{0}")
+  @MethodSource("params")
+  public void testMinimum(String source,
+      IOStatisticsSetters pIoStatisticsSetters, boolean pCreatesNewEntries) throws Throwable {
+    initTestIOStatisticsSetters(source, pIoStatisticsSetters, pCreatesNewEntries);
     // write
     ioStatistics.setMinimum(MINIMUM, 1);
     assertThatStatisticMinimum(ioStatistics, MINIMUM)
@@ -153,8 +157,11 @@ public class TestIOStatisticsSetters extends AbstractHadoopTestBase {
     ioStatistics.setMinimum("c2", 3);
   }
 
-  @Test
-  public void testGauge() throws Throwable {
+  @ParameterizedTest(name="{0}")
+  @MethodSource("params")
+  public void testGauge(String source,
+      IOStatisticsSetters pIoStatisticsSetters, boolean pCreatesNewEntries) throws Throwable {
+    initTestIOStatisticsSetters(source, pIoStatisticsSetters, pCreatesNewEntries);
     // write
     ioStatistics.setGauge(GAUGE, 1);
     assertThatStatisticGauge(ioStatistics, GAUGE)
@@ -169,8 +176,11 @@ public class TestIOStatisticsSetters extends AbstractHadoopTestBase {
     ioStatistics.setGauge("g2", 3);
   }
 
-  @Test
-  public void testMean() throws Throwable {
+  @ParameterizedTest(name="{0}")
+  @MethodSource("params")
+  public void testMean(String source,
+      IOStatisticsSetters pIoStatisticsSetters, boolean pCreatesNewEntries) throws Throwable {
+    initTestIOStatisticsSetters(source, pIoStatisticsSetters, pCreatesNewEntries);
     // write
     final MeanStatistic mean11 = new MeanStatistic(1, 1);
     ioStatistics.setMeanStatistic(MEAN, mean11);

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java

@@ -19,8 +19,8 @@
 package org.apache.hadoop.fs.statistics;
 
 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;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,7 +52,7 @@ public class TestIOStatisticsSnapshot extends AbstractHadoopTestBase {
   /** Saved to the snapshot as "mean1". */
   private MeanStatistic mean1;
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     snapshot.counters().put("c1", 0L);
     snapshot.gauges().put("g1", 1L);

+ 12 - 12
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java

@@ -18,10 +18,9 @@
 
 package org.apache.hadoop.fs.statistics;
 
-import org.assertj.core.api.Assertions;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,6 +35,7 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatis
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticMinimumValue;
 import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Test the IOStatisticStore implementation.
@@ -60,7 +60,7 @@ public class TestIOStatisticsStore extends AbstractHadoopTestBase {
 
   private IOStatisticsStore stats;
 
-  @Before
+  @BeforeEach
   public void setup() {
     stats = iostatisticsStore()
         .withCounters(COUNT)
@@ -71,7 +71,7 @@ public class TestIOStatisticsStore extends AbstractHadoopTestBase {
         .build();
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     LOG.info("stats {}", stats);
   }
@@ -87,13 +87,13 @@ public class TestIOStatisticsStore extends AbstractHadoopTestBase {
     verifyStatisticGaugeValue(stats, GAUGE, 2);
     stats.setGauge(GAUGE, -1);
     verifyStatisticGaugeValue(stats, GAUGE, -1);
-    Assertions.assertThat(stats.incrementGauge(GAUGE, -1))
+    assertThat(stats.incrementGauge(GAUGE, -1))
         .isEqualTo(-2);
     verifyStatisticGaugeValue(stats, GAUGE, -2);
-    Assertions.assertThat(stats.getGaugeReference(GAUGE).get())
+    assertThat(stats.getGaugeReference(GAUGE).get())
         .isEqualTo(-2);
     stats.setGauge(UNKNOWN, 1);
-    Assertions.assertThat(stats.incrementGauge(UNKNOWN, 1))
+    assertThat(stats.incrementGauge(UNKNOWN, 1))
         .isEqualTo(0);
   }
 
@@ -162,15 +162,15 @@ public class TestIOStatisticsStore extends AbstractHadoopTestBase {
 
   @Test
   public void testUnknownCounter() throws Throwable {
-    Assertions.assertThat(stats.incrementCounter("unknown", -10))
+    assertThat(stats.incrementCounter("unknown", -10))
         .isEqualTo(0);
   }
 
   @Test
   public void testNegativeCounterIncrementIgnored() throws Throwable {
-    Assertions.assertThat(stats.incrementCounter(COUNT, 2))
+    assertThat(stats.incrementCounter(COUNT, 2))
         .isEqualTo(2);
-    Assertions.assertThat(stats.incrementCounter(COUNT, -10))
+    assertThat(stats.incrementCounter(COUNT, -10))
         .isEqualTo(2);
   }
 

+ 23 - 23
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java

@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.fs.statistics;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 import org.apache.hadoop.util.JsonSerialization;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Test the {@link MeanStatistic} class.
@@ -48,37 +48,37 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
 
   @Test
   public void testEmptiness() throws Throwable {
-    Assertions.assertThat(empty)
+    assertThat(empty)
         .matches(MeanStatistic::isEmpty, "is empty")
         .isEqualTo(new MeanStatistic(0, TEN))
         .isEqualTo(new MeanStatistic())
         .isNotEqualTo(tenFromOne);
-    Assertions.assertThat(empty.mean())
+    assertThat(empty.mean())
         .isEqualTo(ZEROD);
-    Assertions.assertThat(empty.toString())
+    assertThat(empty.toString())
         .contains("0.0");
   }
 
   @Test
   public void testTenFromOne() throws Throwable {
-    Assertions.assertThat(tenFromOne)
+    assertThat(tenFromOne)
         .matches(p -> !p.isEmpty(), "is not empty")
         .isEqualTo(tenFromOne)
         .isNotEqualTo(tenFromTen);
-    Assertions.assertThat(tenFromOne.mean())
+    assertThat(tenFromOne.mean())
         .isEqualTo(TEND);
   }
 
   @Test
   public void testNegativeSamplesAreEmpty() throws Throwable {
     MeanStatistic stat = new MeanStatistic(-10, 1);
-    Assertions.assertThat(stat)
+    assertThat(stat)
         .describedAs("stat with negative samples")
         .matches(MeanStatistic::isEmpty, "is empty")
         .isEqualTo(empty)
         .extracting(MeanStatistic::mean)
         .isEqualTo(ZEROD);
-    Assertions.assertThat(stat.toString())
+    assertThat(stat.toString())
         .contains("0.0");
 
   }
@@ -86,7 +86,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
   @Test
   public void testCopyNonEmpty() throws Throwable {
     MeanStatistic stat = tenFromOne.copy();
-    Assertions.assertThat(stat)
+    assertThat(stat)
         .describedAs("copy of " + tenFromOne)
         .isEqualTo(tenFromOne)
         .isNotSameAs(tenFromOne);
@@ -95,7 +95,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
   @Test
   public void testCopyEmpty() throws Throwable {
     MeanStatistic stat = empty.copy();
-    Assertions.assertThat(stat)
+    assertThat(stat)
         .describedAs("copy of " + empty)
         .isEqualTo(empty)
         .isNotSameAs(empty);
@@ -104,7 +104,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
   @Test
   public void testDoubleSamples() throws Throwable {
     MeanStatistic stat = tenFromOne.copy();
-    Assertions.assertThat(stat.add(tenFromOne))
+    assertThat(stat.add(tenFromOne))
         .isEqualTo(new MeanStatistic(2, 20))
         .extracting(MeanStatistic::mean)
         .isEqualTo(TEND);
@@ -113,21 +113,21 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
   @Test
   public void testAddEmptyR() throws Throwable {
     MeanStatistic stat = tenFromOne.copy();
-    Assertions.assertThat(stat.add(empty))
+    assertThat(stat.add(empty))
         .isEqualTo(tenFromOne);
   }
 
   @Test
   public void testAddEmptyL() throws Throwable {
     MeanStatistic stat = empty.copy();
-    Assertions.assertThat(stat.add(tenFromOne))
+    assertThat(stat.add(tenFromOne))
         .isEqualTo(tenFromOne);
   }
 
   @Test
   public void testAddEmptyLR() throws Throwable {
     MeanStatistic stat = empty.copy();
-    Assertions.assertThat(stat.add(empty))
+    assertThat(stat.add(empty))
         .isEqualTo(empty);
   }
 
@@ -135,7 +135,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
   public void testAddSampleToEmpty() throws Throwable {
     MeanStatistic stat = empty.copy();
     stat.addSample(TEN);
-    Assertions.assertThat(stat)
+    assertThat(stat)
         .isEqualTo(tenFromOne);
   }
 
@@ -145,7 +145,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
     for (int i = 0; i < 9; i++) {
       stat.addSample(0);
     }
-    Assertions.assertThat(stat)
+    assertThat(stat)
         .isEqualTo(tenFromTen);
   }
 
@@ -153,7 +153,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
   public void testSetSamples() throws Throwable {
     MeanStatistic stat = tenFromOne.copy();
     stat.setSamples(10);
-    Assertions.assertThat(stat)
+    assertThat(stat)
         .isEqualTo(tenFromTen);
   }
 
@@ -162,7 +162,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
     MeanStatistic stat = tenFromOne.copy();
     stat.setSum(100);
     stat.setSamples(20);
-    Assertions.assertThat(stat)
+    assertThat(stat)
         .isEqualTo(new MeanStatistic(20, 100))
         .extracting(MeanStatistic::mean)
         .isEqualTo(5.0d);
@@ -172,7 +172,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
   public void testSetNegativeSamplesMakesEmpty() throws Throwable {
     MeanStatistic stat = tenFromOne.copy();
     stat.setSamples(-3);
-    Assertions.assertThat(stat)
+    assertThat(stat)
         .isEqualTo(empty);
   }
 
@@ -182,14 +182,14 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
 
     String json = serializer.toJson(tenFromTen);
     LOG.info("serialized form\n{}", json);
-    Assertions.assertThat(json)
+    assertThat(json)
         .describedAs("JSON form of %s", tenFromTen)
         .doesNotContain("empty")
         .doesNotContain("mean");
 
     MeanStatistic deser = serializer.fromJson(json);
     LOG.info("deserialized {}", deser);
-    Assertions.assertThat(deser)
+    assertThat(deser)
         .isEqualTo(tenFromTen);
   }
 
@@ -205,7 +205,7 @@ public class TestMeanStatistic extends AbstractHadoopTestBase {
     JsonSerialization<MeanStatistic> serializer = serializer();
     MeanStatistic deser = serializer.fromJson(json);
     LOG.info("deserialized {}", deser);
-    Assertions.assertThat(deser)
+    assertThat(deser)
         .isEqualTo(empty);
   }
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/store/TestFSBuilderSupport.java

@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import javax.annotation.Nonnull;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSBuilder;

+ 31 - 31
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/wrappedio/impl/TestWrappedStatistics.java

@@ -24,9 +24,8 @@ import java.io.Serializable;
 import java.io.UncheckedIOException;
 import java.util.Map;
 
-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;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,6 +44,7 @@ import org.apache.hadoop.util.functional.Tuples;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Unit tests for IOStatistics wrapping.
@@ -76,7 +76,7 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
    */
   private Path jsonPath;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     String testDataDir = new FileSystemTestHelper().getTestRootDir();
     File tempDir = new File(testDataDir);
@@ -91,17 +91,17 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
    */
   @Test
   public void testLoaded() throws Throwable {
-    Assertions.assertThat(statistics.ioStatisticsAvailable())
+    assertThat(statistics.ioStatisticsAvailable())
         .describedAs("IOStatistics class must be available")
         .isTrue();
-    Assertions.assertThat(statistics.ioStatisticsContextAvailable())
+    assertThat(statistics.ioStatisticsContextAvailable())
         .describedAs("IOStatisticsContext must be available")
         .isTrue();
   }
 
   @Test
   public void testCreateEmptySnapshot() throws Throwable {
-    Assertions.assertThat(statistics.iostatisticsSnapshot_create())
+    assertThat(statistics.iostatisticsSnapshot_create())
         .describedAs("iostatisticsSnapshot_create()")
         .isInstanceOf(IOStatisticsSnapshot.class)
         .satisfies(statistics::isIOStatisticsSnapshot)
@@ -110,14 +110,14 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
 
   @Test
   public void testCreateNullSource() throws Throwable {
-    Assertions.assertThat(statistics.iostatisticsSnapshot_create(null))
+    assertThat(statistics.iostatisticsSnapshot_create(null))
         .describedAs("iostatisticsSnapshot_create(null)")
         .isInstanceOf(IOStatisticsSnapshot.class);
   }
 
   @Test
   public void testCreateOther() throws Throwable {
-    Assertions.assertThat(statistics.iostatisticsSnapshot_create(null))
+    assertThat(statistics.iostatisticsSnapshot_create(null))
         .describedAs("iostatisticsSnapshot_create(null)")
         .isInstanceOf(IOStatisticsSnapshot.class);
   }
@@ -130,14 +130,14 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
 
   @Test
   public void testRetrieveNullSource() throws Throwable {
-    Assertions.assertThat(statistics.iostatisticsSnapshot_retrieve(null))
+    assertThat(statistics.iostatisticsSnapshot_retrieve(null))
         .describedAs("iostatisticsSnapshot_retrieve(null)")
         .isNull();
   }
 
   @Test
   public void testRetrieveNonIOStatsSource() throws Throwable {
-    Assertions.assertThat(statistics.iostatisticsSnapshot_retrieve(this))
+    assertThat(statistics.iostatisticsSnapshot_retrieve(this))
         .describedAs("iostatisticsSnapshot_retrieve(this)")
         .isNull();
   }
@@ -184,23 +184,23 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
   @Test
   public void testIOStatisticsContextMethods() {
 
-    Assertions.assertThat(statistics.ioStatisticsContextAvailable())
+    assertThat(statistics.ioStatisticsContextAvailable())
         .describedAs("ioStatisticsContextAvailable() of %s", statistics)
         .isTrue();
-    Assertions.assertThat(statistics.iostatisticsContext_enabled())
+    assertThat(statistics.iostatisticsContext_enabled())
         .describedAs("iostatisticsContext_enabled() of %s", statistics)
         .isTrue();
 
     // get the current context, validate it
     final Object current = statistics.iostatisticsContext_getCurrent();
-    Assertions.assertThat(current)
+    assertThat(current)
         .describedAs("IOStatisticsContext")
         .isInstanceOf(IOStatisticsContext.class)
         .satisfies(statistics::isIOStatisticsSource);
 
     // take a snapshot
     final Serializable snapshot = statistics.iostatisticsContext_snapshot();
-    Assertions.assertThat(snapshot)
+    assertThat(snapshot)
         .satisfies(statistics::isIOStatisticsSnapshot);
 
     // use the retrieve API to create a snapshot from the IOStatisticsSource interface
@@ -223,14 +223,14 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
     // set to null
     statistics.iostatisticsContext_setThreadIOStatisticsContext(null);
 
-    Assertions.assertThat(statistics.iostatisticsContext_getCurrent())
+    assertThat(statistics.iostatisticsContext_getCurrent())
         .describedAs("current IOStatisticsContext after resetting")
         .isNotSameAs(current);
 
     // then set to the "current"  value
     statistics.iostatisticsContext_setThreadIOStatisticsContext(current);
 
-    Assertions.assertThat(statistics.iostatisticsContext_getCurrent())
+    assertThat(statistics.iostatisticsContext_getCurrent())
         .describedAs("current IOStatisticsContext after resetting")
         .isSameAs(current);
 
@@ -238,7 +238,7 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
     statistics.iostatisticsContext_reset();
 
     // now aggregate the retrieved stats into it.
-    Assertions.assertThat(statistics.iostatisticsContext_aggregate(retrieved))
+    assertThat(statistics.iostatisticsContext_aggregate(retrieved))
         .describedAs("iostatisticsContext_aggregate of %s", retrieved)
         .isTrue();
   }
@@ -275,7 +275,7 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
    * @param expected expected value
    */
   private void assertJsonEqual(Serializable actual, Serializable expected) {
-    Assertions.assertThat(toJsonString(actual))
+    assertThat(toJsonString(actual))
         .describedAs("JSON format string of %s", actual)
         .isEqualTo(toJsonString(expected));
   }
@@ -346,26 +346,26 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
 
 
     // get the values
-    Assertions.assertThat(statistics.iostatistics_counters(loaded))
+    assertThat(statistics.iostatistics_counters(loaded))
         .containsOnlyKeys("c1", "c2",
             "d1", "d1.failures",
             "d2", "d2.failures")
         .containsEntry("c1", 1L)
         .containsEntry("d1", 2L)
         .containsEntry("d2", 1L);
-    Assertions.assertThat(statistics.iostatistics_gauges(loaded))
+    assertThat(statistics.iostatistics_gauges(loaded))
         .containsOnlyKeys("g1")
         .containsEntry("g1", 10L);
 
     final Map<String, Long> minimums = statistics.iostatistics_minimums(snapshot);
-    Assertions.assertThat(minimums)
+    assertThat(minimums)
         .containsEntry("d1.min", 0L);
     final long d2FailuresMin = minimums.get("d2.failures.min");
-    Assertions.assertThat(d2FailuresMin)
+    assertThat(d2FailuresMin)
         .describedAs("min d2.failures")
         .isGreaterThan(0);
     final Map<String, Long> maximums = statistics.iostatistics_maximums(snapshot);
-    Assertions.assertThat(maximums)
+    assertThat(maximums)
         .containsEntry("d2.failures.max", d2FailuresMin);
     final long d1Max = maximums.get("d1.max");
 
@@ -373,7 +373,7 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
     final Map<String, Map.Entry<Long, Long>> means =
         statistics.iostatistics_means(snapshot);
 
-    Assertions.assertThat(means)
+    assertThat(means)
         .containsEntry("d1.mean", Tuples.pair(2L, d1Max))
         .containsEntry("d2.failures.mean", Tuples.pair(1L, d2FailuresMin));
 
@@ -401,19 +401,19 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
         new DynamicWrappedStatistics(StubClass.class.getName());
 
     // probes which just return false
-    Assertions.assertThat(missing.ioStatisticsAvailable())
+    assertThat(missing.ioStatisticsAvailable())
         .describedAs("ioStatisticsAvailable() of %s", missing)
         .isFalse();
 
     // probes of type of argument which return false if the
     // methods are missing
-    Assertions.assertThat(missing.isIOStatistics(SERIALIZABLE))
+    assertThat(missing.isIOStatistics(SERIALIZABLE))
         .describedAs("isIOStatistics() of %s", missing)
         .isFalse();
-    Assertions.assertThat(missing.isIOStatisticsSource(SERIALIZABLE))
+    assertThat(missing.isIOStatisticsSource(SERIALIZABLE))
         .describedAs("isIOStatisticsSource() of %s", missing)
         .isFalse();
-    Assertions.assertThat(missing.isIOStatisticsSnapshot(SERIALIZABLE))
+    assertThat(missing.isIOStatisticsSnapshot(SERIALIZABLE))
         .describedAs("isIOStatisticsSnapshot() of %s", missing)
         .isFalse();
 
@@ -464,10 +464,10 @@ public class TestWrappedStatistics extends AbstractHadoopTestBase {
         new DynamicWrappedStatistics(StubClass.class.getName());
 
     // probes which just return false
-    Assertions.assertThat(missing.ioStatisticsContextAvailable())
+    assertThat(missing.ioStatisticsContextAvailable())
         .describedAs("ioStatisticsContextAvailable() of %s", missing)
         .isFalse();
-    Assertions.assertThat(missing.iostatisticsContext_enabled())
+    assertThat(missing.iostatisticsContext_enabled())
         .describedAs("iostatisticsContext_enabled() of %s", missing)
         .isFalse();
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestShadedProtobufHelper.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.ipc;
 
 import java.io.IOException;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.ipc.internal.ShadedProtobufHelper;
 import org.apache.hadoop.test.AbstractHadoopTestBase;

+ 14 - 18
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/AbstractHadoopTestBase.java

@@ -17,13 +17,14 @@
  */
 package org.apache.hadoop.test;
 
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
 import java.util.concurrent.TimeUnit;
 
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.rules.TestName;
-import org.junit.rules.Timeout;
+import static org.apache.hadoop.test.AbstractHadoopTestBase.TEST_DEFAULT_TIMEOUT_VALUE;
 
 /**
  * A base class for JUnit5+ tests that sets a default timeout for all tests
@@ -35,6 +36,7 @@ import org.junit.rules.Timeout;
  * Unlike {@link HadoopTestBase} this class does not extend JUnit Assert
  * so is easier to use with AssertJ.
  */
+@Timeout(value = TEST_DEFAULT_TIMEOUT_VALUE, unit = TimeUnit.MILLISECONDS)
 public abstract class AbstractHadoopTestBase {
 
   /**
@@ -50,12 +52,6 @@ public abstract class AbstractHadoopTestBase {
    */
   public static final int TEST_DEFAULT_TIMEOUT_VALUE = 100000;
 
-  /**
-   * The JUnit rule that sets the default timeout for tests.
-   */
-  @Rule
-  public Timeout defaultTimeout = retrieveTestTimeout();
-
   /**
    * Retrieve the test timeout from the system property
    * {@link #PROPERTY_TEST_DEFAULT_TIMEOUT}, falling back to
@@ -63,7 +59,7 @@ public abstract class AbstractHadoopTestBase {
    * property is not defined.
    * @return the recommended timeout for tests
    */
-  public static Timeout retrieveTestTimeout() {
+  public static int retrieveTestTimeout() {
     String propval = System.getProperty(PROPERTY_TEST_DEFAULT_TIMEOUT,
                                          Integer.toString(
                                            TEST_DEFAULT_TIMEOUT_VALUE));
@@ -72,16 +68,16 @@ public abstract class AbstractHadoopTestBase {
       millis = Integer.parseInt(propval);
     } catch (NumberFormatException e) {
       //fall back to the default value, as the property cannot be parsed
-      millis = TEST_DEFAULT_TIMEOUT_VALUE;
+      millis = 100000;
     }
-    return new Timeout(millis, TimeUnit.MILLISECONDS);
+    return millis;
   }
 
   /**
    * The method name.
    */
-  @Rule
-  public TestName methodName = new TestName();
+  @RegisterExtension
+  private TestName methodName = new TestName();
 
   /**
    * Get the method name; defaults to the value of {@link #methodName}.
@@ -95,7 +91,7 @@ public abstract class AbstractHadoopTestBase {
   /**
    * Static initializer names this thread "JUnit".
    */
-  @BeforeClass
+  @BeforeAll
   public static void nameTestThread() {
     Thread.currentThread().setName("JUnit");
   }
@@ -103,7 +99,7 @@ public abstract class AbstractHadoopTestBase {
   /**
    * Before each method, the thread is renamed to match the method name.
    */
-  @Before
+  @BeforeEach
   public void nameThreadToMethod() {
     Thread.currentThread().setName("JUnit-" + getMethodName());
   }

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEnclosingRoot.java

@@ -31,9 +31,9 @@ import org.apache.hadoop.hdfs.server.namenode.EncryptionFaultInjector;
 import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 import org.apache.hadoop.test.GenericTestUtils;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -59,7 +59,7 @@ public class TestEnclosingRoot extends AbstractHadoopTestBase {
         new Path(testRootDir.toString(), "test.jks").toUri();
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     conf = new HdfsConfiguration();
     fsHelper = new FileSystemTestHelper();
@@ -90,7 +90,7 @@ public class TestEnclosingRoot extends AbstractHadoopTestBase {
         .getProvider());
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     try {
       if (cluster != null) {

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestXMLUtils.java

@@ -17,12 +17,12 @@
  */
 package org.apache.hadoop.hdfs.util;
 
-import org.apache.hadoop.hdfs.util.XMLUtils.UnmanglingError;
-import org.junit.jupiter.api.Test;
-
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.fail;
 
+import org.apache.hadoop.hdfs.util.XMLUtils.UnmanglingError;
+import org.junit.jupiter.api.Test;
+
 public class TestXMLUtils {
   private static void testRoundTripImpl(String str, String expectedMangled,
                                     boolean encodeEntityRefs) {

+ 55 - 55
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java

@@ -24,12 +24,11 @@ import java.net.ConnectException;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.time.Duration;
+import java.util.concurrent.TimeUnit;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.io.TempDir;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import software.amazon.awssdk.core.SdkClient;
@@ -68,16 +67,22 @@ import static java.lang.String.format;
 import static java.util.Objects.requireNonNull;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3ATestConstants.EU_WEST_1;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
-import static org.junit.Assert.*;
+import static org.assertj.core.api.Assertions.assertThat;
+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.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * S3A tests for configuration, especially credentials.
  */
+@Timeout(value = S3A_TEST_TIMEOUT, unit = TimeUnit.MILLISECONDS)
 public class ITestS3AConfiguration extends AbstractHadoopTestBase {
   private static final String EXAMPLE_ID = "AKASOMEACCESSKEY";
   private static final String EXAMPLE_KEY =
@@ -91,13 +96,8 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
   private static final Logger LOG =
       LoggerFactory.getLogger(ITestS3AConfiguration.class);
 
-  @Rule
-  public Timeout testTimeout = new Timeout(
-      S3ATestConstants.S3A_TEST_TIMEOUT
-  );
-
-  @Rule
-  public final TemporaryFolder tempDir = new TemporaryFolder();
+  @TempDir
+  private java.nio.file.Path tempDir;
 
   /**
    * Get the S3 client of the active filesystem.
@@ -154,8 +154,8 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
         fail("Unexpected endpoint");
       }
       String region = getS3AInternals().getBucketLocation();
-      assertEquals("Endpoint config setting and bucket location differ: ",
-          endPointRegion, region);
+      assertEquals(endPointRegion, region,
+          "Endpoint config setting and bucket location differ: ");
     }
   }
 
@@ -263,7 +263,7 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
   public void testCredsFromCredentialProvider() throws Exception {
     // set up conf to have a cred provider
     final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
+    final File file = tempDir.resolve("test.jks").toFile();
     final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
         file.toURI());
     conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
@@ -274,8 +274,8 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
     S3xLoginHelper.Login creds =
         S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
+    assertEquals(EXAMPLE_ID, creds.getUser(), "AccessKey incorrect.");
+    assertEquals(EXAMPLE_KEY, creds.getPassword(), "SecretKey incorrect.");
   }
 
   void provisionAccessKeys(final Configuration conf) throws Exception {
@@ -293,7 +293,7 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
   public void testSecretFromCredentialProviderIDFromConfig() throws Exception {
     // set up conf to have a cred provider
     final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
+    final File file = tempDir.resolve("test.jks").toFile();
     final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
         file.toURI());
     conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
@@ -309,15 +309,15 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID);
     S3xLoginHelper.Login creds =
         S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
+    assertEquals(EXAMPLE_ID, creds.getUser(), "AccessKey incorrect.");
+    assertEquals(EXAMPLE_KEY, creds.getPassword(), "SecretKey incorrect.");
   }
 
   @Test
   public void testIDFromCredentialProviderSecretFromConfig() throws Exception {
     // set up conf to have a cred provider
     final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
+    final File file = tempDir.resolve("test.jks").toFile();
     final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
         file.toURI());
     conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
@@ -333,15 +333,15 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     conf.set(Constants.SECRET_KEY, EXAMPLE_KEY);
     S3xLoginHelper.Login creds =
         S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
+    assertEquals(EXAMPLE_ID, creds.getUser(), "AccessKey incorrect.");
+    assertEquals(EXAMPLE_KEY, creds.getPassword(), "SecretKey incorrect.");
   }
 
   @Test
   public void testExcludingS3ACredentialProvider() throws Exception {
     // set up conf to have a cred provider
     final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
+    final File file = tempDir.resolve("test.jks").toFile();
     final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
         file.toURI());
     conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
@@ -352,7 +352,7 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
         conf, S3AFileSystem.class);
     String newPath = conf.get(
         CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH);
-    assertFalse("Provider Path incorrect", newPath.contains("s3a://"));
+    assertFalse(newPath.contains("s3a://"), "Provider Path incorrect");
 
     // now let's make sure the new path is created by the S3AFileSystem
     // and the integration still works. Let's provision the keys through
@@ -364,8 +364,8 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     URI uri2 = new URI("s3a://foobar");
     S3xLoginHelper.Login creds =
         S3AUtils.getAWSAccessKeys(uri2, conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
+    assertEquals(EXAMPLE_ID, creds.getUser(), "AccessKey incorrect.");
+    assertEquals(EXAMPLE_KEY, creds.getPassword(), "SecretKey incorrect.");
 
   }
 
@@ -388,8 +388,8 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
           "clientConfiguration");
       S3Configuration s3Configuration =
           (S3Configuration)clientConfiguration.option(SdkClientOption.SERVICE_CONFIGURATION);
-      assertTrue("Expected to find path style access to be switched on!",
-          s3Configuration.pathStyleAccessEnabled());
+      assertTrue(s3Configuration.pathStyleAccessEnabled(),
+          "Expected to find path style access to be switched on!");
       byte[] file = ContractTestUtils.toAsciiByteArray("test file");
       ContractTestUtils.writeAndRead(fs,
           createTestPath(new Path("/path/style/access/testFile")),
@@ -423,7 +423,7 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     S3Client s3 = getS3Client("User Agent");
     SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class,
         "clientConfiguration");
-    Assertions.assertThat(clientConfiguration.option(SdkClientOption.CLIENT_USER_AGENT))
+    assertThat(clientConfiguration.option(SdkClientOption.CLIENT_USER_AGENT))
         .describedAs("User Agent prefix")
         .startsWith("Hadoop " + VersionInfo.getVersion());
   }
@@ -439,7 +439,7 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     S3Client s3 = getS3Client("User agent");
     SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class,
         "clientConfiguration");
-    Assertions.assertThat(clientConfiguration.option(SdkClientOption.CLIENT_USER_AGENT))
+    assertThat(clientConfiguration.option(SdkClientOption.CLIENT_USER_AGENT))
         .describedAs("User Agent prefix")
         .startsWith("MyApp, Hadoop " + VersionInfo.getVersion());
   }
@@ -460,7 +460,7 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
       }
       SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class,
           "clientConfiguration");
-      Assertions.assertThat(clientConfiguration.option(SdkClientOption.API_CALL_ATTEMPT_TIMEOUT))
+      assertThat(clientConfiguration.option(SdkClientOption.API_CALL_ATTEMPT_TIMEOUT))
           .describedAs("Configured " + REQUEST_TIMEOUT +
               " is different than what AWS sdk configuration uses internally")
           .isEqualTo(timeout);
@@ -477,12 +477,12 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
         getS3AInternals().shareCredentials("testCloseIdempotent");
     credentials.close();
     fs.close();
-    assertTrue("Closing FS didn't close credentials " + credentials,
-        credentials.isClosed());
-    assertEquals("refcount not zero in " + credentials, 0, credentials.getRefCount());
+    assertTrue(credentials.isClosed(),
+        "Closing FS didn't close credentials " + credentials);
+    assertEquals(0, credentials.getRefCount(), "refcount not zero in " + credentials);
     fs.close();
     // and the numbers should not change
-    assertEquals("refcount not zero in " + credentials, 0, credentials.getRefCount());
+    assertEquals(0, credentials.getRefCount(), "refcount not zero in " + credentials);
   }
 
   @Test
@@ -496,11 +496,11 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     try {
       fs = S3ATestUtils.createTestFileSystem(conf);
       final Configuration fsConf = fs.getConf();
-      Assertions.assertThat(fsConf.get(Constants.BUFFER_DIR))
+      assertThat(fsConf.get(Constants.BUFFER_DIR))
           .describedAs("Config option %s", Constants.BUFFER_DIR)
           .isEqualTo(blank);
       File tmp = createTemporaryFileForWriting();
-      assertTrue("not found: " + tmp, tmp.exists());
+      assertTrue(tmp.exists(), "not found: " + tmp);
       tmp.delete();
     } finally {
       removeAllocatorContexts();
@@ -535,15 +535,15 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     conf.set(format("fs.s3a.bucket.%s.buffer.dir", bucketName), dirs);
     fs = S3ATestUtils.createTestFileSystem(conf);
     final Configuration fsConf = fs.getConf();
-    Assertions.assertThat(fsConf.get(Constants.BUFFER_DIR))
+    assertThat(fsConf.get(Constants.BUFFER_DIR))
         .describedAs("Config option %s", Constants.BUFFER_DIR)
         .isEqualTo(dirs);
     File tmp1 = createTemporaryFileForWriting();
     tmp1.delete();
     File tmp2 = createTemporaryFileForWriting();
     tmp2.delete();
-    assertNotEquals("round robin not working",
-        tmp1.getParent(), tmp2.getParent());
+    assertNotEquals(tmp1.getParent(), tmp2.getParent(),
+        "round robin not working");
   }
 
   @Test
@@ -559,10 +559,10 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
         return S3ATestUtils.createTestFileSystem(conf);
       }
     });
-    assertEquals("username", alice, fs.getUsername());
+    assertEquals(alice, fs.getUsername(), "username");
     FileStatus status = fs.getFileStatus(new Path("/"));
-    assertEquals("owner in " + status, alice, status.getOwner());
-    assertEquals("group in " + status, alice, status.getGroup());
+    assertEquals(alice, status.getOwner(), "owner in " + status);
+    assertEquals(alice, status.getGroup(), "group in " + status);
   }
 
   /**
@@ -579,13 +579,12 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
   private static <T> T getField(Object target, Class<T> fieldType,
       String fieldName) throws IllegalAccessException {
     Object obj = FieldUtils.readField(target, fieldName, true);
-    assertNotNull(format(
+    assertNotNull(obj, format(
         "Could not read field named %s in object with class %s.", fieldName,
-        target.getClass().getName()), obj);
-    assertTrue(format(
+        target.getClass().getName()));
+    assertTrue(fieldType.isAssignableFrom(obj.getClass()), format(
         "Unexpected type found for field named %s, expected %s, actual %s.",
-        fieldName, fieldType.getName(), obj.getClass().getName()),
-        fieldType.isAssignableFrom(obj.getClass()));
+        fieldName, fieldType.getName(), obj.getClass().getName()));
     return fieldType.cast(obj);
   }
 
@@ -600,7 +599,8 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
     assertOptionEquals(updated, "fs.s3a.propagation", "propagated");
   }
 
-  @Test(timeout = 10_000L)
+  @Test
+  @Timeout(10)
   public void testS3SpecificSignerOverride() throws Exception {
     Configuration config = new Configuration();
     removeBaseAndBucketOverrides(config,
@@ -633,10 +633,10 @@ public class ITestS3AConfiguration extends AbstractHadoopTestBase {
         Invoker.once("head", bucket, () ->
             s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build())));
 
-    Assertions.assertThat(CustomS3Signer.isS3SignerCalled())
+    assertThat(CustomS3Signer.isS3SignerCalled())
         .describedAs("Custom S3 signer not called").isTrue();
 
-    Assertions.assertThat(CustomSTSSigner.isSTSSignerCalled())
+    assertThat(CustomSTSSigner.isSTSSignerCalled())
         .describedAs("Custom STS signer not called").isTrue();
   }
 

+ 16 - 16
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestBucketConfiguration.java

@@ -20,13 +20,12 @@ package org.apache.hadoop.fs.s3a;
 
 import java.io.File;
 import java.net.URI;
+import java.nio.file.Path;
 import java.util.Collection;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
@@ -51,8 +50,9 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.getEncryptionAlgorithm;
 import static org.apache.hadoop.fs.s3a.S3AUtils.patchSecurityCredentialProviders;
 import static org.apache.hadoop.fs.s3a.S3AUtils.propagateBucketOptions;
 import static org.apache.hadoop.fs.s3a.S3AUtils.setBucketOption;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * S3A tests for configuration option propagation.
@@ -62,14 +62,14 @@ public class TestBucketConfiguration extends AbstractHadoopTestBase {
 
   private static final String NEW_ALGORITHM_KEY_GLOBAL = "CSE-KMS";
   private static final String OLD_ALGORITHM_KEY_BUCKET = "SSE-KMS";
-  @Rule
-  public final TemporaryFolder tempDir = new TemporaryFolder();
+  @TempDir
+  private Path tempDir;
 
   /**
    * Setup: create the contract then init it.
    * @throws Exception on any failure
    */
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     // forces in deprecation wireup, even when this test method is running isolated
     S3AFileSystem.initializeClass();
@@ -90,10 +90,10 @@ public class TestBucketConfiguration extends AbstractHadoopTestBase {
 
     String[] sources = updated.getPropertySources(basekey);
     assertEquals(1, sources.length);
-    Assertions.assertThat(sources)
+    assertThat(sources)
         .describedAs("base key property sources")
         .hasSize(1);
-    Assertions.assertThat(sources[0])
+    assertThat(sources[0])
         .describedAs("Property source")
         .contains(bucketKey);
   }
@@ -217,7 +217,7 @@ public class TestBucketConfiguration extends AbstractHadoopTestBase {
     // Get the encryption method and verify that the value is per-bucket of
     // old keys.
     String value = getEncryptionAlgorithm("b", updated).getMethod();
-    Assertions.assertThat(value)
+    assertThat(value)
         .describedAs("lookupPassword(%s)", S3_ENCRYPTION_ALGORITHM)
         .isEqualTo(OLD_ALGORITHM_KEY_BUCKET);
   }
@@ -226,7 +226,7 @@ public class TestBucketConfiguration extends AbstractHadoopTestBase {
   public void testJceksDeprecatedEncryptionAlgorithm() throws Exception {
     // set up conf to have a cred provider
     final Configuration conf = new Configuration(false);
-    final File file = tempDir.newFile("test.jks");
+    final File file = tempDir.resolve("test.jks").toFile();
     final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
         file.toURI());
     conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
@@ -251,11 +251,11 @@ public class TestBucketConfiguration extends AbstractHadoopTestBase {
     // Get the encryption method and verify that the value is per-bucket of
     // old keys.
     final EncryptionSecrets secrets = S3AUtils.buildEncryptionSecrets("b", conf);
-    Assertions.assertThat(secrets.getEncryptionMethod().getMethod())
+    assertThat(secrets.getEncryptionMethod().getMethod())
         .describedAs("buildEncryptionSecrets() encryption algorithm resolved to %s", secrets)
         .isEqualTo(OLD_ALGORITHM_KEY_BUCKET);
 
-    Assertions.assertThat(secrets.getEncryptionKey())
+    assertThat(secrets.getEncryptionKey())
         .describedAs("buildEncryptionSecrets() encryption key resolved to %s", secrets)
         .isEqualTo(bucketKey);
 

+ 2 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInstrumentationLifecycle.java

@@ -20,8 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 import java.net.URI;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.fs.impl.WeakRefMetricsSource;
 import org.apache.hadoop.metrics2.MetricsSource;
@@ -58,7 +57,7 @@ public class TestInstrumentationLifecycle extends AbstractHadoopTestBase {
     final MetricsSource source = activeMetrics.getSource(metricSourceName);
     // verify the source is registered through a weak ref, and that the
     // reference maps to the instance.
-    Assertions.assertThat(source)
+    assertThat(source)
         .describedAs("metric source %s", metricSourceName)
         .isNotNull()
         .isInstanceOf(WeakRefMetricsSource.class)

+ 31 - 25
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java

@@ -27,7 +27,10 @@ import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.maybeTranslateAudi
 import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.maybeExtractChannelException;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.*;
 import static org.apache.hadoop.test.LambdaTestUtils.verifyCause;
-import static org.junit.Assert.*;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.EOFException;
 import java.io.FileNotFoundException;
@@ -37,8 +40,7 @@ import java.nio.file.AccessDeniedException;
 import java.util.concurrent.ExecutionException;
 import java.util.function.Consumer;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Before;
+import org.junit.jupiter.api.BeforeEach;
 import software.amazon.awssdk.awscore.exception.AwsErrorDetails;
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 import software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException;
@@ -47,7 +49,7 @@ import software.amazon.awssdk.core.exception.SdkException;
 import software.amazon.awssdk.http.SdkHttpResponse;
 import software.amazon.awssdk.services.s3.model.S3Exception;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.api.UnsupportedRequestException;
@@ -77,7 +79,7 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
    */
   private S3ARetryPolicy retryPolicy;
 
-  @Before
+  @BeforeEach
   public void setup() {
     retryPolicy = new S3ARetryPolicy(new Configuration(false));
   }
@@ -106,8 +108,8 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
   }
 
   protected void assertContained(String text, String contained) {
-    assertTrue("string \""+ contained + "\" not found in \"" + text + "\"",
-        text != null && text.contains(contained));
+    assertTrue(text != null && text.contains(contained),
+        "string \""+ contained + "\" not found in \"" + text + "\"");
   }
 
   protected <E extends Throwable> E verifyTranslated(
@@ -165,7 +167,7 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
     // 416 maps the the subclass of EOFException
     final IOException ex = verifyTranslated(SC_416_RANGE_NOT_SATISFIABLE,
             RangeNotSatisfiableEOFException.class);
-    Assertions.assertThat(ex)
+    assertThat(ex)
         .isInstanceOf(EOFException.class);
   }
 
@@ -192,7 +194,7 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
   }
 
   protected void assertStatusCode(int expected, AWSServiceIOException ex) {
-    assertNotNull("Null exception", ex);
+    assertNotNull(ex, "Null exception");
     if (expected != ex.statusCode()) {
       throw new AssertionError("Expected status code " + expected
           + "but got " + ex.statusCode(),
@@ -260,22 +262,26 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
         new InterruptedIOException("ioirq"));
   }
 
-  @Test(expected = InterruptedIOException.class)
+  @Test
   public void testExtractInterrupted() throws Throwable {
-    throw extractException("", "",
+    assertThrows(InterruptedIOException.class, () -> {
+      throw extractException("", "",
         new ExecutionException(
             SdkException.builder()
-                .cause(new InterruptedException(""))
-                .build()));
+            .cause(new InterruptedException(""))
+            .build()));
+    });
   }
 
-  @Test(expected = InterruptedIOException.class)
+  @Test
   public void testExtractInterruptedIO() throws Throwable {
-    throw extractException("", "",
+    assertThrows(InterruptedIOException.class, () -> {
+      throw extractException("", "",
         new ExecutionException(
             SdkException.builder()
-                .cause(new InterruptedIOException(""))
-                .build()));
+            .cause(new InterruptedIOException(""))
+            .build()));
+    });
   }
 
   @Test
@@ -294,10 +300,10 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
                     new CredentialInitializationException("Credential initialization failed"))));
     // unwrap and verify that the initial client exception has been stripped
     final Throwable cause = ex.getCause();
-    Assertions.assertThat(cause)
+    assertThat(cause)
         .isInstanceOf(CredentialInitializationException.class);
     CredentialInitializationException cie = (CredentialInitializationException) cause;
-    Assertions.assertThat(cie.retryable())
+    assertThat(cie.retryable())
         .describedAs("Retryable flag")
         .isFalse();
   }
@@ -305,11 +311,11 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
 
   @Test
   public void testTranslateNonCredentialException() throws Throwable {
-    Assertions.assertThat(
+    assertThat(
             maybeTranslateCredentialException("/",
                 sdkClientException("not a credential exception", null)))
         .isNull();
-    Assertions.assertThat(
+    assertThat(
             maybeTranslateCredentialException("/",
                 sdkClientException("", sdkClientException("not a credential exception", null))))
         .isNull();
@@ -336,17 +342,17 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
         verifyExceptionClass(UnsupportedRequestException.class,
             maybeTranslateAuditException("/",
                 sdkClientException("", new AuditOperationRejectedException("rejected"))));
-    Assertions.assertThat(ex.getCause())
+    assertThat(ex.getCause())
         .isInstanceOf(AuditOperationRejectedException.class);
   }
 
   @Test
   public void testTranslateNonAuditException() throws Throwable {
-    Assertions.assertThat(
+    assertThat(
             maybeTranslateAuditException("/",
                 sdkClientException("not an audit exception", null)))
         .isNull();
-    Assertions.assertThat(
+    assertThat(
             maybeTranslateAuditException("/",
                 sdkClientException("", sdkClientException("not an audit exception", null))))
         .isNull();
@@ -473,7 +479,7 @@ public class TestS3AExceptionTranslation extends AbstractHadoopTestBase {
   private void assertRetryOutcome(
       final Exception ex,
       final RetryPolicy.RetryAction.RetryDecision decision) throws Exception {
-    Assertions.assertThat(retryPolicy.shouldRetry(ex, 0, 0, true).action)
+    assertThat(retryPolicy.shouldRetry(ex, 0, 0, true).action)
         .describedAs("retry policy for exception %s", ex)
         .isEqualTo(decision);
   }

+ 4 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java

@@ -20,8 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 import java.io.IOException;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import software.amazon.awssdk.http.apache.ProxyConfiguration;
 
 import org.apache.hadoop.conf.Configuration;
@@ -31,6 +30,7 @@ import org.apache.hadoop.test.AbstractHadoopTestBase;
 import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST;
 import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT;
 import static org.apache.hadoop.fs.s3a.Constants.PROXY_SECURED;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Tests to verify {@link S3AUtils} translates the proxy configurations
@@ -82,10 +82,10 @@ public class TestS3AProxy extends AbstractHadoopTestBase {
         AWSClientConfig.createProxyConfiguration(proxyConfig, "testBucket");
     ProxyConfiguration asyncConfig =
         AWSClientConfig.createProxyConfiguration(proxyConfig, "testBucket");
-    Assertions.assertThat(config.scheme())
+    assertThat(config.scheme())
         .describedAs("Proxy protocol not as expected")
         .isEqualTo(isExpectedSecured ? "https" : "http");
-    Assertions.assertThat(asyncConfig.scheme())
+    assertThat(asyncConfig.scheme())
         .describedAs("Proxy protocol not as expected")
         .isEqualTo(isExpectedSecured ? "https" : "http");
   }

+ 4 - 5
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AccessGrantConfiguration.java

@@ -23,8 +23,7 @@ import java.net.URI;
 import java.net.URISyntaxException;
 
 import org.assertj.core.api.AbstractStringAssert;
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import software.amazon.awssdk.awscore.AwsClient;
 import software.amazon.awssdk.s3accessgrants.plugin.S3AccessGrantsIdentityProvider;
 
@@ -32,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESS_GRANTS_ENABLED;
+import static org.assertj.core.api.Assertions.assertThat;
 
 
 /**
@@ -114,9 +114,8 @@ public class TestS3AccessGrantConfiguration extends AbstractHadoopTestBase {
       throws IOException, URISyntaxException {
     AwsClient awsClient = getAwsClient(configuration, asyncClient);
     AbstractStringAssert<?> assertion =
-        Assertions
-            .assertThat(S3_ACCESS_GRANTS_EXPECTED_CREDENTIAL_PROVIDER_CLASS)
-            .describedAs(message);
+        assertThat(S3_ACCESS_GRANTS_EXPECTED_CREDENTIAL_PROVIDER_CLASS)
+        .describedAs(message);
     if (shouldMatch) {
       assertion.isEqualTo(getCredentialProviderName(awsClient));
     } else {

+ 16 - 18
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSSEConfiguration.java

@@ -21,12 +21,11 @@ package org.apache.hadoop.fs.s3a;
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
+import java.nio.file.Path;
+import java.util.concurrent.TimeUnit;
 
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.impl.S3AEncryption;
@@ -34,9 +33,12 @@ import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.io.TempDir;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.*;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.S3A_TEST_TIMEOUT;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 
@@ -45,7 +47,8 @@ import static org.apache.hadoop.test.LambdaTestUtils.*;
  * Tests related to secret providers and AWS credentials are also
  * included, as they share some common setup operations.
  */
-public class TestSSEConfiguration extends Assert {
+@Timeout(value = S3A_TEST_TIMEOUT, unit = TimeUnit.MILLISECONDS)
+public class TestSSEConfiguration extends Assertions {
 
   /** Bucket to use for per-bucket options. */
   public static final String BUCKET = "dataset-1";
@@ -53,13 +56,8 @@ public class TestSSEConfiguration extends Assert {
   /** Valid set of key/value pairs for the encryption context. */
   private static final String VALID_ENCRYPTION_CONTEXT = "key1=value1, key2=value2, key3=value3";
 
-  @Rule
-  public Timeout testTimeout = new Timeout(
-      S3ATestConstants.S3A_TEST_TIMEOUT
-  );
-
-  @Rule
-  public final TemporaryFolder tempDir = new TemporaryFolder();
+  @TempDir
+  private Path tempDir;
 
   @Test
   public void testSSECNoKey() throws Throwable {
@@ -115,8 +113,8 @@ public class TestSSEConfiguration extends Assert {
     conf.set(Constants.S3_ENCRYPTION_KEY, "keyInConfObject");
 
     String sseKey = getS3EncryptionKey(BUCKET, conf);
-    assertNotNull("Proxy password should not retrun null.", sseKey);
-    assertEquals("Proxy password override did NOT work.", key, sseKey);
+    assertNotNull(sseKey, "Proxy password should not retrun null.");
+    assertEquals(key, sseKey, "Proxy password override did NOT work.");
   }
 
   /**
@@ -126,7 +124,7 @@ public class TestSSEConfiguration extends Assert {
    */
   private void addFileProvider(Configuration conf)
       throws Exception {
-    final File file = tempDir.newFile("test.jks");
+    final File file = tempDir.resolve("test.jks").toFile();
     final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
         file.toURI());
     conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
@@ -306,14 +304,14 @@ public class TestSSEConfiguration extends Assert {
   public void testClientEncryptionMethod() throws Throwable {
     S3AEncryptionMethods method = getMethod("CSE-KMS");
     assertEquals(CSE_KMS, method);
-    assertFalse("shouldn't be server side " + method, method.isServerSide());
+    assertFalse(method.isServerSide(), "shouldn't be server side " + method);
   }
 
   @Test
   public void testCSEKMSEncryptionMethod() throws Throwable {
     S3AEncryptionMethods method = getMethod("CSE-CUSTOM");
     assertEquals(CSE_CUSTOM, method);
-    assertFalse("shouldn't be server side " + method, method.isServerSide());
+    assertFalse(method.isServerSide(), "shouldn't be server side " + method);
   }
 
   @Test

+ 3 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.fs.s3a;
 
 import java.io.IOException;
 
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import software.amazon.awssdk.http.apache.ApacheHttpClient;
 
 import org.apache.hadoop.conf.Configuration;
@@ -54,7 +54,7 @@ public class TestWildflyAndOpenSSLBinding extends AbstractHadoopTestBase {
   /** Was wildfly found. */
   private boolean hasWildfly;
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     // determine whether or not wildfly is on the classpath
     ClassLoader loader = this.getClass().getClassLoader();

+ 4 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java

@@ -38,8 +38,8 @@ import software.amazon.awssdk.services.s3.model.GetObjectRequest;
 import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
 import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
 
-import org.junit.After;
-import org.junit.Before;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -87,7 +87,7 @@ public abstract class AbstractAuditingTest extends AbstractHadoopTestBase {
 
   private AuditManagerS3A manager;
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     requestFactory = RequestFactoryImpl.builder()
         .withBucket("bucket")
@@ -103,7 +103,7 @@ public abstract class AbstractAuditingTest extends AbstractHadoopTestBase {
    */
   protected abstract Configuration createConfig();
 
-  @After
+  @AfterEach
   public void teardown() {
     stopQuietly(manager);
   }

+ 2 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java

@@ -28,8 +28,7 @@ import software.amazon.awssdk.core.interceptor.InterceptorContext;
 import software.amazon.awssdk.http.SdkHttpMethod;
 import software.amazon.awssdk.http.SdkHttpRequest;
 import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
@@ -87,7 +86,7 @@ public class TestAuditIntegration extends AbstractHadoopTestBase {
     });
     final S3ARetryPolicy retryPolicy = new S3ARetryPolicy(new Configuration(false));
     final RetryPolicy.RetryAction action = retryPolicy.shouldRetry(ex, 0, 0, true);
-    Assertions.assertThat(action.action)
+    assertThat(action.action)
         .describedAs("retry policy %s for %s", action, ex)
         .isEqualTo(RetryPolicy.RetryAction.RetryDecision.FAIL);
   }

+ 3 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java

@@ -21,8 +21,8 @@ package org.apache.hadoop.fs.s3a.audit;
 import java.util.List;
 
 import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.store.audit.AuditSpan;
@@ -38,7 +38,7 @@ public class TestAuditSpanLifecycle extends AbstractAuditingTest {
 
   private AuditSpan resetSpan;
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     super.setup();
     resetSpan = getManager().getActiveAuditSpan();

+ 5 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java

@@ -24,10 +24,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.regex.Matcher;
 
-import org.assertj.core.api.Assertions;
 import software.amazon.awssdk.http.SdkHttpRequest;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,7 +70,7 @@ public class TestHttpReferrerAuditHeader extends AbstractAuditingTest {
 
   private LoggingAuditor auditor;
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     super.setup();
 
@@ -433,12 +432,12 @@ public class TestHttpReferrerAuditHeader extends AbstractAuditingTest {
     });
     try {
       final HttpReferrerAuditHeader referrer = ReferrerExtractor.getReferrer(auditor, span());
-      Assertions.assertThat(referrer.buildHttpReferrer())
+      assertThat(referrer.buildHttpReferrer())
           .describedAs("referrer header")
           .isBlank();
       // repeat
       LOG.info("second attempt: there should be no second warning below");
-      Assertions.assertThat(referrer.buildHttpReferrer())
+      assertThat(referrer.buildHttpReferrer())
           .describedAs("referrer header 2")
           .isBlank();
       referrer.buildHttpReferrer();

+ 3 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java

@@ -28,8 +28,8 @@ import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest;
 import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
 import software.amazon.awssdk.services.s3.model.UploadPartCopyRequest;
 import software.amazon.awssdk.transfer.s3.progress.TransferListener;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,7 +60,7 @@ public class TestLoggingAuditor extends AbstractAuditingTest {
 
   private LoggingAuditor auditor;
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     super.setup();
     auditor = (LoggingAuditor) getManager().getAuditor();

+ 16 - 16
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/impl/TestActiveAuditManagerThreadLeakage.java

@@ -31,9 +31,8 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 
-import org.assertj.core.api.Assertions;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,6 +45,7 @@ import org.apache.hadoop.test.AbstractHadoopTestBase;
 import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME;
 import static org.apache.hadoop.fs.s3a.audit.impl.ActiveAuditManagerS3A.PRUNE_THRESHOLD;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatisticsStore;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * This test attempts to recreate the OOM problems of
@@ -90,7 +90,7 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
   private final List<WeakReference<ActiveAuditManagerS3A>> auditManagers =
       new ArrayList<>();
 
-  @After
+  @AfterEach
   public void teardown() {
     if (workers != null) {
       workers.shutdown();
@@ -112,11 +112,11 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
       // get the span map
       final WeakReferenceThreadMap<?> spanMap
           = auditManager.getActiveSpanMap();
-      Assertions.assertThat(spanMap.size())
+      assertThat(spanMap.size())
           .describedAs("map size")
           .isEqualTo(1);
       auditManager.stop();
-      Assertions.assertThat(spanMap.size())
+      assertThat(spanMap.size())
           .describedAs("map size")
           .isEqualTo(0);
     }
@@ -137,14 +137,14 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
     // audit managers
     LOG.info("Total prune count {}", pruneCount);
 
-    Assertions.assertThat(pruneCount)
+    assertThat(pruneCount)
         .describedAs("Total prune count")
         .isNotZero();
 
     // now count number of audit managers GC'd
     // some must have been GC'd, showing that no other
     // references are being retained internally.
-    Assertions.assertThat(auditManagers.stream()
+    assertThat(auditManagers.stream()
             .filter((r) -> r.get() == null)
             .count())
         .describedAs("number of audit managers garbage collected")
@@ -241,7 +241,7 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
 
         // get the the span for that ID. which must never be
         // null
-        Assertions.assertThat(spanMap.get(id))
+        assertThat(spanMap.get(id))
             .describedAs("Span map entry for thread %d", id)
             .isNotNull();
 
@@ -257,7 +257,7 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
       // of entries not probed, then at least one span was
       // recreated
       if (derefenced > threadIdArray.length - subset) {
-        Assertions.assertThat(spansRecreated)
+        assertThat(spansRecreated)
             .describedAs("number of recreated spans")
             .isGreaterThan(0);
       }
@@ -268,7 +268,7 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
         LOG.info("{} executed across {} threads and pruned {} entries",
             auditManager, threadsUsed, pruned);
       }
-      Assertions.assertThat(pruned)
+      assertThat(pruned)
           .describedAs("Count of references pruned")
           .isEqualTo(derefenced - spansRecreated);
       return pruned + (int) derefenced;
@@ -293,7 +293,7 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
     auditManager.getActiveAuditSpan();
     final AuditSpanS3A auditSpan =
         auditManager.createSpan("span", null, null);
-    Assertions.assertThat(auditSpan)
+    assertThat(auditSpan)
         .describedAs("audit span for current thread")
         .isNotNull();
     // this is needed to ensure that more of the thread pool is used up
@@ -347,7 +347,7 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
         }
       }
       // pruning must have taken place
-      Assertions.assertThat(pruningCount)
+      assertThat(pruningCount)
           .describedAs("Intermittent pruning count")
           .isEqualTo(2);
     }
@@ -367,13 +367,13 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
           = auditManager.getActiveSpanMap();
       final AuditSpanS3A auditSpan =
           auditManager.createSpan("span", null, null);
-      Assertions.assertThat(auditManager.getActiveAuditSpan())
+      assertThat(auditManager.getActiveAuditSpan())
           .describedAs("active span")
           .isSameAs(auditSpan);
       // this assert gets used repeatedly, so define a lambda-exp
       // which can be envoked with different arguments
       Consumer<Boolean> assertMapHasKey = expected ->
-          Assertions.assertThat(spanMap.containsKey(spanMap.currentThreadId()))
+          assertThat(spanMap.containsKey(spanMap.currentThreadId()))
               .describedAs("map entry for current thread")
               .isEqualTo(expected);
 
@@ -385,7 +385,7 @@ public class TestActiveAuditManagerThreadLeakage extends AbstractHadoopTestBase
 
       // asking for the current span will return the unbonded one
       final AuditSpanS3A newSpan = auditManager.getActiveAuditSpan();
-      Assertions.assertThat(newSpan)
+      assertThat(newSpan)
           .describedAs("active span")
           .isNotNull()
           .matches(s -> !s.isValidSpan());

+ 4 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestIAMInstanceCredentialsProvider.java

@@ -20,13 +20,13 @@ package org.apache.hadoop.fs.s3a.auth;
 
 import java.io.IOException;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import software.amazon.awssdk.auth.credentials.AwsCredentials;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Unit tests for IAMInstanceCredentials provider.
@@ -77,7 +77,7 @@ public class TestIAMInstanceCredentialsProvider extends AbstractHadoopTestBase {
         LOG.info("Credentials: retrieved from {}: key={}",
             provider.isContainerCredentialsProvider() ? "container" : "EC2",
             credentials.accessKeyId());
-        Assertions.assertThat(credentials.accessKeyId())
+        assertThat(credentials.accessKeyId())
             .describedAs("Access key from IMDS")
             .isNotBlank();
 
@@ -88,7 +88,7 @@ public class TestIAMInstanceCredentialsProvider extends AbstractHadoopTestBase {
         LOG.info("Not running in a container/EC2");
         LOG.info("Exception raised", expected);
         // and we expect to have fallen back to InstanceProfileCredentialsProvider
-        Assertions.assertThat(provider.isContainerCredentialsProvider())
+        assertThat(provider.isContainerCredentialsProvider())
             .describedAs("%s: shoud be using InstanceProfileCredentialsProvider")
             .isFalse();
         final Throwable cause = expected.getCause();

+ 30 - 30
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java

@@ -29,9 +29,8 @@ import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
 import software.amazon.awssdk.core.signer.Signer;
 import software.amazon.awssdk.http.SdkHttpFullRequest;
 import software.amazon.awssdk.http.SdkHttpMethod;
-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;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
@@ -47,6 +46,7 @@ import org.apache.hadoop.test.AbstractHadoopTestBase;
 import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS;
 import static org.apache.hadoop.fs.s3a.auth.SignerFactory.S3_V2_SIGNER;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Tests for the SignerManager.
@@ -61,7 +61,7 @@ public class TestSignerManager extends AbstractHadoopTestBase {
   private static final String TESTUSER1 = "testuser1";
   private static final String TESTUSER2 = "testuser2";
 
-  @Before
+  @BeforeEach
   public void beforeTest() {
     SignerForTest1.reset();
     SignerForTest2.reset();
@@ -105,7 +105,7 @@ public class TestSignerManager extends AbstractHadoopTestBase {
     signerManager.initCustomSigners();
     Signer s1 = SignerFactory.createSigner("testsigner1", null);
     s1.sign(null, null);
-    Assertions.assertThat(SignerForTest1.initialized)
+    assertThat(SignerForTest1.initialized)
         .as(SignerForTest1.class.getName() + " not initialized")
         .isEqualTo(true);
   }
@@ -121,13 +121,13 @@ public class TestSignerManager extends AbstractHadoopTestBase {
     signerManager.initCustomSigners();
     Signer s1 = SignerFactory.createSigner("testsigner1", null);
     s1.sign(null, null);
-    Assertions.assertThat(SignerForTest1.initialized)
+    assertThat(SignerForTest1.initialized)
         .as(SignerForTest1.class.getName() + " not initialized")
         .isEqualTo(true);
 
     Signer s2 = SignerFactory.createSigner("testsigner2", null);
     s2.sign(null, null);
-    Assertions.assertThat(SignerForTest2.initialized)
+    assertThat(SignerForTest2.initialized)
         .as(SignerForTest2.class.getName() + " not initialized")
         .isEqualTo(true);
   }
@@ -149,18 +149,18 @@ public class TestSignerManager extends AbstractHadoopTestBase {
     SignerManager signerManager = new SignerManager("bucket1", dtProvider,
         config, ugi);
     signerManager.initCustomSigners();
-    Assertions.assertThat(SignerInitializerForTest.instanceCount)
+    assertThat(SignerInitializerForTest.instanceCount)
         .as(SignerInitializerForTest.class.getName()
             + " creation count mismatch").isEqualTo(1);
-    Assertions.assertThat(SignerInitializerForTest.registerCount)
+    assertThat(SignerInitializerForTest.registerCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(1);
-    Assertions.assertThat(SignerInitializerForTest.unregisterCount)
+    assertThat(SignerInitializerForTest.unregisterCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(0);
 
     signerManager.close();
-    Assertions.assertThat(SignerInitializerForTest.unregisterCount)
+    assertThat(SignerInitializerForTest.unregisterCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(1);
   }
@@ -187,31 +187,31 @@ public class TestSignerManager extends AbstractHadoopTestBase {
         config, ugi);
     signerManager.initCustomSigners();
 
-    Assertions.assertThat(SignerInitializerForTest.instanceCount)
+    assertThat(SignerInitializerForTest.instanceCount)
         .as(SignerInitializerForTest.class.getName()
             + " creation count mismatch").isEqualTo(1);
-    Assertions.assertThat(SignerInitializerForTest.registerCount)
+    assertThat(SignerInitializerForTest.registerCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(1);
-    Assertions.assertThat(SignerInitializerForTest.unregisterCount)
+    assertThat(SignerInitializerForTest.unregisterCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(0);
 
-    Assertions.assertThat(SignerInitializer2ForTest.instanceCount)
+    assertThat(SignerInitializer2ForTest.instanceCount)
         .as(SignerInitializer2ForTest.class.getName()
             + " creation count mismatch").isEqualTo(1);
-    Assertions.assertThat(SignerInitializer2ForTest.registerCount)
+    assertThat(SignerInitializer2ForTest.registerCount)
         .as(SignerInitializer2ForTest.class.getName()
             + " registration count mismatch").isEqualTo(1);
-    Assertions.assertThat(SignerInitializer2ForTest.unregisterCount)
+    assertThat(SignerInitializer2ForTest.unregisterCount)
         .as(SignerInitializer2ForTest.class.getName()
             + " registration count mismatch").isEqualTo(0);
 
     signerManager.close();
-    Assertions.assertThat(SignerInitializerForTest.unregisterCount)
+    assertThat(SignerInitializerForTest.unregisterCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(1);
-    Assertions.assertThat(SignerInitializer2ForTest.unregisterCount)
+    assertThat(SignerInitializer2ForTest.unregisterCount)
         .as(SignerInitializer2ForTest.class.getName()
             + " registration count mismatch").isEqualTo(1);
   }
@@ -238,13 +238,13 @@ public class TestSignerManager extends AbstractHadoopTestBase {
         SignerForInitializerTest.class, SignerInitializerForTest.class, BUCKET2,
         ugiU2);
 
-    Assertions.assertThat(SignerInitializerForTest.instanceCount)
+    assertThat(SignerInitializerForTest.instanceCount)
         .as(SignerInitializerForTest.class.getName()
             + " creation count mismatch").isEqualTo(3);
-    Assertions.assertThat(SignerInitializerForTest.registerCount)
+    assertThat(SignerInitializerForTest.registerCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(3);
-    Assertions.assertThat(SignerInitializerForTest.unregisterCount)
+    assertThat(SignerInitializerForTest.unregisterCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(0);
 
@@ -264,7 +264,7 @@ public class TestSignerManager extends AbstractHadoopTestBase {
     closeAndVerifyNull(signerManagerU2B2, BUCKET2, ugiU2, 1);
     closeAndVerifyNull(signerManagerU2B1, BUCKET1, ugiU2, 0);
 
-    Assertions.assertThat(SignerInitializerForTest.unregisterCount)
+    assertThat(SignerInitializerForTest.unregisterCount)
         .as(SignerInitializerForTest.class.getName()
             + " registration count mismatch").isEqualTo(3);
   }
@@ -284,24 +284,24 @@ public class TestSignerManager extends AbstractHadoopTestBase {
   private void verifyStoreValueInSigner(boolean expectNull, String bucketName,
       String identifier) throws IOException {
     if (expectNull) {
-      Assertions.assertThat(SignerForInitializerTest.retrievedStoreValue)
+      assertThat(SignerForInitializerTest.retrievedStoreValue)
           .as("Retrieved store value expected to be null").isNull();
     } else {
       StoreValue storeValue = SignerForInitializerTest.retrievedStoreValue;
-      Assertions.assertThat(storeValue).as("StoreValue should not be null")
+      assertThat(storeValue).as("StoreValue should not be null")
           .isNotNull();
-      Assertions.assertThat(storeValue.getBucketName())
+      assertThat(storeValue.getBucketName())
           .as("Bucket Name mismatch").isEqualTo(bucketName);
       Configuration conf = storeValue.getStoreConf();
-      Assertions.assertThat(conf).as("Configuration should not be null")
+      assertThat(conf).as("Configuration should not be null")
           .isNotNull();
-      Assertions.assertThat(conf.get(TEST_KEY_IDENTIFIER))
+      assertThat(conf.get(TEST_KEY_IDENTIFIER))
           .as("Identifier mistmatch").isEqualTo(identifier);
       Token<? extends TokenIdentifier> token = storeValue.getDtProvider()
           .getFsDelegationToken();
       String tokenId = new String(token.getIdentifier(),
           StandardCharsets.UTF_8);
-      Assertions.assertThat(tokenId)
+      assertThat(tokenId)
           .as("Mismatch in delegation token identifier").isEqualTo(
           createTokenIdentifierString(identifier, bucketName,
               UserGroupInformation.getCurrentUser().getShortUserName()));
@@ -313,7 +313,7 @@ public class TestSignerManager extends AbstractHadoopTestBase {
       throws IOException, InterruptedException {
     closeable.close();
     attemptSignAndVerify("dontcare", bucketName, ugi, true);
-    Assertions.assertThat(SignerInitializerForTest.storeCache.size())
+    assertThat(SignerInitializerForTest.storeCache.size())
         .as("StoreCache size mismatch").isEqualTo(expectedCount);
   }
 

+ 12 - 12
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestAwsClientConfig.java

@@ -21,9 +21,8 @@ package org.apache.hadoop.fs.s3a.impl;
 import java.time.Duration;
 import java.util.Arrays;
 
-import org.assertj.core.api.Assertions;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,6 +49,7 @@ import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT;
 import static org.apache.hadoop.fs.s3a.impl.AWSClientConfig.createApiConnectionSettings;
 import static org.apache.hadoop.fs.s3a.impl.AWSClientConfig.createConnectionSettings;
 import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.enforceMinimumDuration;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Unit tests for {@link AWSClientConfig}.
@@ -62,7 +62,7 @@ public class TestAwsClientConfig extends AbstractHadoopTestBase {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestAwsClientConfig.class);
 
-  @After
+  @AfterEach
   public void teardown() throws Exception {
     AWSClientConfig.resetMinimumOperationDuration();
   }
@@ -84,13 +84,13 @@ public class TestAwsClientConfig extends AbstractHadoopTestBase {
     final Duration s10 = Duration.ofSeconds(10);
     final Duration s1 = Duration.ofSeconds(1);
 
-    Assertions.assertThat(enforceMinimumDuration("key", s1, s10))
+    assertThat(enforceMinimumDuration("key", s1, s10))
         .describedAs("10s")
         .isEqualTo(s10);
 
     // and a null check
-    Assertions.assertThat(enforceMinimumDuration("key",
-           s1, null))
+    assertThat(enforceMinimumDuration("key",
+        s1, null))
         .describedAs("10s")
         .isEqualTo(s1);
   }
@@ -112,10 +112,10 @@ public class TestAwsClientConfig extends AbstractHadoopTestBase {
         conn.getEstablishTimeout());
     assertDuration(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT_DURATION,
         conn.getSocketTimeout());
-    Assertions.assertThat(conn.getMaxConnections())
+    assertThat(conn.getMaxConnections())
         .describedAs(MAXIMUM_CONNECTIONS)
         .isEqualTo(DEFAULT_MAXIMUM_CONNECTIONS);
-    Assertions.assertThat(conn.isKeepAlive())
+    assertThat(conn.isKeepAlive())
         .describedAs(CONNECTION_KEEPALIVE)
         .isEqualTo(DEFAULT_CONNECTION_KEEPALIVE);
   }
@@ -158,7 +158,7 @@ public class TestAwsClientConfig extends AbstractHadoopTestBase {
    * @param actual actual duration
    */
   private void assertDuration(String name, Duration expected, Duration actual) {
-    Assertions.assertThat(actual)
+    assertThat(actual)
         .describedAs("Duration of %s", name)
         .isEqualTo(expected);
   }
@@ -172,7 +172,7 @@ public class TestAwsClientConfig extends AbstractHadoopTestBase {
     conf.set(REQUEST_TIMEOUT, "1h");
     final AWSClientConfig.ClientSettings settings =
         createApiConnectionSettings(conf);
-    Assertions.assertThat(settings.getApiCallTimeout())
+    assertThat(settings.getApiCallTimeout())
         .describedAs("%s in %s", REQUEST_TIMEOUT, settings)
         .isEqualTo(Duration.ofHours(1));
   }
@@ -184,7 +184,7 @@ public class TestAwsClientConfig extends AbstractHadoopTestBase {
   @Test
   public void testCreateApiConnectionSettingsDefault() {
     final Configuration conf = new Configuration();
-    Assertions.assertThat(conf.get(REQUEST_TIMEOUT))
+    assertThat(conf.get(REQUEST_TIMEOUT))
         .describedAs("Request timeout %s", REQUEST_TIMEOUT)
         .isNull();
 

+ 18 - 18
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestClientManager.java

@@ -27,9 +27,8 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
-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;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import software.amazon.awssdk.services.s3.S3AsyncClient;
@@ -48,6 +47,7 @@ import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.util.functional.FunctionalIO.toUncheckedIOExceptionSupplier;
 import static org.mockito.Mockito.mock;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Test the client manager.
@@ -83,7 +83,7 @@ public class TestClientManager extends AbstractHadoopTestBase {
 
   private URI uri;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     asyncClient = mock(S3AsyncClient.class);
     transferManager = mock(S3TransferManager.class);
@@ -134,20 +134,20 @@ public class TestClientManager extends AbstractHadoopTestBase {
     final StubS3ClientFactory factory = factory(Duration.ZERO);
     final ClientManager manager = manager(factory);
 
-    Assertions.assertThat(manager.getOrCreateS3Client())
+    assertThat(manager.getOrCreateS3Client())
         .describedAs("manager %s", manager)
         .isSameAs(s3Client);
-    Assertions.assertThat(factory.clientCreationCount())
+    assertThat(factory.clientCreationCount())
         .describedAs("client creation count")
         .isEqualTo(1);
 
     // second attempt returns same instance
-    Assertions.assertThat(manager.getOrCreateS3Client())
+    assertThat(manager.getOrCreateS3Client())
         .describedAs("manager %s", manager)
         .isSameAs(s3Client);
 
     // and the factory counter is not incremented.
-    Assertions.assertThat(factory.clientCreationCount())
+    assertThat(factory.clientCreationCount())
         .describedAs("client creation count")
         .isEqualTo(1);
 
@@ -179,13 +179,13 @@ public class TestClientManager extends AbstractHadoopTestBase {
     final StubS3ClientFactory factory = factory(Duration.ofMillis(100));
     final ClientManager manager = manager(factory);
 
-    Assertions.assertThat(manager.getOrCreateAsyncClient())
+    assertThat(manager.getOrCreateAsyncClient())
         .describedAs("manager %s", manager)
         .isSameAs(asyncClient);
 
     manager.getOrCreateAsyncClient();
     // and the factory counter is not incremented.
-    Assertions.assertThat(factory.asyncClientCreationCount())
+    assertThat(factory.asyncClientCreationCount())
         .describedAs("client creation count")
         .isEqualTo(1);
 
@@ -207,15 +207,15 @@ public class TestClientManager extends AbstractHadoopTestBase {
     final StubS3ClientFactory factory = factory(Duration.ZERO);
     final ClientManager manager = manager(factory);
 
-    Assertions.assertThat(manager.getOrCreateTransferManager())
+    assertThat(manager.getOrCreateTransferManager())
         .describedAs("manager %s", manager)
         .isSameAs(transferManager);
 
     // and we created an async client
-    Assertions.assertThat(factory.asyncClientCreationCount())
+    assertThat(factory.asyncClientCreationCount())
         .describedAs("client creation count")
         .isEqualTo(1);
-    Assertions.assertThat(factory.transferManagerCreationCount())
+    assertThat(factory.transferManagerCreationCount())
         .describedAs("client creation count")
         .isEqualTo(1);
 
@@ -235,12 +235,12 @@ public class TestClientManager extends AbstractHadoopTestBase {
     final ClientManager manager = manager(factory);
 
     manager.getOrCreateAsyncClient();
-    Assertions.assertThat(manager.getOrCreateTransferManager())
+    assertThat(manager.getOrCreateTransferManager())
         .describedAs("manager %s", manager)
         .isSameAs(transferManager);
 
     // no new async client was created.
-    Assertions.assertThat(factory.asyncClientCreationCount())
+    assertThat(factory.asyncClientCreationCount())
         .describedAs("client creation count")
         .isEqualTo(1);
   }
@@ -289,12 +289,12 @@ public class TestClientManager extends AbstractHadoopTestBase {
 
     // now assert that the #1 client has succeeded, without
     // even calling futureClient.get() to evaluate the result.
-    Assertions.assertThat(threadRef.get())
+    assertThat(threadRef.get())
         .describedAs("Thread in which client #1 was created")
         .isNotSameAs(Thread.currentThread());
 
     final S3Client orig = futureClient.get();
-    Assertions.assertThat(orig)
+    assertThat(orig)
         .describedAs("second getOrCreate() call to %s", manager)
         .isSameAs(client2);
   }
@@ -340,7 +340,7 @@ public class TestClientManager extends AbstractHadoopTestBase {
 
     // now assert that the #1 client has succeeded, without
     // even calling futureClient.get() to evaluate the result.
-    Assertions.assertThat(threadRef.get())
+    assertThat(threadRef.get())
         .describedAs("Thread in which client #1 was created")
         .isNotSameAs(Thread.currentThread());
 

+ 3 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestErrorTranslation.java

@@ -26,8 +26,7 @@ import java.net.SocketTimeoutException;
 import java.net.UnknownHostException;
 import java.util.Collections;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import software.amazon.awssdk.awscore.retry.conditions.RetryOnErrorCodeCondition;
 import software.amazon.awssdk.core.exception.SdkClientException;
 import software.amazon.awssdk.core.exception.SdkException;
@@ -42,6 +41,7 @@ import org.apache.hadoop.test.AbstractHadoopTestBase;
 import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.maybeExtractIOException;
 import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.maybeProcessEncryptionClientException;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Unit tests related to the {@link ErrorTranslation} class.
@@ -178,7 +178,7 @@ public class TestErrorTranslation extends AbstractHadoopTestBase {
         .build();
     RetryOnErrorCodeCondition retry = RetryOnErrorCodeCondition.create("");
 
-    Assertions.assertThat(retry.shouldRetry(context))
+    assertThat(retry.shouldRetry(context))
         .describedAs("retry policy of MultiObjectException")
         .isFalse();
   }

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.s3a.impl;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 

+ 3 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestS3ExpressStorage.java

@@ -18,12 +18,12 @@
 
 package org.apache.hadoop.fs.s3a.impl;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.isS3ExpressStore;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Test S3 Express Storage methods.
@@ -55,7 +55,7 @@ public class TestS3ExpressStorage extends AbstractHadoopTestBase {
   }
 
   private void assertS3ExpressState(final String bucket, final boolean expected, String endpoint) {
-    Assertions.assertThat(isS3ExpressStore(bucket, endpoint))
+    assertThat(isS3ExpressStore(bucket, endpoint))
         .describedAs("isS3ExpressStore(%s) with endpoint %s", bucket, endpoint)
         .isEqualTo(expected);
   }

+ 11 - 11
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/logging/TestLogControllerFactory.java

@@ -19,10 +19,9 @@
 package org.apache.hadoop.fs.s3a.impl.logging;
 
 import org.assertj.core.api.AbstractStringAssert;
-import org.assertj.core.api.Assertions;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,6 +34,7 @@ import static org.apache.hadoop.fs.s3a.impl.logging.LogControllerFactory.createC
 import static org.apache.hadoop.fs.s3a.impl.logging.LogControllerFactory.createLog4JController;
 import static org.apache.hadoop.test.GenericTestUtils.LogCapturer.captureLogs;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Unit test for log controller factory.
@@ -73,7 +73,7 @@ public class TestLogControllerFactory extends AbstractHadoopTestBase {
   /**
    * Setup: create the contract then init it.
    */
-  @Before
+  @BeforeEach
   public void setup() {
     controller = requireNonNull(createLog4JController());
     capturer = captureLogs(LOG);
@@ -83,7 +83,7 @@ public class TestLogControllerFactory extends AbstractHadoopTestBase {
   /**
    * Teardown.
    */
-  @After
+  @AfterEach
   public void teardown() {
     if (capturer != null) {
       capturer.stopCapturing();
@@ -95,7 +95,7 @@ public class TestLogControllerFactory extends AbstractHadoopTestBase {
    */
   @Test
   public void testInstantationWrongClass() throws Throwable {
-    Assertions.assertThat(createController(CLASSNAME))
+    assertThat(createController(CLASSNAME))
         .describedAs("controller of wrong type")
         .isNull();
   }
@@ -106,7 +106,7 @@ public class TestLogControllerFactory extends AbstractHadoopTestBase {
    */
   @Test
   public void testInstantationNoClass() throws Throwable {
-    Assertions.assertThat(createController("not.a.class"))
+    assertThat(createController("not.a.class"))
         .describedAs("missing class")
         .isNull();
   }
@@ -125,7 +125,7 @@ public class TestLogControllerFactory extends AbstractHadoopTestBase {
         failing.setLevel(CLASSNAME, LogControl.LogLevel.DEBUG));
 
     // outer one doesn't
-    Assertions.assertThat(failing.setLogLevel(CLASSNAME, LogControl.LogLevel.DEBUG))
+    assertThat(failing.setLogLevel(CLASSNAME, LogControl.LogLevel.DEBUG))
         .describedAs("Invocation of setLogLevel()")
         .isFalse();
   }
@@ -174,7 +174,7 @@ public class TestLogControllerFactory extends AbstractHadoopTestBase {
     capturer.clearOutput();
     setLogLevel(level);
     logMessages();
-    return Assertions.assertThat(capturer.getOutput())
+    return assertThat(capturer.getOutput())
         .describedAs("captured output")
         .contains(contains);
   }
@@ -184,7 +184,7 @@ public class TestLogControllerFactory extends AbstractHadoopTestBase {
    * @param level level to set to.
    */
   private void setLogLevel(final LogControl.LogLevel level) {
-    Assertions.assertThat(controller.setLogLevel(CLASSNAME, level))
+    assertThat(controller.setLogLevel(CLASSNAME, level))
         .describedAs("Set log level %s", level)
         .isTrue();
   }

+ 9 - 10
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java

@@ -22,8 +22,7 @@ import java.io.IOException;
 import java.io.UncheckedIOException;
 
 import org.apache.hadoop.fs.s3a.Statistic;
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import software.amazon.awssdk.services.s3.S3AsyncClient;
 
 import org.apache.hadoop.conf.Configuration;
@@ -74,7 +73,7 @@ public class TestStreamFactories extends AbstractHadoopTestBase {
         load(INPUT_STREAM_TYPE_CLASSIC, DEFAULT_STREAM_TYPE,
             ClassicObjectInputStreamFactory.class);
     final StreamFactoryRequirements requirements = f.factoryRequirements();
-    Assertions.assertThat(requirements.requiresFuturePool())
+    assertThat(requirements.requiresFuturePool())
         .describedAs("requires future pool of %s", requirements)
         .isFalse();
     assertRequirement(requirements,
@@ -92,7 +91,7 @@ public class TestStreamFactories extends AbstractHadoopTestBase {
       final StreamFactoryRequirements requirements,
       final StreamFactoryRequirements.Requirements probe,
       final boolean shouldMatch) {
-    Assertions.assertThat(requirements.requires(probe))
+    assertThat(requirements.requires(probe))
         .describedAs("%s of %s", probe, requirements)
         .isEqualTo(shouldMatch);
   }
@@ -107,7 +106,7 @@ public class TestStreamFactories extends AbstractHadoopTestBase {
         InputStreamType.Prefetch,
         PrefetchingInputStreamFactory.class);
     final StreamFactoryRequirements requirements = f.factoryRequirements();
-    Assertions.assertThat(requirements.requiresFuturePool())
+    assertThat(requirements.requiresFuturePool())
         .describedAs("requires future pool of %s", requirements)
         .isTrue();
     assertRequirement(requirements,
@@ -146,16 +145,16 @@ public class TestStreamFactories extends AbstractHadoopTestBase {
         new StreamFactoryRequirements(1, 2, vertex);
     assertRequirement(r1, ExpectUnauditedGetRequests, false);
     assertRequirement(r1, RequiresFuturePool, false);
-    Assertions.assertThat(r1.requiresFuturePool())
+    assertThat(r1.requiresFuturePool())
         .describedAs("requiresFuturePool() %s", r1)
         .isFalse();
-    Assertions.assertThat(r1)
+    assertThat(r1)
         .describedAs("%s", r1)
         .matches(r -> !r.requiresFuturePool(), "requiresFuturePool")
         .satisfies(r ->
-            Assertions.assertThat(r.sharedThreads()).isEqualTo(1))
+            assertThat(r.sharedThreads()).isEqualTo(1))
         .satisfies(r ->
-            Assertions.assertThat(r.streamThreads()).isEqualTo(2));
+            assertThat(r.streamThreads()).isEqualTo(2));
   }
 
   @Test
@@ -166,7 +165,7 @@ public class TestStreamFactories extends AbstractHadoopTestBase {
         new StreamFactoryRequirements(1, 2, vertex, RequiresFuturePool);
     assertRequirement(r1, ExpectUnauditedGetRequests, false);
     assertRequirement(r1, RequiresFuturePool, true);
-    Assertions.assertThat(r1.requiresFuturePool())
+    assertThat(r1.requiresFuturePool())
         .describedAs("requiresFuturePool() %s", r1)
         .isTrue();
   }

+ 2 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java

@@ -22,14 +22,14 @@ package org.apache.hadoop.fs.s3a.prefetch;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.fs.impl.prefetch.BlockData;
 import org.apache.hadoop.fs.impl.prefetch.BufferData;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestS3ABlockManager extends AbstractHadoopTestBase {
 

+ 9 - 9
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java

@@ -24,7 +24,7 @@ import java.nio.ByteBuffer;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalDirAllocator;
@@ -42,7 +42,7 @@ import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_PREFETCH_MAX_BLOCKS_COU
 import static org.apache.hadoop.fs.s3a.Constants.HADOOP_TMP_DIR;
 import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Tests to perform read from S3ACachingBlockManager.
@@ -253,13 +253,13 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
     }
   }
 
-  // @Ignore
+  // @Disabled
   @Test
   public void testGet() throws Exception {
     testGetHelper(false);
   }
 
-  // @Ignore
+  // @Disabled
   @Test
   public void testGetFailure() throws Exception {
     testGetHelper(true);
@@ -300,13 +300,13 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
     }
   }
 
-  // @Ignore
+  // @Disabled
   @Test
   public void testPrefetch() throws IOException, InterruptedException {
     testPrefetchHelper(false);
   }
 
-  // @Ignore
+  // @Disabled
   @Test
   public void testPrefetchFailure() throws IOException, InterruptedException {
     testPrefetchHelper(true);
@@ -356,7 +356,7 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
             CONF.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT));
   }
 
-  // @Ignore
+  // @Disabled
   @Test
   public void testCachingOfPrefetched()
       throws IOException, InterruptedException {
@@ -392,13 +392,13 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
     assertEquals(0, this.totalErrors(blockManager));
   }
 
-  // @Ignore
+  // @Disabled
   @Test
   public void testCachingOfGet() throws IOException, InterruptedException {
     testCachingOfGetHelper(false);
   }
 
-  // @Ignore
+  // @Disabled
   @Test
   public void testCachingFailureOfGet()
       throws IOException, InterruptedException {

+ 2 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java

@@ -25,7 +25,7 @@ import java.io.InputStream;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSExceptionMessages;
@@ -39,7 +39,7 @@ import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Applies the same set of tests to both S3ACachingInputStream and S3AInMemoryInputStream.

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java

@@ -22,7 +22,7 @@ package org.apache.hadoop.fs.s3a.prefetch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts;
 import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;

+ 2 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java

@@ -21,12 +21,12 @@ package org.apache.hadoop.fs.s3a.prefetch;
 
 import java.nio.ByteBuffer;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestS3ARemoteObjectReader extends AbstractHadoopTestBase {
 

+ 5 - 5
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestMetastoreChecking.java

@@ -20,9 +20,8 @@ package org.apache.hadoop.fs.s3a.s3guard;
 
 import java.net.URI;
 
-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;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -35,6 +34,7 @@ import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
 import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.NULL_METADATA_STORE;
 import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.checkNoS3Guard;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Verify thqt the metastore checking
@@ -50,7 +50,7 @@ public class TestMetastoreChecking extends AbstractHadoopTestBase {
 
   private static final String BASE = "s3a://bucket";
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     fsUri = new URI(BASE +"/");
   }
@@ -93,7 +93,7 @@ public class TestMetastoreChecking extends AbstractHadoopTestBase {
   private void checkOutcome(final String classname, final boolean outcome) throws PathIOException {
     Configuration conf = chooseStore(classname);
 
-    Assertions.assertThat(checkNoS3Guard(fsUri, conf))
+    assertThat(checkNoS3Guard(fsUri, conf))
         .describedAs("check with classname %s", classname)
         .isEqualTo(outcome);
   }

+ 13 - 15
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/TestErrorCodeMapping.java

@@ -21,13 +21,10 @@ package org.apache.hadoop.fs.s3a.statistics;
 import java.util.Arrays;
 import java.util.Collection;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.fs.s3a.statistics.impl.StatisticsFromAwsSdkImpl;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
@@ -40,17 +37,16 @@ import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_500;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_503;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_5XX;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Test mapping logic of {@link StatisticsFromAwsSdkImpl}.
  */
-@RunWith(Parameterized.class)
 public class TestErrorCodeMapping extends AbstractHadoopTestBase {
 
   /**
    * Parameterization.
    */
-  @Parameterized.Parameters(name = "http {0} to {1}")
   public static Collection<Object[]> params() {
     return Arrays.asList(new Object[][]{
         {200, null},
@@ -65,18 +61,20 @@ public class TestErrorCodeMapping extends AbstractHadoopTestBase {
     });
   }
 
-  private final int code;
+  private int code;
 
-  private final String name;
+  private String name;
 
-  public TestErrorCodeMapping(final int code, final String name) {
-    this.code = code;
-    this.name = name;
+  public void initTestErrorCodeMapping(final int pCode, final String pName) {
+    this.code = pCode;
+    this.name = pName;
   }
 
-  @Test
-  public void testMapping() throws Throwable {
-    Assertions.assertThat(mapErrorStatusCodeToStatisticName(code))
+  @ParameterizedTest(name = "http {0} to {1}")
+  @MethodSource("params")
+  public void testMapping(int pCode, String pName) throws Throwable {
+    initTestErrorCodeMapping(pCode, pName);
+    assertThat(mapErrorStatusCodeToStatisticName(code))
         .describedAs("Mapping of status code %d", code)
         .isEqualTo(name);
   }

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/sdk/TestAWSV2SDK.java

@@ -26,7 +26,7 @@ import java.util.List;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

+ 1 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsErrorTranslation.java

@@ -22,7 +22,7 @@ import java.io.FileNotFoundException;
 import java.net.HttpURLConnection;
 import java.nio.file.AccessDeniedException;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.Path;