ソースを参照

HADOOP-19425. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-azure Part1. (#7369)

Upgrade some unit tests of the hadoop-azure module to JUnit 5.

Co-authored-by: Anuj Modi <anujmodi2011@gmail.com>
Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Co-authored-by: Steve Loughran <stevel@apache.org>
Reviewed-by: Anuj Modi <anujmodi2011@gmail.com>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Steve Loughran <stevel@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 5 ヶ月 前
コミット
744fe66c82
52 ファイル変更1088 行追加1065 行削除
  1. 39 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestName.java
  2. 5 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestBase.java
  3. 20 17
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestWithTimeout.java
  4. 4 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
  5. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIo.java
  6. 2 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureFileSystemErrorConditions.java
  7. 8 10
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobDataValidation.java
  8. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobTypeSpeedDifference.java
  9. 34 37
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlockBlobInputStream.java
  10. 17 18
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestContainerChecks.java
  11. 83 67
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionHandling.java
  12. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java
  13. 154 140
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java
  14. 5 14
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java
  15. 18 16
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestListPerformance.java
  16. 2 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthorizationCaching.java
  17. 20 16
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAppend.java
  18. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAtomicRenameDirList.java
  19. 6 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java
  20. 16 15
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java
  21. 2 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
  22. 4 5
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java
  23. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestOutOfBandAzureBlobOperationsLive.java
  24. 26 33
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestOutputStreamSemantics.java
  25. 14 18
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestPageBlobInputStream.java
  26. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestPageBlobOutputStream.java
  27. 8 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestReadAndSeekPageBlobAfterWrite.java
  28. 109 144
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbRemoteCallHelper.java
  29. 22 27
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java
  30. 19 15
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
  31. 6 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobMetadata.java
  32. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java
  33. 14 21
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestClientThrottlingAnalyzer.java
  34. 3 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestKeyPageBlobDirectories.java
  35. 266 253
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
  36. 11 12
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java
  37. 7 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrency.java
  38. 3 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemFileNameCheck.java
  39. 7 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemMocked.java
  40. 8 5
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemUploadLogic.java
  41. 6 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
  42. 2 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestShellDecryptionKeyProvider.java
  43. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestSyncableDataOutputStream.java
  44. 8 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbFsck.java
  45. 4 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AbstractAzureScaleTest.java
  46. 10 11
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java
  47. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/CleanupTestContainers.java
  48. 17 16
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/ITestAzureHugeFiles.java
  49. 58 61
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/ITestAzureFileSystemInstrumentation.java
  50. 5 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestBandwidthGaugeUpdater.java
  51. 5 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestNativeAzureFileSystemMetricsSystem.java
  52. 2 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestRollingWindowAverage.java

+ 39 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestName.java

@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.test;
+
+import org.junit.jupiter.api.extension.BeforeEachCallback;
+import org.junit.jupiter.api.extension.ExtensionContext;
+
+/**
+ * This is a custom JUnit5 `RegisterExtension`
+ * we created to obtain the methond name of the executing function.
+ */
+public class TestName implements BeforeEachCallback {
+
+  private volatile String name;
+
+  @Override
+  public void beforeEach(ExtensionContext extensionContext) throws Exception {
+    name = extensionContext.getTestMethod().get().getName();
+  }
+
+  public String getMethodName() {
+    return this.name;
+  }
+}

+ 5 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestBase.java

@@ -21,8 +21,8 @@ package org.apache.hadoop.fs.azure;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-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;
 
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
 import org.apache.hadoop.io.IOUtils;
 
-import static org.junit.Assume.assumeNotNull;
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.*;
 
 /**
@@ -49,14 +48,14 @@ public abstract class AbstractWasbTestBase extends AbstractWasbTestWithTimeout
   protected NativeAzureFileSystem fs;
   protected AzureBlobStorageTestAccount testAccount;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     AzureBlobStorageTestAccount account = createTestAccount();
-    assumeNotNull("test account", account);
+    assumeNotNull(account, "test account");
     bindToTestAccount(account);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     describe("closing test account and filesystem");
     testAccount = cleanupTestAccount(testAccount);

+ 20 - 17
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestWithTimeout.java

@@ -18,38 +18,34 @@
 
 package org.apache.hadoop.fs.azure;
 
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.rules.TestName;
-import org.junit.rules.Timeout;
-
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Timeout;
 import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.apache.hadoop.test.TestName;
+
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
 /**
  * Base class for any Wasb test with timeouts & named threads.
  * This class does not attempt to bind to Azure.
  */
-public class AbstractWasbTestWithTimeout extends Assert {
+@Timeout(AzureTestConstants.AZURE_TEST_TIMEOUT)
+public class AbstractWasbTestWithTimeout extends Assertions {
 
   /**
    * The name of the current method.
    */
-  @Rule
+  @RegisterExtension
   public TestName methodName = new TestName();
-  /**
-   * Set the timeout for every test.
-   * This is driven by the value returned by {@link #getTestTimeoutMillis()}.
-   */
-  @Rule
-  public Timeout testTimeout = new Timeout(getTestTimeoutMillis());
 
   /**
    * Name the junit thread for the class. This will overridden
    * before the individual test methods are run.
    */
-  @BeforeClass
+  @BeforeAll
   public static void nameTestThread() {
     Thread.currentThread().setName("JUnit");
   }
@@ -57,7 +53,7 @@ public class AbstractWasbTestWithTimeout extends Assert {
   /**
    * Name the thread to the current test method.
    */
-  @Before
+  @BeforeEach
   public void nameThread() {
     Thread.currentThread().setName("JUnit-" + methodName.getMethodName());
   }
@@ -70,4 +66,11 @@ public class AbstractWasbTestWithTimeout extends Assert {
     return AzureTestConstants.AZURE_TEST_TIMEOUT;
   }
 
+  public static void assumeNotNull(Object objects) {
+    assumeTrue(objects != null);
+  }
+
+  public static void assumeNotNull(Object objects, String message) {
+    assumeTrue(objects != null, message);
+  }
 }

+ 4 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.fs.azure;
 import com.microsoft.azure.storage.*;
 import com.microsoft.azure.storage.blob.*;
 import com.microsoft.azure.storage.core.Base64;
-import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,6 +49,7 @@ import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.DEFAULT_STOR
 import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_LOCAL_SAS_KEY_MODE;
 import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_SECURE_MODE;
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.verifyWasbAccountNameInConfig;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 /**
  * Helper class to create WASB file systems backed by either a mock in-memory
@@ -212,9 +212,9 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable,
    * @return
    */
   private boolean wasGeneratedByMe(MetricsRecord currentRecord) {
-    Assert.assertNotNull("null filesystem", fs);
-    Assert.assertNotNull("null filesystemn instance ID",
-        fs.getInstrumentation().getFileSystemInstanceId());
+    assertNotNull(fs, "null filesystem");
+    assertNotNull(fs.getInstrumentation().getFileSystemInstanceId(),
+        "null filesystemn instance ID");
     String myFsId = fs.getInstrumentation().getFileSystemInstanceId().toString();
     for (MetricsTag currentTag : currentRecord.tags()) {
       if (currentTag.name().equalsIgnoreCase("wasbFileSystemId")) {

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

@@ -23,7 +23,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Arrays;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 2 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureFileSystemErrorConditions.java

@@ -30,7 +30,7 @@ import java.util.concurrent.Callable;
 import com.microsoft.azure.storage.OperationContext;
 import com.microsoft.azure.storage.SendingRequestEvent;
 import com.microsoft.azure.storage.StorageEvent;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -41,7 +41,6 @@ import org.apache.hadoop.test.GenericTestUtils;
 
 import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.NO_ACCESS_TO_CONTAINER_MSG;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assume.assumeNotNull;
 
 /**
  * Error handling.
@@ -76,7 +75,7 @@ public class ITestAzureFileSystemErrorConditions extends
     try {
       FileSystem.get(noAccessPath.toUri(), new Configuration())
         .open(noAccessPath);
-      assertTrue("Should've thrown.", false);
+      assertTrue(false, "Should've thrown.");
     } catch (AzureException ex) {
       GenericTestUtils.assertExceptionContains(
           String.format(NO_ACCESS_TO_CONTAINER_MSG, account, container), ex);

+ 8 - 10
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobDataValidation.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.azure;
 
 import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_CHECK_BLOCK_MD5;
 import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_STORE_BLOB_MD5;
-import static org.junit.Assume.assumeNotNull;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
@@ -35,8 +34,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.TestHookOperationContext;
 import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
 
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
 import com.microsoft.azure.storage.Constants;
 import com.microsoft.azure.storage.OperationContext;
@@ -56,7 +55,7 @@ import com.microsoft.azure.storage.core.Base64;
 public class ITestBlobDataValidation extends AbstractWasbTestWithTimeout {
   private AzureBlobStorageTestAccount testAccount;
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     testAccount = AzureTestUtils.cleanupTestAccount(testAccount);
   }
@@ -95,8 +94,7 @@ public class ITestBlobDataValidation extends AbstractWasbTestWithTimeout {
     assumeNotNull(testAccount);
     // Write a test file.
     NativeAzureFileSystem fs = testAccount.getFileSystem();
-    Path testFilePath = AzureTestUtils.pathForTests(fs,
-        methodName.getMethodName());
+    Path testFilePath = AzureTestUtils.pathForTests(fs, methodName.getMethodName());
     String testFileKey = trim(testFilePath.toUri().getPath(), "/");
     OutputStream outStream = fs.create(testFilePath);
     outStream.write(new byte[] { 5, 15 });
@@ -109,7 +107,7 @@ public class ITestBlobDataValidation extends AbstractWasbTestWithTimeout {
     if (expectMd5Stored) {
       assertNotNull(obtainedMd5);
     } else {
-      assertNull("Expected no MD5, found: " + obtainedMd5, obtainedMd5);
+      assertNull(obtainedMd5, "Expected no MD5, found: " + obtainedMd5);
     }
 
     // Mess with the content so it doesn't match the MD5.
@@ -137,8 +135,8 @@ public class ITestBlobDataValidation extends AbstractWasbTestWithTimeout {
       }
       StorageException cause = (StorageException)ex.getCause();
       assertNotNull(cause);
-      assertEquals("Unexpected cause: " + cause,
-          StorageErrorCodeStrings.INVALID_MD5, cause.getErrorCode());
+      assertEquals(StorageErrorCodeStrings.INVALID_MD5, cause.getErrorCode(),
+          "Unexpected cause: " + cause);
     }
   }
 
@@ -192,7 +190,7 @@ public class ITestBlobDataValidation extends AbstractWasbTestWithTimeout {
       if (expectMd5) {
         assertNotNull(obtainedMd5);
       } else {
-        assertNull("Expected no MD5, found: " + obtainedMd5, obtainedMd5);
+        assertNull(obtainedMd5, "Expected no MD5, found: " + obtainedMd5);
       }
     }
 

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

@@ -23,7 +23,7 @@ import java.io.OutputStream;
 import java.util.Arrays;
 import java.util.Date;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;

+ 34 - 37
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlockBlobInputStream.java

@@ -25,9 +25,10 @@ import java.util.EnumSet;
 import java.util.Random;
 import java.util.concurrent.Callable;
 
-import org.junit.FixMethodOrder;
-import org.junit.Test;
-import org.junit.runners.MethodSorters;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestMethodOrder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,8 +45,6 @@ import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
 
-import static org.junit.Assume.assumeNotNull;
-
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 
 /**
@@ -53,7 +52,7 @@ import static org.apache.hadoop.test.LambdaTestUtils.*;
  * (KEY_INPUT_STREAM_VERSION=1) and the new
  * <code>BlockBlobInputStream</code> (KEY_INPUT_STREAM_VERSION=2).
  */
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+@TestMethodOrder(MethodOrderer.Alphanumeric.class)
 
 public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -73,6 +72,7 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
   private FileStatus testFileStatus;
   private Path hugefile;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -172,7 +172,7 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
     ContractTestUtils.assertPathExists(fs, "huge file not created", hugefile);
     FileStatus status = fs.getFileStatus(hugefile);
     ContractTestUtils.assertIsFile(hugefile, status);
-    assertTrue("File " + hugefile + " is empty", status.getLen() > 0);
+    assertTrue(status.getLen() > 0, "File " + hugefile + " is empty");
   }
 
   /**
@@ -299,12 +299,12 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
       byte[] bufferV2) throws IOException {
     int size = bufferV1.length;
     final int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, size);
-    assertEquals("Bytes read from V1 stream", size, numBytesReadV1);
+    assertEquals(size, numBytesReadV1, "Bytes read from V1 stream");
 
     final int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, size);
-    assertEquals("Bytes read from V2 stream", size, numBytesReadV2);
+    assertEquals(size, numBytesReadV2, "Bytes read from V2 stream");
 
-    assertArrayEquals("Mismatch in read data", bufferV1, bufferV2);
+    assertArrayEquals(bufferV1, bufferV2, "Mismatch in read data");
   }
 
   @Test
@@ -348,18 +348,18 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
       throws IOException {
     int size = bufferV1.length;
     int numBytesReadV1 = inputStreamV1.read(pos, bufferV1, 0, size);
-    assertEquals("Bytes read from V1 stream", size, numBytesReadV1);
+    assertEquals(size, numBytesReadV1, "Bytes read from V1 stream");
 
     int numBytesReadV2 = inputStreamV2.read(pos, bufferV2, 0, size);
-    assertEquals("Bytes read from V2 stream", size, numBytesReadV2);
+    assertEquals(size, numBytesReadV2, "Bytes read from V2 stream");
 
     int numBytesReadV2NoBuffer = inputStreamV2NoBuffer.read(pos,
         bufferV2NoBuffer, 0, size);
-    assertEquals("Bytes read from V2 stream (buffered pread disabled)", size,
-        numBytesReadV2NoBuffer);
+    assertEquals(size, numBytesReadV2NoBuffer,
+        "Bytes read from V2 stream (buffered pread disabled)");
 
-    assertArrayEquals("Mismatch in read data", bufferV1, bufferV2);
-    assertArrayEquals("Mismatch in read data", bufferV2, bufferV2NoBuffer);
+    assertArrayEquals(bufferV1, bufferV2, "Mismatch in read data");
+    assertArrayEquals(bufferV2, bufferV2NoBuffer, "Mismatch in read data");
   }
 
   /**
@@ -383,7 +383,7 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
   private void validateMarkSupported(FileSystem fs) throws IOException {
     assumeHugeFileExists();
     try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      assertTrue("mark is not supported", inputStream.markSupported());
+      assertTrue(inputStream.markSupported(), "mark is not supported");
     }
   }
 
@@ -417,7 +417,7 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
       assertEquals(buffer.length, bytesRead);
 
       inputStream.reset();
-      assertEquals("rest -> pos 0", 0, inputStream.getPos());
+      assertEquals(0, inputStream.getPos(), "rest -> pos 0");
 
       inputStream.mark(8 * KILOBYTE - 1);
 
@@ -510,11 +510,9 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
           }
       );
       long elapsedTimeMs = timer.elapsedTimeMs();
-      assertTrue(
-          String.format(
-              "There should not be any network I/O (elapsedTimeMs=%1$d).",
-              elapsedTimeMs),
-          elapsedTimeMs < 20);
+      assertTrue(elapsedTimeMs < 20, String.format(
+          "There should not be any network I/O (elapsedTimeMs=%1$d).",
+          elapsedTimeMs));
     }
   }
 
@@ -559,7 +557,7 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
           }
       );
 
-      assertTrue("Test file length only " + testFileLength, testFileLength > 0);
+      assertTrue(testFileLength > 0, "Test file length only " + testFileLength);
       inputStream.seek(testFileLength);
       assertEquals(testFileLength, inputStream.getPos());
 
@@ -576,10 +574,9 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
 
       long elapsedTimeMs = timer.elapsedTimeMs();
       assertTrue(
-          String.format(
-              "There should not be any network I/O (elapsedTimeMs=%1$d).",
-              elapsedTimeMs),
-          elapsedTimeMs < 20);
+          elapsedTimeMs < 20, String.format(
+          "There should not be any network I/O (elapsedTimeMs=%1$d).",
+          elapsedTimeMs));
     }
   }
 
@@ -770,13 +767,13 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
           (long) v2ElapsedMs,
           ratio));
     }
-    assertTrue(String.format(
+    assertTrue(
+       ratio < maxAcceptableRatio, String.format(
         "Performance of version 2 is not acceptable: v1ElapsedMs=%1$d,"
             + " v2ElapsedMs=%2$d, ratio=%3$.2f",
         (long) v1ElapsedMs,
         (long) v2ElapsedMs,
-        ratio),
-        ratio < maxAcceptableRatio);
+        ratio));
   }
 
   /**
@@ -804,14 +801,14 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
           (long) afterSeekElapsedMs,
           ratio));
     }
-    assertTrue(String.format(
+    assertTrue(
+       ratio < maxAcceptableRatio, String.format(
         "Performance of version 2 after reverse seek is not acceptable:"
             + " beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d,"
             + " ratio=%3$.2f",
         (long) beforeSeekElapsedMs,
         (long) afterSeekElapsedMs,
-        ratio),
-        ratio < maxAcceptableRatio);
+        ratio));
   }
 
   private long sequentialRead(int version,
@@ -871,13 +868,13 @@ public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
           (long) v2ElapsedMs,
           ratio));
     }
-    assertTrue(String.format(
+    assertTrue(
+       ratio < maxAcceptableRatio, String.format(
         "Performance of version 2 is not acceptable: v1ElapsedMs=%1$d,"
             + " v2ElapsedMs=%2$d, ratio=%3$.2f",
         (long) v1ElapsedMs,
         (long) v2ElapsedMs,
-        ratio),
-        ratio < maxAcceptableRatio);
+        ratio));
   }
 
   private long randomRead(int version, FileSystem fs) throws IOException {

+ 17 - 18
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestContainerChecks.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assume.assumeNotNull;
+import static org.junit.jupiter.api.Assumptions.assumeFalse;
 
 import java.io.FileNotFoundException;
 import java.util.EnumSet;
@@ -31,10 +31,9 @@ import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.CreateOptions;
 import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 
-import org.junit.After;
-import org.junit.Assume;
-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 com.microsoft.azure.storage.blob.BlobOutputStream;
 import com.microsoft.azure.storage.blob.CloudBlobContainer;
@@ -47,12 +46,12 @@ public class ITestContainerChecks extends AbstractWasbTestWithTimeout {
   private AzureBlobStorageTestAccount testAccount;
   private boolean runningInSASMode = false;
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     testAccount = AzureTestUtils.cleanup(testAccount);
   }
 
-  @Before
+  @BeforeEach
   public void setMode() {
     runningInSASMode = AzureBlobStorageTestAccount.createTestConfiguration().
         getBoolean(AzureNativeFileSystemStore.KEY_USE_SECURE_MODE, false);
@@ -72,10 +71,10 @@ public class ITestContainerChecks extends AbstractWasbTestWithTimeout {
     // state to DoesNotExist
     try {
       fs.listStatus(new Path("/"));
-      assertTrue("Should've thrown.", false);
+      assertTrue(false, "Should've thrown.");
     } catch (FileNotFoundException ex) {
-      assertTrue("Unexpected exception: " + ex,
-          ex.getMessage().contains("is not found"));
+      assertTrue(ex.getMessage().contains("is not found"),
+          "Unexpected exception: " + ex);
     }
     assertFalse(container.exists());
 
@@ -112,10 +111,10 @@ public class ITestContainerChecks extends AbstractWasbTestWithTimeout {
     // state to DoesNotExist
     try {
       assertNull(fs.listStatus(new Path("/")));
-      assertTrue("Should've thrown.", false);
+      assertTrue(false, "Should've thrown.");
     } catch (FileNotFoundException ex) {
-      assertTrue("Unexpected exception: " + ex,
-          ex.getMessage().contains("is not found"));
+      assertTrue(ex.getMessage().contains("is not found"),
+          "Unexpected exception: " + ex);
     }
     assertFalse(container.exists());
 
@@ -137,10 +136,10 @@ public class ITestContainerChecks extends AbstractWasbTestWithTimeout {
     // A list shouldn't create the container.
     try {
       fs.listStatus(new Path("/"));
-      assertTrue("Should've thrown.", false);
+      assertTrue(false, "Should've thrown.");
     } catch (FileNotFoundException ex) {
-      assertTrue("Unexpected exception: " + ex,
-          ex.getMessage().contains("is not found"));
+      assertTrue(ex.getMessage().contains("is not found"),
+          "Unexpected exception: " + ex);
     }
     assertFalse(container.exists());
 
@@ -170,7 +169,7 @@ public class ITestContainerChecks extends AbstractWasbTestWithTimeout {
   @Test
   public void testContainerChecksWithSas() throws Exception {
 
-    Assume.assumeFalse(runningInSASMode);
+    assumeFalse(runningInSASMode);
     testAccount = AzureBlobStorageTestAccount.create("",
         EnumSet.of(CreateOptions.UseSas));
     assumeNotNull(testAccount);
@@ -183,7 +182,7 @@ public class ITestContainerChecks extends AbstractWasbTestWithTimeout {
     // A write should just fail
     try {
       fs.createNewFile(new Path("/testContainerChecksWithSas-foo"));
-      assertFalse("Should've thrown.", true);
+      assertFalse(true, "Should've thrown.");
     } catch (AzureException ex) {
     }
     assertFalse(container.exists());

+ 83 - 67
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionHandling.java

@@ -28,8 +28,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import static org.apache.hadoop.fs.FSExceptionMessages.STREAM_IS_CLOSED;
 import static org.apache.hadoop.fs.azure.ExceptionHandlingTestHelper.*;
@@ -46,6 +47,7 @@ public class ITestFileSystemOperationExceptionHandling
   private Path testPath;
   private Path testFolderPath;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -81,113 +83,127 @@ public class ITestFileSystemOperationExceptionHandling
   /**
    * Tests a basic single threaded read scenario for Page blobs.
    */
-  @Test(expected=FileNotFoundException.class)
+  @Test
   public void testSingleThreadedPageBlobReadScenario() throws Throwable {
-    AzureBlobStorageTestAccount testAccount = getPageBlobTestStorageAccount();
-    setupInputStreamToTest(testAccount);
-    byte[] readBuffer = new byte[512];
-    inputStream.read(readBuffer);
+    assertThrows(FileNotFoundException.class, () -> {
+      AzureBlobStorageTestAccount testAccount = getPageBlobTestStorageAccount();
+      setupInputStreamToTest(testAccount);
+      byte[] readBuffer = new byte[512];
+      inputStream.read(readBuffer);
+    });
   }
 
   /**
    * Tests a basic single threaded seek scenario for Page blobs.
    */
-  @Test(expected=FileNotFoundException.class)
+  @Test
   public void testSingleThreadedPageBlobSeekScenario() throws Throwable {
-    AzureBlobStorageTestAccount testAccount = getPageBlobTestStorageAccount();
-    setupInputStreamToTest(testAccount);
-    inputStream.seek(5);
+    assertThrows(FileNotFoundException.class, () -> {
+      AzureBlobStorageTestAccount testAccount = getPageBlobTestStorageAccount();
+      setupInputStreamToTest(testAccount);
+      inputStream.seek(5);
+    });
   }
 
   /**
    * Test a basic single thread seek scenario for Block blobs.
    */
-  @Test(expected=FileNotFoundException.class)
+  @Test
   public void testSingleThreadBlockBlobSeekScenario() throws Throwable {
-
-    AzureBlobStorageTestAccount testAccount = createTestAccount();
-    setupInputStreamToTest(testAccount);
-    inputStream.seek(5);
-    inputStream.read();
+    assertThrows(FileNotFoundException.class, () -> {
+      AzureBlobStorageTestAccount testAccount = createTestAccount();
+      setupInputStreamToTest(testAccount);
+      inputStream.seek(5);
+      inputStream.read();
+    });
   }
 
   /**
    * Tests a basic single threaded read scenario for Block blobs.
    */
-  @Test(expected=FileNotFoundException.class)
-  public void testSingledThreadBlockBlobReadScenario() throws Throwable{
-    AzureBlobStorageTestAccount testAccount = createTestAccount();
-    setupInputStreamToTest(testAccount);
-    byte[] readBuffer = new byte[512];
-    inputStream.read(readBuffer);
+  @Test
+  public void testSingledThreadBlockBlobReadScenario() throws Throwable {
+    assertThrows(FileNotFoundException.class, () -> {
+      AzureBlobStorageTestAccount testAccount = createTestAccount();
+      setupInputStreamToTest(testAccount);
+      byte[] readBuffer = new byte[512];
+      inputStream.read(readBuffer);
+    });
   }
 
   /**
    * Tests basic single threaded setPermission scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testSingleThreadedBlockBlobSetPermissionScenario() throws Throwable {
-
-    createEmptyFile(createTestAccount(), testPath);
-    fs.delete(testPath, true);
-    fs.setPermission(testPath,
-        new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(createTestAccount(), testPath);
+      fs.delete(testPath, true);
+      fs.setPermission(testPath,
+         new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+    });
   }
 
   /**
    * Tests basic single threaded setPermission scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testSingleThreadedPageBlobSetPermissionScenario()
       throws Throwable {
-    createEmptyFile(getPageBlobTestStorageAccount(), testPath);
-    fs.delete(testPath, true);
-    fs.setOwner(testPath, "testowner", "testgroup");
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(getPageBlobTestStorageAccount(), testPath);
+      fs.delete(testPath, true);
+      fs.setOwner(testPath, "testowner", "testgroup");
+    });
   }
 
   /**
    * Tests basic single threaded setPermission scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testSingleThreadedBlockBlobSetOwnerScenario() throws Throwable {
-
-    createEmptyFile(createTestAccount(), testPath);
-    fs.delete(testPath, true);
-    fs.setOwner(testPath, "testowner", "testgroup");
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(createTestAccount(), testPath);
+      fs.delete(testPath, true);
+      fs.setOwner(testPath, "testowner", "testgroup");
+    });
   }
 
   /**
    * Tests basic single threaded setPermission scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testSingleThreadedPageBlobSetOwnerScenario() throws Throwable {
-    createEmptyFile(getPageBlobTestStorageAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    fs.setPermission(testPath,
-        new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+    assertThrows(FileNotFoundException.class, ()->{
+      createEmptyFile(getPageBlobTestStorageAccount(), testPath);
+      fs.delete(testPath, true);
+      fs.setPermission(testPath,
+          new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+    });
   }
 
   /**
    * Test basic single threaded listStatus scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testSingleThreadedBlockBlobListStatusScenario() throws Throwable {
-    createTestFolder(createTestAccount(),
-        testFolderPath);
-    fs.delete(testFolderPath, true);
-    fs.listStatus(testFolderPath);
+    assertThrows(FileNotFoundException.class, () -> {
+      createTestFolder(createTestAccount(), testFolderPath);
+      fs.delete(testFolderPath, true);
+      fs.listStatus(testFolderPath);
+    });
   }
 
   /**
    * Test basic single threaded listStatus scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testSingleThreadedPageBlobListStatusScenario() throws Throwable {
-    createTestFolder(getPageBlobTestStorageAccount(),
-        testFolderPath);
-    fs.delete(testFolderPath, true);
-    fs.listStatus(testFolderPath);
+    assertThrows(FileNotFoundException.class, () -> {
+      createTestFolder(getPageBlobTestStorageAccount(), testFolderPath);
+      fs.delete(testFolderPath, true);
+      fs.listStatus(testFolderPath);
+    });
   }
 
   /**
@@ -247,25 +263,25 @@ public class ITestFileSystemOperationExceptionHandling
   /**
    * Test basic single threaded listStatus scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testSingleThreadedBlockBlobOpenScenario() throws Throwable {
-
-    createEmptyFile(createTestAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    inputStream = fs.open(testPath);
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(createTestAccount(), testPath);
+      fs.delete(testPath, true);
+      inputStream = fs.open(testPath);
+    });
   }
 
   /**
    * Test delete then open a file.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testSingleThreadedPageBlobOpenScenario() throws Throwable {
-
-    createEmptyFile(getPageBlobTestStorageAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    inputStream = fs.open(testPath);
+    assertThrows(FileNotFoundException.class, ()->{
+      createEmptyFile(getPageBlobTestStorageAccount(), testPath);
+      fs.delete(testPath, true);
+      inputStream = fs.open(testPath);
+    });
   }
 
   /**
@@ -285,7 +301,7 @@ public class ITestFileSystemOperationExceptionHandling
     out.close();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (inputStream != null) {
       inputStream.close();

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

@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 
 import com.microsoft.azure.storage.CloudStorageAccount;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.NO_ACCESS_TO_CONTAINER_MSG;
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.verifyWasbAccountNameInConfig;

+ 154 - 140
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java

@@ -20,7 +20,8 @@ package org.apache.hadoop.fs.azure;
 
 import java.io.FileNotFoundException;
 
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -44,6 +45,7 @@ public class ITestFileSystemOperationsExceptionHandlingMultiThreaded
   private Path testPath;
   private Path testFolderPath;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -83,211 +85,222 @@ public class ITestFileSystemOperationsExceptionHandlingMultiThreaded
    * Test to validate correct exception is thrown for Multithreaded read
    * scenario for block blobs.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedBlockBlobReadScenario() throws Throwable {
-
-    AzureBlobStorageTestAccount testAccount = createTestAccount();
-    NativeAzureFileSystem fs = testAccount.getFileSystem();
-    Path base = methodPath();
-    Path testFilePath1 = new Path(base, "test1.dat");
-    Path renamePath = new Path(base, "test2.dat");
-    getInputStreamToTest(fs, testFilePath1);
-    Thread renameThread = new Thread(
-        new RenameThread(fs, testFilePath1, renamePath));
-    renameThread.start();
-
-    renameThread.join();
-
-    byte[] readBuffer = new byte[512];
-    inputStream.read(readBuffer);
+    assertThrows(FileNotFoundException.class, () -> {
+      AzureBlobStorageTestAccount testAccount = createTestAccount();
+      NativeAzureFileSystem fs = testAccount.getFileSystem();
+      Path base = methodPath();
+      Path testFilePath1 = new Path(base, "test1.dat");
+      Path renamePath = new Path(base, "test2.dat");
+      getInputStreamToTest(fs, testFilePath1);
+      Thread renameThread = new Thread(
+          new RenameThread(fs, testFilePath1, renamePath));
+      renameThread.start();
+
+      renameThread.join();
+
+      byte[] readBuffer = new byte[512];
+      inputStream.read(readBuffer);
+    });
   }
 
   /**
    * Test to validate correct exception is thrown for Multithreaded seek
    * scenario for block blobs.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadBlockBlobSeekScenario() throws Throwable {
-
-/*
-    AzureBlobStorageTestAccount testAccount = createTestAccount();
-    fs = testAccount.getFileSystem();
-*/
-    Path base = methodPath();
-    Path testFilePath1 = new Path(base, "test1.dat");
-    Path renamePath = new Path(base, "test2.dat");
-
-    getInputStreamToTest(fs, testFilePath1);
-    Thread renameThread = new Thread(
-        new RenameThread(fs, testFilePath1, renamePath));
-    renameThread.start();
-
-    renameThread.join();
-
-    inputStream.seek(5);
-    inputStream.read();
+    assertThrows(FileNotFoundException.class, () -> {
+      /*
+       * AzureBlobStorageTestAccount testAccount = createTestAccount();
+       * fs = testAccount.getFileSystem();
+       */
+      Path base = methodPath();
+      Path testFilePath1 = new Path(base, "test1.dat");
+      Path renamePath = new Path(base, "test2.dat");
+
+      getInputStreamToTest(fs, testFilePath1);
+      Thread renameThread = new Thread(
+              new RenameThread(fs, testFilePath1, renamePath));
+      renameThread.start();
+
+      renameThread.join();
+
+      inputStream.seek(5);
+      inputStream.read();
+    });
   }
 
   /**
    * Tests basic multi threaded setPermission scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedPageBlobSetPermissionScenario()
       throws Throwable {
-    createEmptyFile(
-        getPageBlobTestStorageAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(
+          getPageBlobTestStorageAccount(),
+          testPath);
+      Thread t = new Thread(new DeleteThread(fs, testPath));
+      t.start();
+      while (t.isAlive()) {
+        fs.setPermission(testPath,
+            new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+      }
       fs.setPermission(testPath,
           new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
-    }
-    fs.setPermission(testPath,
-        new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+    });
   }
 
   /**
    * Tests basic multi threaded setPermission scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedBlockBlobSetPermissionScenario()
       throws Throwable {
-    createEmptyFile(createTestAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(createTestAccount(), testPath);
+      Thread t = new Thread(new DeleteThread(fs, testPath));
+      t.start();
+      while (t.isAlive()) {
+        fs.setPermission(testPath,
+            new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+      }
       fs.setPermission(testPath,
           new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
-    }
-    fs.setPermission(testPath,
-        new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+    });
   }
 
   /**
    * Tests basic multi threaded setPermission scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedPageBlobOpenScenario() throws Throwable {
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(createTestAccount(), testPath);
+      Thread t = new Thread(new DeleteThread(fs, testPath));
+      t.start();
+      while (t.isAlive()) {
+        inputStream = fs.open(testPath);
+        inputStream.close();
+      }
 
-    createEmptyFile(createTestAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
       inputStream = fs.open(testPath);
       inputStream.close();
-    }
-
-    inputStream = fs.open(testPath);
-    inputStream.close();
+    });
   }
 
   /**
    * Tests basic multi threaded setPermission scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedBlockBlobOpenScenario() throws Throwable {
-
-    createEmptyFile(
-        getPageBlobTestStorageAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-
-    while (t.isAlive()) {
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(
+          getPageBlobTestStorageAccount(),
+          testPath);
+      Thread t = new Thread(new DeleteThread(fs, testPath));
+      t.start();
+
+      while (t.isAlive()) {
+        inputStream = fs.open(testPath);
+        inputStream.close();
+      }
       inputStream = fs.open(testPath);
       inputStream.close();
-    }
-    inputStream = fs.open(testPath);
-    inputStream.close();
+    });
   }
 
   /**
    * Tests basic multi threaded setOwner scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedBlockBlobSetOwnerScenario() throws Throwable {
-
-    createEmptyFile(createTestAccount(), testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(createTestAccount(), testPath);
+      Thread t = new Thread(new DeleteThread(fs, testPath));
+      t.start();
+      while (t.isAlive()) {
+        fs.setOwner(testPath, "testowner", "testgroup");
+      }
       fs.setOwner(testPath, "testowner", "testgroup");
-    }
-    fs.setOwner(testPath, "testowner", "testgroup");
+    });
   }
 
   /**
    * Tests basic multi threaded setOwner scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedPageBlobSetOwnerScenario() throws Throwable {
-    createEmptyFile(
-        getPageBlobTestStorageAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
+    assertThrows(FileNotFoundException.class, () -> {
+      createEmptyFile(
+          getPageBlobTestStorageAccount(),
+          testPath);
+      Thread t = new Thread(new DeleteThread(fs, testPath));
+      t.start();
+      while (t.isAlive()) {
+        fs.setOwner(testPath, "testowner", "testgroup");
+      }
       fs.setOwner(testPath, "testowner", "testgroup");
-    }
-    fs.setOwner(testPath, "testowner", "testgroup");
+    });
   }
 
   /**
    * Tests basic multi threaded listStatus scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedBlockBlobListStatusScenario() throws Throwable {
-
-    createTestFolder(createTestAccount(),
-        testFolderPath);
-    Thread t = new Thread(new DeleteThread(fs, testFolderPath));
-    t.start();
-    while (t.isAlive()) {
+    assertThrows(FileNotFoundException.class, () -> {
+      createTestFolder(createTestAccount(), testFolderPath);
+      Thread t = new Thread(new DeleteThread(fs, testFolderPath));
+      t.start();
+      while (t.isAlive()) {
+        fs.listStatus(testFolderPath);
+      }
       fs.listStatus(testFolderPath);
-    }
-    fs.listStatus(testFolderPath);
+    });
   }
 
   /**
    * Tests basic multi threaded listStatus scenario.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedPageBlobListStatusScenario() throws Throwable {
-
-    createTestFolder(
-        getPageBlobTestStorageAccount(),
-        testFolderPath);
-    Thread t = new Thread(new DeleteThread(fs, testFolderPath));
-    t.start();
-    while (t.isAlive()) {
+    assertThrows(FileNotFoundException.class, () -> {
+      createTestFolder(
+          getPageBlobTestStorageAccount(),
+          testFolderPath);
+      Thread t = new Thread(new DeleteThread(fs, testFolderPath));
+      t.start();
+      while (t.isAlive()) {
+        fs.listStatus(testFolderPath);
+      }
       fs.listStatus(testFolderPath);
-    }
-    fs.listStatus(testFolderPath);
+    });
   }
 
   /**
    * Test to validate correct exception is thrown for Multithreaded read
    * scenario for page blobs.
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedPageBlobReadScenario() throws Throwable {
-
-    bindToTestAccount(getPageBlobTestStorageAccount());
-    Path base = methodPath();
-    Path testFilePath1 = new Path(base, "test1.dat");
-    Path renamePath = new Path(base, "test2.dat");
-
-    getInputStreamToTest(fs, testFilePath1);
-    Thread renameThread = new Thread(
-        new RenameThread(fs, testFilePath1, renamePath));
-    renameThread.start();
-
-    renameThread.join();
-    byte[] readBuffer = new byte[512];
-    inputStream.read(readBuffer);
+    assertThrows(FileNotFoundException.class, () -> {
+      bindToTestAccount(getPageBlobTestStorageAccount());
+      Path base = methodPath();
+      Path testFilePath1 = new Path(base, "test1.dat");
+      Path renamePath = new Path(base, "test2.dat");
+
+      getInputStreamToTest(fs, testFilePath1);
+      Thread renameThread = new Thread(
+              new RenameThread(fs, testFilePath1, renamePath));
+      renameThread.start();
+
+      renameThread.join();
+      byte[] readBuffer = new byte[512];
+      inputStream.read(readBuffer);
+    });
   }
 
   /**
@@ -295,22 +308,23 @@ public class ITestFileSystemOperationsExceptionHandlingMultiThreaded
    * scenario for page blobs.
    */
 
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testMultiThreadedPageBlobSeekScenario() throws Throwable {
+    assertThrows(FileNotFoundException.class, () -> {
+      bindToTestAccount(getPageBlobTestStorageAccount());
 
-    bindToTestAccount(getPageBlobTestStorageAccount());
-
-    Path base = methodPath();
-    Path testFilePath1 = new Path(base, "test1.dat");
-    Path renamePath = new Path(base, "test2.dat");
+      Path base = methodPath();
+      Path testFilePath1 = new Path(base, "test1.dat");
+      Path renamePath = new Path(base, "test2.dat");
 
-    getInputStreamToTest(fs, testFilePath1);
-    Thread renameThread = new Thread(
-        new RenameThread(fs, testFilePath1, renamePath));
-    renameThread.start();
+      getInputStreamToTest(fs, testFilePath1);
+      Thread renameThread = new Thread(
+              new RenameThread(fs, testFilePath1, renamePath));
+      renameThread.start();
 
-    renameThread.join();
-    inputStream.seek(5);
+      renameThread.join();
+      inputStream.seek(5);
+    });
   }
 
 

+ 5 - 14
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java

@@ -18,10 +18,6 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.net.URI;
 import java.util.concurrent.RejectedExecutionException;
@@ -33,10 +29,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -53,10 +47,7 @@ public class ITestFileSystemOperationsWithThreads extends AbstractWasbTestBase {
   private int iterations = 1;
   private LogCapturer logs = null;
 
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
     Configuration conf = fs.getConf();
@@ -207,7 +198,7 @@ public class ITestFileSystemOperationsWithThreads extends AbstractWasbTestBase {
    * @param term search term
    */
   protected void assertInLog(String content, String term) {
-    assertTrue("Empty log", !content.isEmpty());
+    assertTrue(!content.isEmpty(), "Empty log");
     if (!content.contains(term)) {
       String message = "No " + term + " found in logs";
       LOG.error(message);
@@ -222,7 +213,7 @@ public class ITestFileSystemOperationsWithThreads extends AbstractWasbTestBase {
    * @param term search term
    */
   protected void assertNotInLog(String content, String term) {
-    assertTrue("Empty log", !content.isEmpty());
+    assertTrue(!content.isEmpty(), "Empty log");
     if (content.contains(term)) {
       String message = term + " found in logs";
       LOG.error(message);

+ 18 - 16
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestListPerformance.java

@@ -32,10 +32,10 @@ import java.util.concurrent.TimeUnit;
 
 import com.microsoft.azure.storage.blob.CloudBlobContainer;
 import com.microsoft.azure.storage.blob.CloudBlockBlob;
-import org.junit.Assume;
-import org.junit.FixMethodOrder;
-import org.junit.Test;
-import org.junit.runners.MethodSorters;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestMethodOrder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,11 +48,12 @@ import org.apache.hadoop.fs.azure.integration.AbstractAzureScaleTest;
 import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 
+import static org.junit.jupiter.api.Assumptions.assumeFalse;
+
 /**
  * Test list performance.
  */
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
-
+@TestMethodOrder(MethodOrderer.Alphanumeric.class)
 public class ITestListPerformance extends AbstractAzureScaleTest {
   private static final Logger LOG = LoggerFactory.getLogger(
       ITestListPerformance.class);
@@ -69,6 +70,7 @@ public class ITestListPerformance extends AbstractAzureScaleTest {
 
   private int expectedFileCount;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -97,7 +99,7 @@ public class ITestListPerformance extends AbstractAzureScaleTest {
 
   @Test
   public void test_0101_CreateDirectoryWithFiles() throws Exception {
-    Assume.assumeFalse("Test path exists; skipping", fs.exists(TEST_DIR_PATH));
+    assumeFalse(fs.exists(TEST_DIR_PATH), "Test path exists; skipping");
 
     ExecutorService executorService = Executors.newFixedThreadPool(threads);
     CloudBlobContainer container = testAccount.getRealContainer();
@@ -137,9 +139,9 @@ public class ITestListPerformance extends AbstractAzureScaleTest {
     LOG.info("time to create files: {} millis", elapsedMs);
 
     for (Future<Integer> future : futures) {
-      assertTrue("Future timed out", future.isDone());
-      assertEquals("Future did not write all files timed out",
-          filesPerThread, future.get().intValue());
+      assertTrue(future.isDone(), "Future timed out");
+      assertEquals(filesPerThread, future.get().intValue(),
+          "Future did not write all files timed out");
     }
   }
 
@@ -159,8 +161,8 @@ public class ITestListPerformance extends AbstractAzureScaleTest {
       LOG.info("{}: {}", fileStatus.getPath(),
           fileStatus.isDirectory() ? "dir" : "file");
     }
-    assertEquals("Mismatch between expected files and actual",
-        expectedFileCount, fileList.length);
+    assertEquals(expectedFileCount, fileList.length,
+        "Mismatch between expected files and actual");
 
 
     // now do a listFiles() recursive
@@ -174,14 +176,14 @@ public class ITestListPerformance extends AbstractAzureScaleTest {
       FileStatus fileStatus = listing.next();
       Path path = fileStatus.getPath();
       FileStatus removed = foundInList.remove(path);
-      assertNotNull("Did not find "  + path + "{} in the previous listing",
-          removed);
+      assertNotNull(removed,
+          "Did not find "  + path + "{} in the previous listing");
     }
     elapsedMs = timer.elapsedTimeMs();
     LOG.info("time for listFiles() initial call: {} millis;"
         + " time to iterate: {} millis", initialListTime, elapsedMs);
-    assertEquals("Not all files from listStatus() were found in listFiles()",
-        0, foundInList.size());
+    assertEquals(0, foundInList.size(),
+        "Not all files from listStatus() were found in listFiles()");
 
   }
 

+ 2 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthorizationCaching.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.fs.azure;
 
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.apache.hadoop.fs.azure.CachingAuthorizer.KEY_AUTH_SERVICE_CACHING_ENABLE;
 
@@ -48,6 +48,6 @@ public class ITestNativeAzureFSAuthorizationCaching
     cache.put("TEST", 1);
     cache.put("TEST", 3);
     int result = cache.get("TEST");
-    assertEquals("Cache returned unexpected result", 3, result);
+    assertEquals(3, result, "Cache returned unexpected result");
   }
 }

+ 20 - 16
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAppend.java

@@ -29,7 +29,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.test.GenericTestUtils;
 
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test append operations.
@@ -46,6 +47,7 @@ public class ITestNativeAzureFileSystemAppend extends AbstractWasbTestBase {
     return conf;
   }
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -323,28 +325,30 @@ public class ITestNativeAzureFileSystemAppend extends AbstractWasbTestBase {
     }
   }
 
-  @Test(expected=UnsupportedOperationException.class)
+  @Test
   /*
    * Test to verify the behavior when Append Support configuration flag is set to false
    */
   public void testFalseConfigurationFlagBehavior() throws Throwable {
+    assertThrows(UnsupportedOperationException.class, ()->{
+      fs = testAccount.getFileSystem();
+      Configuration conf = fs.getConf();
+      conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME, false);
+      URI uri = fs.getUri();
+      fs.initialize(uri, conf);
 
-    fs = testAccount.getFileSystem();
-    Configuration conf = fs.getConf();
-    conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME, false);
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-
-    FSDataOutputStream appendStream = null;
+      FSDataOutputStream appendStream = null;
 
-    try {
-      createBaseFileWithData(0, testPath);
-      appendStream = fs.append(testPath, 10);
-    } finally {
-      if (appendStream != null) {
-        appendStream.close();
+      try {
+        createBaseFileWithData(0, testPath);
+        appendStream = fs.append(testPath, 10);
+      } finally {
+        if (appendStream != null) {
+          appendStream.close();
+        }
       }
-    }
+
+    });
   }
 
 }

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

@@ -23,7 +23,7 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test atomic renaming.

+ 6 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -104,13 +104,13 @@ public class ITestNativeAzureFileSystemClientLogging
     performWASBOperations();
 
     String output = getLogOutput(logs);
-    assertTrue("Log entry " + TEMP_DIR + " not found  in " + output,
-        verifyStorageClientLogs(output, TEMP_DIR));
+    assertTrue(verifyStorageClientLogs(output, TEMP_DIR),
+        "Log entry " + TEMP_DIR + " not found  in " + output);
   }
 
   protected String getLogOutput(LogCapturer logs) {
     String output = logs.getOutput();
-    assertTrue("No log created/captured", !output.isEmpty());
+    assertTrue(!output.isEmpty(), "No log created/captured");
     return output;
   }
 
@@ -125,8 +125,8 @@ public class ITestNativeAzureFileSystemClientLogging
     performWASBOperations();
     String output = getLogOutput(logs);
 
-    assertFalse("Log entry " + TEMP_DIR + " found  in " + output,
-        verifyStorageClientLogs(output, TEMP_DIR));
+    assertFalse(verifyStorageClientLogs(output, TEMP_DIR),
+        "Log entry " + TEMP_DIR + " found  in " + output);
   }
 
   @Override

+ 16 - 15
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java

@@ -23,8 +23,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -40,7 +40,7 @@ public class ITestNativeAzureFileSystemConcurrencyLive
     extends AbstractWasbTestBase {
 
   private static final int THREAD_COUNT = 102;
-  private static final int TEST_EXECUTION_TIMEOUT = 30000;
+  private static final int TEST_EXECUTION_TIMEOUT = 30;
 
   @Override
   protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
@@ -53,7 +53,8 @@ public class ITestNativeAzureFileSystemConcurrencyLive
    * overwritten, even if the original destination exists but is deleted by an
    * external agent during the create operation.
    */
-  @Test(timeout = TEST_EXECUTION_TIMEOUT)
+  @Test
+  @Timeout(TEST_EXECUTION_TIMEOUT)
   public void testConcurrentCreateDeleteFile() throws Exception {
     Path testFile = methodPath();
 
@@ -71,12 +72,12 @@ public class ITestNativeAzureFileSystemConcurrencyLive
       List<Future<Void>> futures = es.invokeAll(tasks);
 
       for (Future<Void> future : futures) {
-        Assert.assertTrue(future.isDone());
+        assertTrue(future.isDone());
 
         // we are using Callable<V>, so if an exception
         // occurred during the operation, it will be thrown
         // when we call get
-        Assert.assertEquals(null, future.get());
+        assertEquals(null, future.get());
       }
     } finally {
       if (es != null) {
@@ -90,7 +91,8 @@ public class ITestNativeAzureFileSystemConcurrencyLive
    * One of the threads should successfully delete the file and return true;
    * all other threads should return false.
    */
-  @Test(timeout = TEST_EXECUTION_TIMEOUT)
+  @Test
+  @Timeout(TEST_EXECUTION_TIMEOUT)
   public void testConcurrentDeleteFile() throws Exception {
     Path testFile = new Path("test.dat");
     fs.create(testFile).close();
@@ -109,7 +111,7 @@ public class ITestNativeAzureFileSystemConcurrencyLive
 
       int successCount = 0;
       for (Future<Boolean> future : futures) {
-        Assert.assertTrue(future.isDone());
+        assertTrue(future.isDone());
 
         // we are using Callable<V>, so if an exception
         // occurred during the operation, it will be thrown
@@ -120,10 +122,8 @@ public class ITestNativeAzureFileSystemConcurrencyLive
         }
       }
 
-      Assert.assertEquals(
-          "Exactly one delete operation should return true.",
-          1,
-          successCount);
+      assertEquals(1, successCount,
+          "Exactly one delete operation should return true.");
     } finally {
       if (es != null) {
         es.shutdownNow();
@@ -139,7 +139,8 @@ public class ITestNativeAzureFileSystemConcurrencyLive
    *
    * @see <a href="https://github.com/Azure/azure-storage-java/pull/546">https://github.com/Azure/azure-storage-java/pull/546</a>
    */
-  @Test(timeout = TEST_EXECUTION_TIMEOUT)
+  @Test
+  @Timeout(TEST_EXECUTION_TIMEOUT)
   public void testConcurrentList() throws Exception {
     final Path testDir = new Path("/tmp/data-loss/11230174258112/_temporary/0/_temporary/attempt_20200624190514_0006_m_0");
     final Path testFile = new Path(testDir, "part-00004-15ea87b1-312c-4fdf-1820-95afb3dfc1c3-a010.snappy.parquet");
@@ -157,13 +158,13 @@ public class ITestNativeAzureFileSystemConcurrencyLive
       List<Future<Integer>> futures = es.invokeAll(tasks);
 
       for (Future<Integer> future : futures) {
-        Assert.assertTrue(future.isDone());
+        assertTrue(future.isDone());
 
         // we are using Callable<V>, so if an exception
         // occurred during the operation, it will be thrown
         // when we call get
         long fileCount = future.get();
-        assertEquals("The list should always contain 1 file.", 1, fileCount);
+        assertEquals(1, fileCount, "The list should always contain 1 file.");
       }
     } finally {
       if (es != null) {

+ 2 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import com.microsoft.azure.storage.StorageException;
 
@@ -299,7 +299,7 @@ public class ITestNativeAzureFileSystemLive extends
     AzureNativeFileSystemStore store = nfs.getStore();
     // Acquire the lease on the folder
     lease = store.acquireLease(fullKey);
-    assertNotNull("lease ID", lease.getLeaseID() != null);
+    assertNotNull(lease.getLeaseID() != null, "lease ID");
     // Try to create the same folder
     store.storeEmptyFolder(fullKey,
       nfs.createPermissionStatus(FsPermission.getDirDefault()));

+ 4 - 5
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java

@@ -18,20 +18,19 @@
 
 package org.apache.hadoop.fs.azure;
 
-import org.junit.FixMethodOrder;
-import org.junit.Test;
-import org.junit.runners.MethodSorters;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestMethodOrder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import static org.junit.Assume.assumeNotNull;
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.cleanupTestAccount;
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.readStringFromFile;
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.writeStringToFile;
 
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+@TestMethodOrder(MethodOrderer.Alphanumeric.class)
 /**
  * Because FileSystem.Statistics is per FileSystem, so statistics can not be ran in
  * parallel, hence in this test file, force them to run in sequential.

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

@@ -21,7 +21,7 @@ package org.apache.hadoop.fs.azure;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import com.microsoft.azure.storage.blob.BlobOutputStream;
 import com.microsoft.azure.storage.blob.CloudBlockBlob;

+ 26 - 33
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestOutputStreamSemantics.java

@@ -33,12 +33,11 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.fs.StreamCapabilities;
-import org.hamcrest.core.IsEqual;
-import org.hamcrest.core.IsNot;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertHasStreamCapabilities;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertLacksStreamCapabilities;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Test semantics of functions flush, hflush, hsync, and close for block blobs,
@@ -58,8 +57,8 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
     return buffer;
   }
 
-  private Path getBlobPathWithTestName(String parentDir) {
-    return new Path(parentDir + "/" + methodName.getMethodName());
+  private Path getBlobPathWithTestName(String parentDir, String name) {
+    return new Path(parentDir + "/" + name);
   }
 
   private void validate(Path path, byte[] writeBuffer, boolean isEqual)
@@ -71,17 +70,11 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
       int numBytesRead = inputStream.read(readBuffer, 0, readBuffer.length);
 
       if (isEqual) {
-        assertArrayEquals(
-            String.format("Bytes read do not match bytes written to %1$s",
-                blobPath),
-            writeBuffer,
-            readBuffer);
+        assertArrayEquals(writeBuffer, readBuffer,
+            String.format("Bytes read do not match bytes written to %1$s", blobPath));
       } else {
-        assertThat(
-            String.format("Bytes read unexpectedly match bytes written to %1$s",
-                blobPath),
-            readBuffer,
-            IsNot.not(IsEqual.equalTo(writeBuffer)));
+        assertThat(readBuffer).isNotEqualTo(writeBuffer).as(
+            String.format("Bytes read unexpectedly match bytes written to %1$s", blobPath));
       }
     }
   }
@@ -123,7 +116,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Verify flush writes data to storage for Page Blobs
   @Test
   public void testPageBlobFlush() throws IOException {
-    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR);
+    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR, methodName.getMethodName());
 
     try (FSDataOutputStream stream = fs.create(path)) {
       byte[] buffer = getRandomBytes();
@@ -146,7 +139,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Verify hflush writes data to storage for Page Blobs
   @Test
   public void testPageBlobHFlush() throws IOException {
-    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR);
+    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR, methodName.getMethodName());
 
     try (FSDataOutputStream stream = fs.create(path)) {
       assertTrue(isPageBlobStreamWrapper(stream));
@@ -160,7 +153,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // HSync must write data to storage for Page Blobs
   @Test
   public void testPageBlobHSync() throws IOException {
-    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR);
+    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR, methodName.getMethodName());
 
     try (FSDataOutputStream stream = fs.create(path)) {
       assertTrue(isPageBlobStreamWrapper(stream));
@@ -174,7 +167,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Close must write data to storage for Page Blobs
   @Test
   public void testPageBlobClose() throws IOException {
-    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR);
+    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR, methodName.getMethodName());
 
     try (FSDataOutputStream stream = fs.create(path)) {
       assertTrue(isPageBlobStreamWrapper(stream));
@@ -188,7 +181,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Page Blobs have StreamCapabilities.HFLUSH and StreamCapabilities.HSYNC.
   @Test
   public void testPageBlobCapabilities() throws IOException {
-    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR);
+    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR, methodName.getMethodName());
     try (FSDataOutputStream stream = fs.create(path)) {
       assertHasStreamCapabilities(stream,
           StreamCapabilities.HFLUSH,
@@ -204,7 +197,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Verify flush does not write data to storage for Block Blobs
   @Test
   public void testBlockBlobFlush() throws Exception {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR, methodName.getMethodName());
     byte[] buffer = getRandomBytes();
 
     try (FSDataOutputStream stream = fs.create(path)) {
@@ -226,7 +219,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Verify hflush does not write data to storage for Block Blobs
   @Test
   public void testBlockBlobHFlush() throws Exception {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR, methodName.getMethodName());
     byte[] buffer = getRandomBytes();
 
     try (FSDataOutputStream stream = fs.create(path)) {
@@ -248,7 +241,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Verify hsync does not write data to storage for Block Blobs
   @Test
   public void testBlockBlobHSync() throws Exception {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR, methodName.getMethodName());
     byte[] buffer = getRandomBytes();
 
     try (FSDataOutputStream stream = fs.create(path)) {
@@ -270,7 +263,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Close must write data to storage for Block Blobs
   @Test
   public void testBlockBlobClose() throws IOException {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR, methodName.getMethodName());
 
     try (FSDataOutputStream stream = fs.create(path)) {
       byte[] buffer = getRandomBytes();
@@ -283,7 +276,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Block Blobs do not have any StreamCapabilities.
   @Test
   public void testBlockBlobCapabilities() throws IOException {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR, methodName.getMethodName());
     try (FSDataOutputStream stream = fs.create(path)) {
       assertLacksStreamCapabilities(stream,
           StreamCapabilities.HFLUSH,
@@ -298,7 +291,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Verify flush writes data to storage for Block Blobs with compaction
   @Test
   public void testBlockBlobCompactionFlush() throws Exception {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR, methodName.getMethodName());
     byte[] buffer = getRandomBytes();
 
     try (FSDataOutputStream stream = fs.create(path)) {
@@ -321,7 +314,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Verify hflush writes data to storage for Block Blobs with Compaction
   @Test
   public void testBlockBlobCompactionHFlush() throws Exception {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR, methodName.getMethodName());
     byte[] buffer = getRandomBytes();
 
     try (FSDataOutputStream stream = fs.create(path)) {
@@ -344,7 +337,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Verify hsync writes data to storage for Block Blobs with compaction
   @Test
   public void testBlockBlobCompactionHSync() throws Exception {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR, methodName.getMethodName());
     byte[] buffer = getRandomBytes();
 
     try (FSDataOutputStream stream = fs.create(path)) {
@@ -367,7 +360,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Close must write data to storage for Block Blobs with compaction
   @Test
   public void testBlockBlobCompactionClose() throws IOException {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR, methodName.getMethodName());
     try (FSDataOutputStream stream = fs.create(path)) {
       assertTrue(isBlockBlobAppendStreamWrapper(stream));
       byte[] buffer = getRandomBytes();
@@ -380,7 +373,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // Block Blobs with Compaction have StreamCapabilities.HFLUSH and HSYNC.
   @Test
   public void testBlockBlobCompactionCapabilities() throws IOException {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR, methodName.getMethodName());
     try (FSDataOutputStream stream = fs.create(path)) {
       assertHasStreamCapabilities(stream,
           StreamCapabilities.HFLUSH,
@@ -396,7 +389,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // A small write does not write data to storage for Page Blobs
   @Test
   public void testPageBlobSmallWrite() throws IOException {
-    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR);
+    Path path = getBlobPathWithTestName(PAGE_BLOB_DIR, methodName.getMethodName());
     try (FSDataOutputStream stream = fs.create(path)) {
       assertTrue(isPageBlobStreamWrapper(stream));
       byte[] buffer = getRandomBytes();
@@ -408,7 +401,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // A small write does not write data to storage for Block Blobs
   @Test
   public void testBlockBlobSmallWrite() throws IOException {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_DIR, methodName.getMethodName());
     try (FSDataOutputStream stream = fs.create(path)) {
       byte[] buffer = getRandomBytes();
       stream.write(buffer);
@@ -420,7 +413,7 @@ public class ITestOutputStreamSemantics extends AbstractWasbTestBase {
   // with Compaction
   @Test
   public void testBlockBlobCompactionSmallWrite() throws IOException {
-    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR);
+    Path path = getBlobPathWithTestName(BLOCK_BLOB_COMPACTION_DIR, methodName.getMethodName());
     try (FSDataOutputStream stream = fs.create(path)) {
       assertTrue(isBlockBlobAppendStreamWrapper(stream));
       byte[] buffer = getRandomBytes();

+ 14 - 18
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestPageBlobInputStream.java

@@ -24,11 +24,11 @@ import java.io.IOException;
 import java.util.EnumSet;
 import java.util.concurrent.Callable;
 
-import org.junit.FixMethodOrder;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.junit.runners.MethodSorters;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestMethodOrder;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,8 +45,8 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 /**
  * Test semantics of the page blob input stream
  */
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
-
+@TestMethodOrder(MethodOrderer.Alphanumeric.class)
+@Timeout(600)
 public class ITestPageBlobInputStream extends AbstractWasbTestBase {
   private static final Logger LOG = LoggerFactory.getLogger(
       ITestPageBlobInputStream.class);
@@ -58,14 +58,10 @@ public class ITestPageBlobInputStream extends AbstractWasbTestBase {
 
   private long testFileLength;
 
-  /**
-   * Long test timeout.
-   */
-  @Rule
-  public Timeout testTimeout = new Timeout(10 * 60 * 1000);
   private FileStatus testFileStatus;
   private Path hugefile;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -132,7 +128,7 @@ public class ITestPageBlobInputStream extends AbstractWasbTestBase {
     ContractTestUtils.assertPathExists(fs, "huge file not created", hugefile);
     FileStatus status = fs.getFileStatus(hugefile);
     ContractTestUtils.assertIsFile(hugefile, status);
-    assertTrue("File " + hugefile + " is empty", status.getLen() > 0);
+    assertTrue(status.getLen() > 0, "File " + hugefile + " is empty");
   }
 
   @Test
@@ -246,14 +242,14 @@ public class ITestPageBlobInputStream extends AbstractWasbTestBase {
                                      long position) throws IOException {
     int size = buffer.length;
     final int numBytesRead = inputStream.read(buffer, 0, size);
-    assertEquals("Bytes read from stream", size, numBytesRead);
+    assertEquals(size, numBytesRead, "Bytes read from stream");
 
     byte[] expected = new byte[size];
     for (int i = 0; i < expected.length; i++) {
       expected[i] = (byte) ((position + i) % 256);
     }
 
-    assertArrayEquals("Mismatch", expected, buffer);
+    assertArrayEquals(expected, buffer, "Mismatch");
   }
 
   /**
@@ -264,7 +260,7 @@ public class ITestPageBlobInputStream extends AbstractWasbTestBase {
   public void test_0301_MarkSupported() throws IOException {
     assumeHugeFileExists();
     try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      assertTrue("mark is not supported", inputStream.markSupported());
+      assertTrue(inputStream.markSupported(), "mark is not supported");
     }
   }
 
@@ -284,7 +280,7 @@ public class ITestPageBlobInputStream extends AbstractWasbTestBase {
       assertEquals(buffer.length, bytesRead);
 
       inputStream.reset();
-      assertEquals("rest -> pos 0", 0, inputStream.getPos());
+      assertEquals(0, inputStream.getPos(), "rest -> pos 0");
 
       inputStream.mark(8 * KILOBYTE - 1);
 
@@ -374,7 +370,7 @@ public class ITestPageBlobInputStream extends AbstractWasbTestBase {
           }
       );
 
-      assertTrue("Test file length only " + testFileLength, testFileLength > 0);
+      assertTrue(testFileLength > 0, "Test file length only " + testFileLength);
       inputStream.seek(testFileLength);
       assertEquals(testFileLength, inputStream.getPos());
 

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

@@ -20,7 +20,7 @@ package org.apache.hadoop.fs.azure;
 import java.io.IOException;
 import java.util.EnumSet;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;

+ 8 - 7
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestReadAndSeekPageBlobAfterWrite.java

@@ -30,7 +30,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.integration.AbstractAzureScaleTest;
 import org.apache.hadoop.util.Time;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,6 +64,7 @@ public class ITestReadAndSeekPageBlobAfterWrite extends AbstractAzureScaleTest {
   // path of page blob file to read and write
   private Path blobPath;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -92,7 +94,7 @@ public class ITestReadAndSeekPageBlobAfterWrite extends AbstractAzureScaleTest {
     AzureNativeFileSystemStore store = ((NativeAzureFileSystem) fs).getStore();
     String[] a = blobPath.toUri().getPath().split("/");
     String key2 = a[1] + "/";
-    assertTrue("Not a page blob: " + blobPath, store.isPageBlobKey(key2));
+    assertTrue(store.isPageBlobKey(key2), "Not a page blob: " + blobPath);
   }
 
   /**
@@ -274,10 +276,9 @@ public class ITestReadAndSeekPageBlobAfterWrite extends AbstractAzureScaleTest {
       long end = Time.monotonicNow();
       LOG.debug("close duration = " + (end - start) + " msec.");
       if (writesSinceHFlush > 0) {
-        assertTrue(String.format(
+        assertTrue(end - start >= MINIMUM_EXPECTED_TIME, String.format(
             "close duration with >= 1 pending write is %d, less than minimum expected of %d",
-            end - start, MINIMUM_EXPECTED_TIME),
-            end - start >= MINIMUM_EXPECTED_TIME);
+            end - start, MINIMUM_EXPECTED_TIME));
         }
     }
 
@@ -332,8 +333,8 @@ public class ITestReadAndSeekPageBlobAfterWrite extends AbstractAzureScaleTest {
 
     // Verify we can list the new size. That will prove we expanded the file.
     FileStatus[] status = fs.listStatus(blobPath);
-    assertEquals("File size hasn't changed " + status,
-        numWrites * writeSize, status[0].getLen());
+    assertEquals(numWrites * writeSize, status[0].getLen(),
+        "File size hasn't changed " + status);
     LOG.debug("Total bytes written to " + blobPath + " = " + status[0].getLen());
     fs.delete(blobPath, false);
   }

+ 109 - 144
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbRemoteCallHelper.java

@@ -32,12 +32,8 @@ import org.apache.http.ParseException;
 import org.apache.http.HeaderElement;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
-import org.hamcrest.Description;
-import org.hamcrest.TypeSafeMatcher;
-import org.junit.Assume;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentMatcher;
 import org.mockito.Mockito;
 
@@ -51,6 +47,8 @@ import static org.mockito.ArgumentMatchers.argThat;
 import static org.mockito.Mockito.atLeast;
 import static org.mockito.Mockito.times;
 
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
+
 /**
  * Test class to hold all WasbRemoteCallHelper tests.
  */
@@ -67,19 +65,17 @@ public class ITestWasbRemoteCallHelper
     return AzureBlobStorageTestAccount.create(conf);
   }
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
     boolean useSecureMode = fs.getConf().getBoolean(KEY_USE_SECURE_MODE, false);
     boolean useAuthorization = fs.getConf()
         .getBoolean(NativeAzureFileSystem.KEY_AZURE_AUTHORIZATION, false);
-    Assume.assumeTrue("Test valid when both SecureMode and Authorization are enabled .. skipping",
-        useSecureMode && useAuthorization);
+    assumeTrue(useSecureMode && useAuthorization,
+        "Test valid when both SecureMode and Authorization are enabled .. skipping");
   }
 
-  @Rule
-  public ExpectedException expectedEx = ExpectedException.none();
-
   /**
    * Test invalid status-code.
    * @throws Throwable
@@ -87,18 +83,18 @@ public class ITestWasbRemoteCallHelper
   @Test // (expected = WasbAuthorizationException.class)
   public void testInvalidStatusCode() throws Throwable {
 
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any()))
-        .thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine())
-        .thenReturn(newStatusLine(INVALID_HTTP_STATUS_CODE_999));
-    // finished setting up mocks
+    assertThrows(WasbAuthorizationException.class, () -> {
+      // set up mocks
+      HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+      HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+      Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any()))
+          .thenReturn(mockHttpResponse);
+      Mockito.when(mockHttpResponse.getStatusLine())
+          .thenReturn(newStatusLine(INVALID_HTTP_STATUS_CODE_999));
+      // finished setting up mocks
 
-    performop(mockHttpClient);
+      performop(mockHttpClient);
+    });
   }
 
   /**
@@ -107,19 +103,17 @@ public class ITestWasbRemoteCallHelper
    */
   @Test // (expected = WasbAuthorizationException.class)
   public void testInvalidContentType() throws Throwable {
-
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "text/plain"));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      // set up mocks
+      HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+      HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+      Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+      Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+          .thenReturn(newHeader("Content-Type", "text/plain"));
+      // finished setting up mocks
+      performop(mockHttpClient);
+    });
   }
 
   /**
@@ -129,18 +123,18 @@ public class ITestWasbRemoteCallHelper
   @Test // (expected = WasbAuthorizationException.class)
   public void testMissingContentLength() throws Throwable {
 
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    // finished setting up mocks
+    assertThrows(WasbAuthorizationException.class, () -> {
+      // set up mocks
+      HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+      HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+      Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+      Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+          .thenReturn(newHeader("Content-Type", "application/json"));
+      // finished setting up mocks
 
-    performop(mockHttpClient);
+      performop(mockHttpClient);
+    });
   }
 
   /**
@@ -150,20 +144,19 @@ public class ITestWasbRemoteCallHelper
   @Test // (expected = WasbAuthorizationException.class)
   public void testContentLengthExceedsMax() throws Throwable {
 
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "2048"));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      // set up mocks
+      HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+      HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+      Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+      Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+          .thenReturn(newHeader("Content-Type", "application/json"));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+          .thenReturn(newHeader("Content-Length", "2048"));
+      // finished setting up mocks
+      performop(mockHttpClient);
+    });
   }
 
   /**
@@ -173,20 +166,20 @@ public class ITestWasbRemoteCallHelper
   @Test // (expected = WasbAuthorizationException.class)
   public void testInvalidContentLengthValue() throws Throwable {
 
-    setupExpectations();
+    assertThrows(WasbAuthorizationException.class, () -> {
+      // set up mocks
+      HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+      HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+      Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+      Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+          .thenReturn(newHeader("Content-Type", "application/json"));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+          .thenReturn(newHeader("Content-Length", "20abc48"));
+      // finished setting up mocks
 
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "20abc48"));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
+      performop(mockHttpClient);
+    });
   }
 
   /**
@@ -225,27 +218,29 @@ public class ITestWasbRemoteCallHelper
   @Test // (expected = WasbAuthorizationException.class)
   public void testMalFormedJSONResponse() throws Throwable {
 
-    expectedEx.expect(WasbAuthorizationException.class);
-    expectedEx.expectMessage("com.fasterxml.jackson.core.JsonParseException: Unexpected end-of-input in FIELD_NAME");
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
+    String errorMsg =
+        "com.fasterxml.jackson.core.JsonParseException: Unexpected end-of-input in FIELD_NAME";
+    assertThrows(WasbAuthorizationException.class, () -> {
+      // set up mocks
+      HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+
+      HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+      HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
+
+      Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+      Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+          .thenReturn(newHeader("Content-Type", "application/json"));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+          .thenReturn(newHeader("Content-Length", "1024"));
+      Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
+      Mockito.when(mockHttpEntity.getContent())
+          .thenReturn(new ByteArrayInputStream(malformedJsonResponse().getBytes(StandardCharsets.UTF_8)));
+      // finished setting up mocks
 
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
-    Mockito.when(mockHttpEntity.getContent())
-        .thenReturn(new ByteArrayInputStream(malformedJsonResponse().getBytes(StandardCharsets.UTF_8)));
-    // finished setting up mocks
+      performop(mockHttpClient);
+    }, errorMsg);
 
-    performop(mockHttpClient);
   }
 
   /**
@@ -254,28 +249,28 @@ public class ITestWasbRemoteCallHelper
    */
   @Test // (expected = WasbAuthorizationException.class)
   public void testFailureCodeJSONResponse() throws Throwable {
-
-    expectedEx.expect(WasbAuthorizationException.class);
-    expectedEx.expectMessage("Remote authorization service encountered an error Unauthorized");
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
-
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
-    Mockito.when(mockHttpEntity.getContent())
-        .thenReturn(new ByteArrayInputStream(failureCodeJsonResponse().getBytes(StandardCharsets.UTF_8)));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
+    String errorMsg = "Remote authorization service encountered an error Unauthorized";
+
+    assertThrows(WasbAuthorizationException.class, () -> {
+      // set up mocks
+      HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+
+      HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+      HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
+
+      Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+      Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+          .thenReturn(newHeader("Content-Type", "application/json"));
+      Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+          .thenReturn(newHeader("Content-Length", "1024"));
+      Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
+      Mockito.when(mockHttpEntity.getContent())
+          .thenReturn(new ByteArrayInputStream(failureCodeJsonResponse()
+          .getBytes(StandardCharsets.UTF_8)));
+      // finished setting up mocks
+      performop(mockHttpClient);
+    }, errorMsg);
   }
 
   @Test
@@ -410,36 +405,6 @@ public class ITestWasbRemoteCallHelper
     }
   }
 
-  private void setupExpectations() {
-    expectedEx.expect(WasbAuthorizationException.class);
-
-    class MatchesPattern extends TypeSafeMatcher<String> {
-      private String pattern;
-
-      MatchesPattern(String pattern) {
-        this.pattern = pattern;
-      }
-
-      @Override protected boolean matchesSafely(String item) {
-        return item.matches(pattern);
-      }
-
-      @Override public void describeTo(Description description) {
-        description.appendText("matches pattern ").appendValue(pattern);
-      }
-
-      @Override protected void describeMismatchSafely(String item,
-          Description mismatchDescription) {
-        mismatchDescription.appendText("does not match");
-      }
-    }
-
-    expectedEx.expectMessage(new MatchesPattern(
-        "org\\.apache\\.hadoop\\.fs\\.azure\\.WasbRemoteCallException: "
-            + "Encountered error while making remote call to "
-            + "http:\\/\\/localhost1\\/,http:\\/\\/localhost2\\/,http:\\/\\/localhost:8080 retried 6 time\\(s\\)\\."));
-  }
-
   private void performop(HttpClient mockHttpClient) throws Throwable {
 
     Path testPath = new Path("/", "test.dat");

+ 22 - 27
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java

@@ -21,8 +21,7 @@ package org.apache.hadoop.fs.azure;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.fs.azure.NativeAzureFileSystem.RETURN_URI_AS_CANONICAL_SERVICE_NAME_PROPERTY_NAME;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assume.assumeFalse;
-import static org.junit.Assume.assumeNotNull;
+import static org.junit.jupiter.api.Assumptions.assumeFalse;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
@@ -48,13 +47,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.CreateOptions;
 import org.apache.hadoop.test.GenericTestUtils;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 
 import com.microsoft.azure.storage.StorageException;
 import com.microsoft.azure.storage.blob.CloudBlobContainer;
@@ -70,17 +66,15 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
   protected String accountKey;
   protected static Configuration conf = null;
   private boolean runningInSASMode = false;
-  @Rule
-  public final TemporaryFolder tempDir = new TemporaryFolder();
 
   private AzureBlobStorageTestAccount testAccount;
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     testAccount = AzureTestUtils.cleanupTestAccount(testAccount);
   }
 
-  @Before
+  @BeforeEach
   public void setMode() {
     runningInSASMode = AzureBlobStorageTestAccount.createTestConfiguration().
         getBoolean(AzureNativeFileSystemStore.KEY_USE_SECURE_MODE, false);
@@ -138,7 +132,7 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
   @Test
   public void testConnectUsingSAS() throws Exception {
 
-    Assume.assumeFalse(runningInSASMode);
+    assumeFalse(runningInSASMode);
     // Create the test account with SAS credentials.
     testAccount = AzureBlobStorageTestAccount.create("",
         EnumSet.of(CreateOptions.UseSas, CreateOptions.CreateContainer));
@@ -154,7 +148,7 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
   @Test
   public void testConnectUsingSASReadonly() throws Exception {
 
-    Assume.assumeFalse(runningInSASMode);
+    assumeFalse(runningInSASMode);
     // Create the test account with SAS credentials.
     testAccount = AzureBlobStorageTestAccount.create("", EnumSet.of(
         CreateOptions.UseSas, CreateOptions.CreateContainer,
@@ -303,7 +297,7 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
     } catch (Exception e) {
       String errMsg = String.format(
           "Expected AzureException but got %s instead.", e);
-      assertTrue(errMsg, false);
+      assertTrue(false, errMsg);
     }
   }
 
@@ -336,11 +330,11 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
       int expectedValue) throws Exception {
     InputStream inputStream = fs.open(testFile);
     int byteRead = inputStream.read();
-    assertTrue("File unexpectedly empty: " + testFile, byteRead >= 0);
-    assertTrue("File has more than a single byte: " + testFile,
-        inputStream.read() < 0);
+    assertTrue(byteRead >= 0, "File unexpectedly empty: " + testFile);
+    assertTrue(
+       inputStream.read() < 0, "File has more than a single byte: " + testFile);
     inputStream.close();
-    assertEquals("Unxpected content in: " + testFile, expectedValue, byteRead);
+    assertEquals(expectedValue, byteRead, "Unxpected content in: " + testFile);
   }
 
   @Test
@@ -381,14 +375,15 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
   }
 
   @Test
-  public void testCredsFromCredentialProvider() throws Exception {
+  public void testCredsFromCredentialProvider(@TempDir java.nio.file.Path tempDir)
+      throws Exception {
 
     assumeFalse(runningInSASMode);
     String account = "testacct";
     String key = "testkey";
     // set up conf to have a cred provider
     final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
+    final File file = new File(tempDir.toFile(), "myfile.txt");
     final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
         file.toURI());
     conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
@@ -403,7 +398,7 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
     String result = AzureNativeFileSystemStore.getAccountKeyFromConfiguration(
         account, conf);
     // result should contain the credential provider key not the config key
-    assertEquals("AccountKey incorrect.", key, result);
+    assertEquals(key, result, "AccountKey incorrect.");
   }
 
   void provisionAccountKey(
@@ -439,7 +434,7 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
         "org.apache.Nonexistant.Class");
     try {
       AzureNativeFileSystemStore.getAccountKeyFromConfiguration(account, conf);
-      Assert.fail("Nonexistant key provider class should have thrown a "
+      fail("Nonexistant key provider class should have thrown a "
           + "KeyProviderException");
     } catch (KeyProviderException e) {
     }
@@ -453,7 +448,7 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
     conf.set("fs.azure.account.keyprovider." + account, "java.lang.String");
     try {
       AzureNativeFileSystemStore.getAccountKeyFromConfiguration(account, conf);
-      Assert.fail("Key provider class that doesn't implement KeyProvider "
+      fail("Key provider class that doesn't implement KeyProvider "
           + "should have thrown a KeyProviderException");
     } catch (KeyProviderException e) {
     }
@@ -659,8 +654,8 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
 
       conf.setBoolean(RETURN_URI_AS_CANONICAL_SERVICE_NAME_PROPERTY_NAME, true);
       FileSystem fs1 = FileSystem.newInstance(defaultUri, conf);
-      Assert.assertEquals("getCanonicalServiceName() should return URI",
-              fs1.getUri().toString(), fs1.getCanonicalServiceName());
+      assertEquals(fs1.getUri().toString(), fs1.getCanonicalServiceName(),
+          "getCanonicalServiceName() should return URI");
     } finally {
       testAccount.cleanup();
       FileSystem.closeAll();

+ 19 - 15
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java

@@ -41,7 +41,8 @@ import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending;
 
 import com.microsoft.azure.storage.AccessCondition;
@@ -76,6 +77,7 @@ public abstract class NativeAzureFileSystemBaseTest
   public static final Logger LOG = LoggerFactory.getLogger(NativeAzureFileSystemBaseTest.class);
   protected NativeAzureFileSystem fs;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -495,7 +497,7 @@ public abstract class NativeAzureFileSystemBaseTest
     assertTrue(fs.mkdirs(dir));
     try {
       fs.open(dir).close();
-      assertTrue("Should've thrown", false);
+      assertTrue(false, "Should've thrown");
     } catch (FileNotFoundException ex) {
       assertExceptionContains("a directory not a file.", ex);
     }
@@ -507,7 +509,7 @@ public abstract class NativeAzureFileSystemBaseTest
     assertTrue(fs.mkdirs(dir));
     try {
       fs.create(dir).close();
-      assertTrue("Should've thrown", false);
+      assertTrue(false, "Should've thrown");
     } catch (IOException ex) {
       assertExceptionContains("Cannot create file", ex);
       assertExceptionContains("already exists as a directory", ex);
@@ -527,7 +529,8 @@ public abstract class NativeAzureFileSystemBaseTest
   }
 
   @Test
-  public void testInputStreamReadWithBufferReturnsMinusOneOnEof() throws Exception {
+  public void testInputStreamReadWithBufferReturnsMinusOneOnEof()
+      throws Exception {
     Path newFile = methodPath();
     OutputStream output = fs.create(newFile);
     output.write(10);
@@ -549,7 +552,8 @@ public abstract class NativeAzureFileSystemBaseTest
   }
 
   @Test
-  public void testInputStreamReadWithBufferReturnsMinusOneOnEofForLargeBuffer() throws Exception {
+  public void testInputStreamReadWithBufferReturnsMinusOneOnEofForLargeBuffer()
+      throws Exception {
     Path newFile = methodPath();
     OutputStream output = fs.create(newFile);
     byte[] outputBuff = new byte[97331];
@@ -1051,7 +1055,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
     // Make sure rename pending file is gone.
     FileStatus[] listed = fs.listStatus(new Path("/"));
-    assertEquals("Pending directory still found", 1, listed.length);
+    assertEquals(1, listed.length, "Pending directory still found");
     assertTrue(listed[0].isDirectory());
   }
 
@@ -1348,7 +1352,7 @@ public abstract class NativeAzureFileSystemBaseTest
           assertTrue(fs.createNewFile(makePath(prefix, name)));
         }
       } else {
-        assertTrue("The object must be a (leaf) file or a folder.", false);
+        assertTrue(false, "The object must be a (leaf) file or a folder.");
       }
     }
 
@@ -1506,7 +1510,7 @@ public abstract class NativeAzureFileSystemBaseTest
     Path testFile = new Path(testFolder, "testFile");
     try {
       fs.createNonRecursive(testFile, true, 1024, (short)1, 1024, null);
-      assertTrue("Should've thrown", false);
+      assertTrue(false, "Should've thrown");
     } catch (FileNotFoundException e) {
     }
     fs.mkdirs(testFolder);
@@ -1530,11 +1534,11 @@ public abstract class NativeAzureFileSystemBaseTest
     long currentUtcTime = utc.getTime().getTime();
     FileStatus fileStatus = fs.getFileStatus(testPath);
     final long errorMargin = 60 * 1000; // Give it +/-60 seconds
-    assertTrue("Modification time " +
-        new Date(fileStatus.getModificationTime()) + " is not close to now: " +
-        utc.getTime(),
+    assertTrue(
         fileStatus.getModificationTime() > (currentUtcTime - errorMargin) &&
-        fileStatus.getModificationTime() < (currentUtcTime + errorMargin));
+        fileStatus.getModificationTime() < (currentUtcTime + errorMargin),
+        "Modification time " + new Date(fileStatus.getModificationTime())
+        + " is not close to now: " + utc.getTime());
   }
 
   private void createEmptyFile(Path testFile, FsPermission permission)
@@ -1675,7 +1679,7 @@ public abstract class NativeAzureFileSystemBaseTest
           lease = nfs.getStore().acquireLease(key);
           LOG.info(name + " acquired lease " + lease.getLeaseID());
         } catch (AzureException e) {
-          assertTrue("Unanticipated exception", false);
+          assertTrue(false, "Unanticipated exception");
         }
         assertTrue(lease != null);
         try {
@@ -1706,14 +1710,14 @@ public abstract class NativeAzureFileSystemBaseTest
           secondStartTime = System.currentTimeMillis();
           LOG.info(name + " acquired lease " + lease.getLeaseID());
         } catch (AzureException e) {
-          assertTrue("Unanticipated exception", false);
+          assertTrue(false, "Unanticipated exception");
         }
         assertTrue(lease != null);
         try {
           lease.free();
           LOG.info(name + " freed lease " + lease.getLeaseID());
         } catch (StorageException e) {
-          assertTrue("Unanticipated exception", false);
+          assertTrue(false, "Unanticipated exception");
         }
       } else {
         fail("Unknown thread name");

+ 6 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobMetadata.java

@@ -30,9 +30,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
-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;
 
 /**
  * Tests that we put the correct metadata on blobs created through WASB.
@@ -42,14 +42,14 @@ public class TestBlobMetadata extends AbstractWasbTestWithTimeout {
   private FileSystem fs;
   private InMemoryBlockBlobStore backingStore;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     testAccount = AzureBlobStorageTestAccount.createMock();
     fs = testAccount.getFileSystem();
     backingStore = testAccount.getMockStorage().getBackingStore();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     testAccount.cleanup();
     fs = null;
@@ -203,7 +203,7 @@ public class TestBlobMetadata extends AbstractWasbTestWithTimeout {
     fs.create(selfishFile, justMe, true, 4096, fs.getDefaultReplication(),
         fs.getDefaultBlockSize(), null).close();
     String mockUri = AzureBlobStorageTestAccount.toMockUri(selfishFile);
-    assertNotNull("converted URI", mockUri);
+    assertNotNull(mockUri, "converted URI");
     HashMap<String, String> metadata = backingStore
         .getMetadata(mockUri);
     assertNotNull(metadata);

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

@@ -29,7 +29,7 @@ import com.microsoft.azure.storage.blob.CloudBlobContainer;
 import com.microsoft.azure.storage.blob.CloudBlockBlob;
 import com.microsoft.azure.storage.blob.CloudPageBlob;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.net.HttpURLConnection;
 import java.nio.charset.StandardCharsets;

+ 14 - 21
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestClientThrottlingAnalyzer.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.fs.azure;
 
 import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests for <code>ClientThrottlingAnalyzer</code>.
@@ -43,32 +43,25 @@ public class TestClientThrottlingAnalyzer extends AbstractWasbTestWithTimeout {
     final double lowerBound = Math.max(expected - percentage / 100 * expected, 0);
     final double upperBound = expected + percentage / 100 * expected;
 
-    assertTrue(
-        String.format(
-            "The actual value %1$d is not within the expected range: "
-                + "[%2$.2f, %3$.2f].",
-            actual,
-            lowerBound,
-            upperBound),
-        actual >= lowerBound && actual <= upperBound);
+    assertTrue(actual >= lowerBound && actual <= upperBound, String.format(
+        "The actual value %1$d is not within the expected range: "
+        + "[%2$.2f, %3$.2f].",
+        actual,
+        lowerBound,
+        upperBound));
   }
 
   private void validate(long expected, long actual) {
-    assertEquals(
-        String.format("The actual value %1$d is not the expected value %2$d.",
-            actual,
-            expected),
-        expected, actual);
+    assertEquals(expected, actual,
+       String.format("The actual value %1$d is not the expected value %2$d.", actual, expected));
   }
 
   private void validateLessThanOrEqual(long maxExpected, long actual) {
-    assertTrue(
-        String.format(
-            "The actual value %1$d is not less than or equal to the maximum"
-            + " expected value %2$d.",
-            actual,
-            maxExpected),
-        actual < maxExpected);
+    assertTrue(actual < maxExpected, String.format(
+        "The actual value %1$d is not less than or equal to the maximum"
+        + " expected value %2$d.",
+        actual,
+        maxExpected));
   }
 
   /**

+ 3 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestKeyPageBlobDirectories.java

@@ -23,7 +23,7 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test config property KEY_PAGE_BLOB_DIRECTORIES.
@@ -36,9 +36,8 @@ public class TestKeyPageBlobDirectories extends AbstractWasbTestBase{
   }
 
   public void expectPageBlobKey(boolean expectedOutcome, AzureNativeFileSystemStore store, String path) {
-    assertEquals("Unexpected result for isPageBlobKey(" + path + ")",
-            expectedOutcome, store.isPageBlobKey(path));
-
+    assertEquals(expectedOutcome, store.isPageBlobKey(path),
+        "Unexpected result for isPageBlobKey(" + path + ")");
   }
 
   @Test

+ 266 - 253
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java

@@ -40,17 +40,15 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.StringUtils;
 
-import org.junit.Assume;
-import org.junit.Rule;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
-import org.junit.rules.ExpectedException;
 import org.apache.hadoop.classification.VisibleForTesting;
 
 import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_SECURE_MODE;
 import static org.apache.hadoop.fs.azure.CachingAuthorizer.KEY_AUTH_SERVICE_CACHING_ENABLE;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
 /**
  * Test class to hold all WASB authorization tests.
@@ -89,21 +87,19 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   @Override
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
     boolean useSecureMode = fs.getConf().getBoolean(KEY_USE_SECURE_MODE, false);
     boolean useAuthorization = fs.getConf().getBoolean(NativeAzureFileSystem.KEY_AZURE_AUTHORIZATION, false);
-    Assume.assumeTrue("Test valid when both SecureMode and Authorization are enabled .. skipping",
-        useSecureMode && useAuthorization);
+    assumeTrue((useSecureMode && useAuthorization),
+        "Test valid when both SecureMode and Authorization are enabled .. skipping");
 
     authorizer = new MockWasbAuthorizerImpl(fs);
     authorizer.init(fs.getConf());
     fs.updateWasbAuthorizer(authorizer);
   }
 
-  @Rule
-  public ExpectedException expectedEx = ExpectedException.none();
-
   /**
    * Setup up permissions to allow a recursive delete for cleanup purposes.
    */
@@ -123,10 +119,9 @@ public class TestNativeAzureFileSystemAuthorization
   /**
    * Setup the expected exception class, and exception message that the test is supposed to fail with.
    */
-  protected void setExpectedFailureMessage(String operation, Path path) {
-    expectedEx.expect(WasbAuthorizationException.class);
-    expectedEx.expectMessage(String.format("%s operation for Path : %s not allowed",
-        operation, path.makeQualified(fs.getUri(), fs.getWorkingDirectory())));
+  protected String setExpectedFailureMessage(String operation, Path path) {
+    return String.format("%s operation for Path : %s not allowed",
+        operation, path.makeQualified(fs.getUri(), fs.getWorkingDirectory()));
   }
 
   /**
@@ -198,19 +193,19 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/");
     Path testPath = new Path(parentDir, "test.dat");
 
-    setExpectedFailureMessage("create", testPath);
-
-    authorizer.addAuthRuleForOwner("/", WRITE, true);
-    fs.updateWasbAuthorizer(authorizer);
+    String errorMsg = setExpectedFailureMessage("create", testPath);
 
-    try {
-      fs.create(testPath);
-      ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
-      fs.create(testPath, true);
-    }
-    finally {
-      fs.delete(testPath, false);
-    }
+    assertThrows(WasbAuthorizationException.class, () -> {
+      authorizer.addAuthRuleForOwner("/", WRITE, true);
+      fs.updateWasbAuthorizer(authorizer);
+      try {
+        fs.create(testPath);
+        ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
+        fs.create(testPath, true);
+      } finally {
+        fs.delete(testPath, false);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -249,19 +244,21 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/testCreateAccessCheckNegative");
     Path testPath = new Path(parentDir, "test.dat");
 
-    setExpectedFailureMessage("create", testPath);
+    String errorMsg = setExpectedFailureMessage("create", testPath);
 
-    authorizer.addAuthRuleForOwner("/", WRITE, false);
-    fs.updateWasbAuthorizer(authorizer);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      authorizer.addAuthRuleForOwner("/", WRITE, false);
+      fs.updateWasbAuthorizer(authorizer);
 
-    try {
-      fs.create(testPath);
-    }
-    finally {
-      /* Provide permissions to cleanup in case the file got created */
-      allowRecursiveDelete(fs, parentDir.toString());
-      fs.delete(parentDir, true);
-    }
+      try {
+        fs.create(testPath);
+      }
+      finally {
+        /* Provide permissions to cleanup in case the file got created */
+        allowRecursiveDelete(fs, parentDir.toString());
+        fs.delete(parentDir, true);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -300,20 +297,19 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/testListAccessCheckNegative");
     Path testPath = new Path(parentDir, "test.dat");
 
-    setExpectedFailureMessage("liststatus", testPath);
-
-    authorizer.addAuthRuleForOwner("/", WRITE, true);
-    authorizer.addAuthRuleForOwner(testPath.toString(), READ, false);
-    fs.updateWasbAuthorizer(authorizer);
-
-    try {
-      fs.create(testPath);
-      fs.listStatus(testPath);
-    }
-    finally {
-      allowRecursiveDelete(fs, parentDir.toString());
-      fs.delete(parentDir, true);
-    }
+    String errorMsg = setExpectedFailureMessage("liststatus", testPath);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      authorizer.addAuthRuleForOwner("/", WRITE, true);
+      authorizer.addAuthRuleForOwner(testPath.toString(), READ, false);
+      fs.updateWasbAuthorizer(authorizer);
+      try {
+        fs.create(testPath);
+        fs.listStatus(testPath);
+      } finally {
+        allowRecursiveDelete(fs, parentDir.toString());
+        fs.delete(parentDir, true);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -356,24 +352,26 @@ public class TestNativeAzureFileSystemAuthorization
     Path srcPath = new Path(parentDir, "test1.dat");
     Path dstPath = new Path(parentDir, "test2.dat");
 
-    setExpectedFailureMessage("rename", srcPath);
+    String errorMsg = setExpectedFailureMessage("rename", srcPath);
 
-    /* to create parent dir */
-    authorizer.addAuthRuleForOwner("/", WRITE, true);
-    authorizer.addAuthRuleForOwner(parentDir.toString(), WRITE, false);
-    fs.updateWasbAuthorizer(authorizer);
-
-    try {
-      fs.create(srcPath);
-      ContractTestUtils.assertPathExists(fs, "sourcePath does not exist", srcPath);
-      fs.rename(srcPath, dstPath);
-      ContractTestUtils.assertPathExists(fs, "destPath does not exist", dstPath);
-    } finally {
-      ContractTestUtils.assertPathExists(fs, "sourcePath does not exist after rename failure!", srcPath);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      /* to create parent dir */
+      authorizer.addAuthRuleForOwner("/", WRITE, true);
+      authorizer.addAuthRuleForOwner(parentDir.toString(), WRITE, false);
+      fs.updateWasbAuthorizer(authorizer);
 
-      allowRecursiveDelete(fs, parentDir.toString());
-      fs.delete(parentDir, true);
-    }
+      try {
+        fs.create(srcPath);
+        ContractTestUtils.assertPathExists(fs, "sourcePath does not exist", srcPath);
+        fs.rename(srcPath, dstPath);
+        ContractTestUtils.assertPathExists(fs, "destPath does not exist", dstPath);
+      } finally {
+        ContractTestUtils.assertPathExists(fs,
+            "sourcePath does not exist after rename failure!", srcPath);
+        allowRecursiveDelete(fs, parentDir.toString());
+        fs.delete(parentDir, true);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -388,23 +386,26 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDstDir = new Path("/testRenameAccessCheckNegativeDst");
     Path dstPath = new Path(parentDstDir, "test2.dat");
 
-    setExpectedFailureMessage("rename", dstPath);
+    String errorMsg = setExpectedFailureMessage("rename", dstPath);
 
-    authorizer.addAuthRuleForOwner("/", WRITE, true); /* to create parent dir */
-    authorizer.addAuthRuleForOwner(parentSrcDir.toString(), WRITE, true);
-    authorizer.addAuthRuleForOwner(parentDstDir.toString(), WRITE, false);
-    fs.updateWasbAuthorizer(authorizer);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      authorizer.addAuthRuleForOwner("/", WRITE, true); /* to create parent dir */
+      authorizer.addAuthRuleForOwner(parentSrcDir.toString(), WRITE, true);
+      authorizer.addAuthRuleForOwner(parentDstDir.toString(), WRITE, false);
+      fs.updateWasbAuthorizer(authorizer);
 
-    try {
-      touch(fs, srcPath);
-      ContractTestUtils.assertPathExists(fs, "sourcePath does not exist", srcPath);
-      fs.mkdirs(parentDstDir);
-      fs.rename(srcPath, dstPath);
-      ContractTestUtils.assertPathDoesNotExist(fs, "destPath does not exist", dstPath);
-    } finally {
-      ContractTestUtils.assertPathExists(fs, "sourcePath does not exist after rename !", srcPath);
-      recursiveDelete(parentSrcDir);
-    }
+      try {
+        touch(fs, srcPath);
+        ContractTestUtils.assertPathExists(fs, "sourcePath does not exist", srcPath);
+        fs.mkdirs(parentDstDir);
+        fs.rename(srcPath, dstPath);
+        ContractTestUtils.assertPathDoesNotExist(fs, "destPath does not exist", dstPath);
+      } finally {
+        ContractTestUtils.assertPathExists(fs,
+            "sourcePath does not exist after rename !", srcPath);
+        recursiveDelete(parentSrcDir);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -630,52 +631,53 @@ public class TestNativeAzureFileSystemAuthorization
     final Path parentDstDir = new Path("/testRenameWithStickyBitNegativeDst");
     final Path dstPath = new Path(parentDstDir, "test2.dat");
 
-    expectedEx.expect(WasbAuthorizationException.class);
-    expectedEx.expectMessage(String.format("Rename operation for %s is not permitted."
-      + " Details : Stickybit check failed.", srcPath.toString()));
+    String errorMsg = String.format("Rename operation for %s is not permitted."
+        + " Details : Stickybit check failed.", srcPath.toString());
 
-    /* to create parent dirs */
-    authorizer.addAuthRuleForOwner("/", WRITE, true);
-    authorizer.addAuthRuleForOwner(parentSrcDir.toString(),
-        WRITE, true);
-    /* Required for asserPathExists calls */
-    fs.updateWasbAuthorizer(authorizer);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      /* to create parent dirs */
+      authorizer.addAuthRuleForOwner("/", WRITE, true);
+      authorizer.addAuthRuleForOwner(parentSrcDir.toString(),
+          WRITE, true);
+      /* Required for asserPathExists calls */
+      fs.updateWasbAuthorizer(authorizer);
 
-    try {
-      touch(fs, srcPath);
-      assertPathExists(fs, "sourcePath does not exist", srcPath);
-      fs.mkdirs(parentDstDir);
-      assertIsDirectory(fs, parentDstDir);
-      // set stickybit on parent of source folder
-      fs.setPermission(parentSrcDir, new FsPermission(STICKYBIT_PERMISSION_CONSTANT));
+      try {
+        touch(fs, srcPath);
+        assertPathExists(fs, "sourcePath does not exist", srcPath);
+        fs.mkdirs(parentDstDir);
+        assertIsDirectory(fs, parentDstDir);
+        // set stickybit on parent of source folder
+        fs.setPermission(parentSrcDir, new FsPermission(STICKYBIT_PERMISSION_CONSTANT));
 
-      UserGroupInformation dummyUser = UserGroupInformation.createUserForTesting(
-          "dummyUser", new String[] {"dummygroup"});
+        UserGroupInformation dummyUser = UserGroupInformation.createUserForTesting(
+            "dummyUser", new String[] {"dummygroup"});
 
-      dummyUser.doAs(new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          // Add auth rules for dummyuser
-          authorizer.addAuthRule(parentSrcDir.toString(),
-            WRITE, getCurrentUserShortName(), true);
-          authorizer.addAuthRule(parentDstDir.toString(),
-            WRITE, getCurrentUserShortName(), true);
+        dummyUser.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            // Add auth rules for dummyuser
+            authorizer.addAuthRule(parentSrcDir.toString(),
+                WRITE, getCurrentUserShortName(), true);
+            authorizer.addAuthRule(parentDstDir.toString(),
+                WRITE, getCurrentUserShortName(), true);
+
+            try {
+              fs.rename(srcPath, dstPath);
+            } catch (WasbAuthorizationException wae) {
+              assertPathExists(fs, "sourcePath does not exist", srcPath);
+              assertPathDoesNotExist(fs, "destPath exists", dstPath);
+              throw wae;
+            }
 
-          try {
-            fs.rename(srcPath, dstPath);
-          } catch (WasbAuthorizationException wae) {
-            assertPathExists(fs, "sourcePath does not exist", srcPath);
-            assertPathDoesNotExist(fs, "destPath exists", dstPath);
-            throw wae;
+            return null;
           }
-
-          return null;
-        }
-      });
-    } finally {
-      recursiveDelete(parentSrcDir);
-      recursiveDelete(parentDstDir);
-    }
+        });
+      } finally {
+        recursiveDelete(parentSrcDir);
+        recursiveDelete(parentDstDir);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -777,33 +779,35 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/testReadAccessCheckNegative");
     Path testPath = new Path(parentDir, "test.dat");
 
-    setExpectedFailureMessage("read", testPath);
+    String errorMsg = setExpectedFailureMessage("read", testPath);
 
-    authorizer.addAuthRuleForOwner("/", WRITE, true);
-    authorizer.addAuthRuleForOwner(testPath.toString(), READ, false);
-    fs.updateWasbAuthorizer(authorizer);
-
-    FSDataInputStream inputStream = null;
-    FSDataOutputStream fso = null;
+    assertThrows(WasbAuthorizationException.class, () -> {
+      authorizer.addAuthRuleForOwner("/", WRITE, true);
+      authorizer.addAuthRuleForOwner(testPath.toString(), READ, false);
+      fs.updateWasbAuthorizer(authorizer);
 
-    try {
-      fso = fs.create(testPath);
-      String data = "Hello World";
-      fso.writeBytes(data);
-      fso.close();
+      FSDataInputStream inputStream = null;
+      FSDataOutputStream fso = null;
 
-      inputStream = fs.open(testPath);
-      ContractTestUtils.verifyRead(inputStream, data.getBytes(), 0, data.length());
-    } finally {
-      if (fso != null) {
+      try {
+        fso = fs.create(testPath);
+        String data = "Hello World";
+        fso.writeBytes(data);
         fso.close();
+
+        inputStream = fs.open(testPath);
+        ContractTestUtils.verifyRead(inputStream, data.getBytes(), 0, data.length());
+      } finally {
+        if (fso != null) {
+          fso.close();
+        }
+        if (inputStream != null) {
+          inputStream.close();
+        }
+        allowRecursiveDelete(fs, parentDir.toString());
+        fs.delete(parentDir, true);
       }
-      if (inputStream != null) {
-        inputStream.close();
-      }
-      allowRecursiveDelete(fs, parentDir.toString());
-      fs.delete(parentDir, true);
-    }
+    }, errorMsg);
   }
 
   /**
@@ -838,31 +842,33 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/");
     Path testPath = new Path(parentDir, "test.dat");
 
-    setExpectedFailureMessage("delete", testPath);
-
-    authorizer.addAuthRuleForOwner("/", WRITE, true);
-    fs.updateWasbAuthorizer(authorizer);
-    try {
-      fs.create(testPath);
-      ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
-
+    String errorMsg = setExpectedFailureMessage("delete", testPath);
 
-      /* Remove permissions for delete to force failure */
-      authorizer.deleteAllAuthRules();
-      authorizer.addAuthRuleForOwner("/", WRITE, false);
-      fs.updateWasbAuthorizer(authorizer);
-
-      fs.delete(testPath, false);
-    }
-    finally {
-      /* Restore permissions to force a successful delete */
-      authorizer.deleteAllAuthRules();
+    assertThrows(WasbAuthorizationException.class, () -> {
       authorizer.addAuthRuleForOwner("/", WRITE, true);
       fs.updateWasbAuthorizer(authorizer);
+      try {
+        fs.create(testPath);
+        ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
 
-      fs.delete(testPath, false);
-      ContractTestUtils.assertPathDoesNotExist(fs, "testPath exists after deletion!", testPath);
-    }
+
+        /* Remove permissions for delete to force failure */
+        authorizer.deleteAllAuthRules();
+        authorizer.addAuthRuleForOwner("/", WRITE, false);
+        fs.updateWasbAuthorizer(authorizer);
+
+        fs.delete(testPath, false);
+      }
+      finally {
+        /* Restore permissions to force a successful delete */
+        authorizer.deleteAllAuthRules();
+        authorizer.addAuthRuleForOwner("/", WRITE, true);
+        fs.updateWasbAuthorizer(authorizer);
+
+        fs.delete(testPath, false);
+        ContractTestUtils.assertPathDoesNotExist(fs, "testPath exists after deletion!", testPath);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -982,43 +988,44 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/testSingleFileDeleteWithStickyBitNegative");
     Path testPath = new Path(parentDir, "test.dat");
 
-    expectedEx.expect(WasbAuthorizationException.class);
-    expectedEx.expectMessage(String.format("%s has sticky bit set. File %s cannot be deleted.",
-        parentDir.toString(), testPath.toString()));
+    String errorMsg = String.format("%s has sticky bit set. File %s cannot be deleted.",
+        parentDir.toString(), testPath.toString());
 
-    authorizer.addAuthRuleForOwner("/", WRITE, true);
-    authorizer.addAuthRuleForOwner(parentDir.toString(), WRITE, true);
-    fs.updateWasbAuthorizer(authorizer);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      authorizer.addAuthRuleForOwner("/", WRITE, true);
+      authorizer.addAuthRuleForOwner(parentDir.toString(), WRITE, true);
+      fs.updateWasbAuthorizer(authorizer);
 
-    try {
-      fs.create(testPath);
-      ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
-      // set stickybit on parent directory
-      fs.setPermission(parentDir, new FsPermission(STICKYBIT_PERMISSION_CONSTANT));
+      try {
+        fs.create(testPath);
+        ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
+        // set stickybit on parent directory
+        fs.setPermission(parentDir, new FsPermission(STICKYBIT_PERMISSION_CONSTANT));
 
-      UserGroupInformation dummyUser = UserGroupInformation.createUserForTesting(
-          "dummyUser", new String[] {"dummygroup"});
+        UserGroupInformation dummyUser = UserGroupInformation.createUserForTesting(
+           "dummyUser", new String[] {"dummygroup"});
 
-      dummyUser.doAs(new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try {
-            authorizer.addAuthRule(parentDir.toString(), WRITE,
-                getCurrentUserShortName(), true);
-            fs.delete(testPath, true);
-            return null;
-          }
-          catch (WasbAuthorizationException wae) {
-            ContractTestUtils.assertPathExists(fs, "testPath should not be deleted!", testPath);
-            throw wae;
+        dummyUser.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              authorizer.addAuthRule(parentDir.toString(), WRITE,
+                  getCurrentUserShortName(), true);
+              fs.delete(testPath, true);
+              return null;
+            }
+            catch (WasbAuthorizationException wae) {
+              ContractTestUtils.assertPathExists(fs, "testPath should not be deleted!", testPath);
+              throw wae;
+            }
           }
-        }
-      });
-    }
-    finally {
-      allowRecursiveDelete(fs, parentDir.toString());
-      fs.delete(parentDir, true);
-    }
+        });
+      }
+      finally {
+        allowRecursiveDelete(fs, parentDir.toString());
+        fs.delete(parentDir, true);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -1362,19 +1369,21 @@ public class TestNativeAzureFileSystemAuthorization
 
     Path testPath = new Path("/testMkdirsAccessCheckNegative/1/2/3");
 
-    setExpectedFailureMessage("mkdirs", testPath);
+    String errorMsg = setExpectedFailureMessage("mkdirs", testPath);
 
-    authorizer.addAuthRuleForOwner("/", WRITE, false);
-    fs.updateWasbAuthorizer(authorizer);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      authorizer.addAuthRuleForOwner("/", WRITE, false);
+      fs.updateWasbAuthorizer(authorizer);
 
-    try {
-      fs.mkdirs(testPath);
-      ContractTestUtils.assertPathDoesNotExist(fs, "testPath was not created", testPath);
-    }
-    finally {
-      allowRecursiveDelete(fs, "/testMkdirsAccessCheckNegative");
-      fs.delete(new Path("/testMkdirsAccessCheckNegative"), true);
-    }
+      try {
+        fs.mkdirs(testPath);
+        ContractTestUtils.assertPathDoesNotExist(fs, "testPath was not created", testPath);
+      }
+      finally {
+        allowRecursiveDelete(fs, "/testMkdirsAccessCheckNegative");
+        fs.delete(new Path("/testMkdirsAccessCheckNegative"), true);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -1434,30 +1443,32 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/testOwnerPermissionNegative");
     Path childDir = new Path(parentDir, "childDir");
 
-    setExpectedFailureMessage("mkdirs", childDir);
+    String errorMsg = setExpectedFailureMessage("mkdirs", childDir);
 
-    authorizer.addAuthRuleForOwner("/", WRITE, true);
-    authorizer.addAuthRuleForOwner(parentDir.toString(), WRITE, true);
+    assertThrows(WasbAuthorizationException.class, () -> {
+      authorizer.addAuthRuleForOwner("/", WRITE, true);
+      authorizer.addAuthRuleForOwner(parentDir.toString(), WRITE, true);
 
-    fs.updateWasbAuthorizer(authorizer);
+      fs.updateWasbAuthorizer(authorizer);
 
-    try{
-      fs.mkdirs(parentDir);
-      UserGroupInformation ugiSuperUser = UserGroupInformation.createUserForTesting(
-          "testuser", new String[] {});
+      try {
+        fs.mkdirs(parentDir);
+        UserGroupInformation ugiSuperUser = UserGroupInformation.createUserForTesting(
+            "testuser", new String[] {});
 
-      ugiSuperUser.doAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws Exception {
-          fs.mkdirs(childDir);
-          return null;
-        }
-      });
+        ugiSuperUser.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            fs.mkdirs(childDir);
+            return null;
+          }
+        });
 
-    } finally {
-       allowRecursiveDelete(fs, parentDir.toString());
-       fs.delete(parentDir, true);
-    }
+      } finally {
+        allowRecursiveDelete(fs, parentDir.toString());
+        fs.delete(parentDir, true);
+      }
+    }, errorMsg);
   }
 
   /**
@@ -1533,8 +1544,8 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertPathExists(fs, "test path does not exist", testPath);
 
       String owner = fs.getFileStatus(testPath).getOwner();
-      Assume.assumeTrue("changing owner requires original and new owner to be different",
-        !StringUtils.equalsIgnoreCase(owner, newOwner));
+      assumeTrue(!StringUtils.equalsIgnoreCase(owner, newOwner),
+          "changing owner requires original and new owner to be different");
 
       authorisedUser.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
@@ -1576,8 +1587,8 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertPathExists(fs, "test path does not exist", testPath);
 
       String owner = fs.getFileStatus(testPath).getOwner();
-      Assume.assumeTrue("changing owner requires original and new owner to be different",
-        !StringUtils.equalsIgnoreCase(owner, newOwner));
+      assumeTrue(!StringUtils.equalsIgnoreCase(owner, newOwner),
+          "changing owner requires original and new owner to be different");
 
       user.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
@@ -1892,17 +1903,18 @@ public class TestNativeAzureFileSystemAuthorization
    */
   @Test
   public void testAccessFileDoesNotExist() throws Throwable{
-    expectedEx.expect(FileNotFoundException.class);
-    Configuration conf = fs.getConf();
-    fs.setConf(conf);
-    final Path testPath = new Path("/testAccessFileDoesNotExist");
-
-    authorizer.init(conf);
-    authorizer.addAuthRuleForOwner(testPath.toString(), READ,  true);
-    authorizer.addAuthRuleForOwner(testPath.toString(), WRITE,  true);
-    fs.updateWasbAuthorizer(authorizer);
-    assertPathDoesNotExist(fs, "test path exists", testPath);
-    fs.access(testPath, FsAction.ALL);
+    assertThrows(FileNotFoundException.class, () -> {
+      Configuration conf = fs.getConf();
+      fs.setConf(conf);
+      final Path testPath = new Path("/testAccessFileDoesNotExist");
+
+      authorizer.init(conf);
+      authorizer.addAuthRuleForOwner(testPath.toString(), READ, true);
+      authorizer.addAuthRuleForOwner(testPath.toString(), WRITE, true);
+      fs.updateWasbAuthorizer(authorizer);
+      assertPathDoesNotExist(fs, "test path exists", testPath);
+      fs.access(testPath, FsAction.ALL);
+    });
   }
 
   /**
@@ -1910,15 +1922,16 @@ public class TestNativeAzureFileSystemAuthorization
    */
   @Test
   public void testAccessFileDoesNotExistWhenNoAccessPermission() throws Throwable {
-    expectedEx.expect(FileNotFoundException.class);
-    Configuration conf = fs.getConf();
-    fs.setConf(conf);
-    final Path testPath = new Path("/testAccessFileDoesNotExistWhenNoAccessPermission");
+    assertThrows(FileNotFoundException.class, () -> {
+      Configuration conf = fs.getConf();
+      fs.setConf(conf);
+      final Path testPath = new Path("/testAccessFileDoesNotExistWhenNoAccessPermission");
 
-    authorizer.init(conf);
-    fs.updateWasbAuthorizer(authorizer);
-    assertPathDoesNotExist(fs, "test path exists", testPath);
-    fs.access(testPath, FsAction.ALL);
+      authorizer.init(conf);
+      fs.updateWasbAuthorizer(authorizer);
+      assertPathDoesNotExist(fs, "test path exists", testPath);
+      fs.access(testPath, FsAction.ALL);
+    });
   }
 
   /**
@@ -2074,13 +2087,13 @@ public class TestNativeAzureFileSystemAuthorization
   private void assertPermissionEquals(Path path, FsPermission newPermission)
       throws IOException {
     FileStatus status = fs.getFileStatus(path);
-    assertEquals("Wrong permissions in " + status,
-        newPermission, status.getPermission());
+    assertEquals(newPermission, status.getPermission(),
+        "Wrong permissions in " + status);
   }
 
   private void assertOwnerEquals(Path path, String owner) throws IOException {
     FileStatus status = fs.getFileStatus(path);
-    assertEquals("Wrong owner in " + status, owner, status.getOwner());
+    assertEquals(owner, status.getOwner(), "Wrong owner in " + status);
   }
 
   private void assertNoAccess(final Path path, final FsAction action)

+ 11 - 12
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java

@@ -24,9 +24,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.ByteArrayOutputStream;
 import java.io.OutputStream;
@@ -47,7 +46,7 @@ public class TestNativeAzureFileSystemBlockCompaction extends AbstractWasbTestBa
 
   private AzureBlobStorageTestAccount testAccount = null;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
     testAccount = createTestAccount();
@@ -88,8 +87,8 @@ public class TestNativeAzureFileSystemBlockCompaction extends AbstractWasbTestBa
       dataOutputStream = (SyncableDataOutputStream) appendStream.getWrappedStream();
     }
 
-    Assert.assertNotNull("Did not recognize " + dataOutputStream,
-        dataOutputStream);
+    assertNotNull(
+       dataOutputStream, "Did not recognize " + dataOutputStream);
 
     return (BlockBlobAppendStream) dataOutputStream.getOutStream();
   }
@@ -97,11 +96,11 @@ public class TestNativeAzureFileSystemBlockCompaction extends AbstractWasbTestBa
   private void verifyBlockList(BlockBlobAppendStream blockBlobStream,
                                int[] testData) throws Throwable {
     List<BlockEntry> blockList = blockBlobStream.getBlockList();
-    Assert.assertEquals("Block list length", testData.length, blockList.size());
+    assertEquals(testData.length, blockList.size(), "Block list length");
 
     int i = 0;
     for (BlockEntry block: blockList) {
-      Assert.assertTrue(block.getSize() == testData[i++]);
+      assertTrue(block.getSize() == testData[i++]);
     }
   }
 
@@ -135,13 +134,13 @@ public class TestNativeAzureFileSystemBlockCompaction extends AbstractWasbTestBa
       } else if (wrappedStream instanceof SyncableDataOutputStream) {
         dataOutputStream = (SyncableDataOutputStream) wrappedStream;
       } else {
-        Assert.fail("Unable to determine type of " + wrappedStream
+        fail("Unable to determine type of " + wrappedStream
             + " class of " + wrappedStream.getClass());
       }
 
-      Assert.assertFalse("Data output stream is a BlockBlobAppendStream: "
-          + dataOutputStream,
-          dataOutputStream.getOutStream() instanceof BlockBlobAppendStream);
+      assertFalse(dataOutputStream.getOutStream() instanceof BlockBlobAppendStream,
+          "Data output stream is a BlockBlobAppendStream: "
+          + dataOutputStream);
 
     }
   }

+ 7 - 7
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrency.java

@@ -30,12 +30,14 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 public class TestNativeAzureFileSystemConcurrency extends AbstractWasbTestBase {
   private InMemoryBlockBlobStore backingStore;
 
   @Override
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
     backingStore = getTestAccount().getMockStorage().getBackingStore();
@@ -95,8 +97,8 @@ public class TestNativeAzureFileSystemConcurrency extends AbstractWasbTestBase {
     FSDataOutputStream outputStream = fs.create(filePath);
     // Make sure I can't see the temporary blob if I ask for a listing
     FileStatus[] listOfRoot = fs.listStatus(new Path("/"));
-    assertEquals("Expected one file listed, instead got: "
-        + toString(listOfRoot), 1, listOfRoot.length);
+    assertEquals(1, listOfRoot.length, "Expected one file listed, instead got: "
+        + toString(listOfRoot));
     assertEquals(fs.makeQualified(filePath), listOfRoot[0].getPath());
     outputStream.close();
   }
@@ -169,10 +171,8 @@ public class TestNativeAzureFileSystemConcurrency extends AbstractWasbTestBase {
       for (Thread t : threads) {
         t.join();
       }
-      assertTrue(
-          "Encountered exceptions: "
-              + StringUtils.join("\r\n", selectToString(exceptionsEncountered)),
-          exceptionsEncountered.isEmpty());
+      assertTrue(exceptionsEncountered.isEmpty(), "Encountered exceptions: "
+          + StringUtils.join("\r\n", selectToString(exceptionsEncountered)));
       tearDown();
       setUp();
     }

+ 3 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemFileNameCheck.java

@@ -23,7 +23,8 @@ import java.util.HashMap;
 
 import org.apache.hadoop.fs.Path;
 
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests the scenario where a colon is included in the file/directory name.
@@ -35,6 +36,7 @@ import org.junit.Test;
 public class TestNativeAzureFileSystemFileNameCheck extends AbstractWasbTestBase {
   private String root = null;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();

+ 7 - 7
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemMocked.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.azure;
 
+import org.junit.jupiter.api.Disabled;
 import java.io.IOException;
-import org.junit.Ignore;
 
 /**
  * Run {@link NativeAzureFileSystemBaseTest} tests against a mocked store,
@@ -36,32 +36,32 @@ public class TestNativeAzureFileSystemMocked extends
   // Ignore the following tests because taking a lease requires a real
   // (not mock) file system store. These tests don't work on the mock.
   @Override
-  @Ignore
+  @Disabled
   public void testLeaseAsDistributedLock() {
   }
 
   @Override
-  @Ignore
+  @Disabled
   public void testSelfRenewingLease() {
   }
 
   @Override
-  @Ignore
+  @Disabled
   public void testRedoFolderRenameAll() {
   }
 
   @Override
-  @Ignore
+  @Disabled
   public void testCreateNonRecursive() {
   }
 
   @Override
-  @Ignore
+  @Disabled
   public void testSelfRenewingLeaseFileDelete() {
   }
 
   @Override
-  @Ignore
+  @Disabled
   public void testRenameRedoFolderAlreadyDone() throws IOException{
   }
 }

+ 8 - 5
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemUploadLogic.java

@@ -24,8 +24,8 @@ import java.io.OutputStream;
 
 import org.apache.hadoop.fs.Path;
 
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests for the upload, buffering and flush logic in WASB.
@@ -65,7 +65,8 @@ public class TestNativeAzureFileSystemUploadLogic extends AbstractWasbTestBase {
    * bit of data.
    */
   @Test
-  @Ignore /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
+  @Disabled
+  /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
   public void testConsistencyAfterSmallFlushes() throws Exception {
     testConsistencyAfterManyFlushes(FlushFrequencyVariation.BeforeSingleBufferFull);
   }
@@ -75,7 +76,8 @@ public class TestNativeAzureFileSystemUploadLogic extends AbstractWasbTestBase {
    * bit of data.
    */
   @Test
-  @Ignore /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
+  @Disabled
+  /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
   public void testConsistencyAfterMediumFlushes() throws Exception {
     testConsistencyAfterManyFlushes(FlushFrequencyVariation.AfterSingleBufferFull);
   }
@@ -85,7 +87,8 @@ public class TestNativeAzureFileSystemUploadLogic extends AbstractWasbTestBase {
    * of data.
    */
   @Test
-  @Ignore /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
+  @Disabled
+  /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
   public void testConsistencyAfterLargeFlushes() throws Exception {
     testConsistencyAfterManyFlushes(FlushFrequencyVariation.AfterAllRingBufferFull);
   }

+ 6 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java

@@ -24,9 +24,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-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;
 
 /**
  * Tests that WASB handles things gracefully when users add blobs to the Azure
@@ -38,14 +38,14 @@ public class TestOutOfBandAzureBlobOperations
   private FileSystem fs;
   private InMemoryBlockBlobStore backingStore;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     testAccount = AzureBlobStorageTestAccount.createMock();
     fs = testAccount.getFileSystem();
     backingStore = testAccount.getMockStorage().getBackingStore();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     testAccount.cleanup();
     fs = null;
@@ -115,7 +115,7 @@ public class TestOutOfBandAzureBlobOperations
       // Trying to delete root/b/c would cause a dilemma for WASB, so
       // it should throw.
       fs.delete(new Path("/root/b/c"), true);
-      assertTrue("Should've thrown.", false);
+      assertTrue(false, "Should've thrown.");
     } catch (AzureException e) {
       assertEquals("File /root/b/c has a parent directory /root/b"
           + " which is also a file. Can't resolve.", e.getMessage());

+ 2 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestShellDecryptionKeyProvider.java

@@ -25,8 +25,7 @@ import java.nio.charset.StandardCharsets;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,8 +50,7 @@ public class TestShellDecryptionKeyProvider
     conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key);
     try {
       provider.getStorageAccountKey(account, conf);
-      Assert
-          .fail("fs.azure.shellkeyprovider.script is not specified, we should throw");
+      fail("fs.azure.shellkeyprovider.script is not specified, we should throw");
     } catch (KeyProviderException e) {
       LOG.info("Received an expected exception: " + e.getMessage());
     }

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

@@ -20,7 +20,7 @@ package org.apache.hadoop.fs.azure;
 import java.io.IOException;
 import java.io.OutputStream;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.test.LambdaTestUtils;
 

+ 8 - 7
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbFsck.java

@@ -23,10 +23,10 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests which look at fsck recovery.
@@ -36,14 +36,14 @@ public class TestWasbFsck extends AbstractWasbTestWithTimeout {
   private FileSystem fs;
   private InMemoryBlockBlobStore backingStore;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     testAccount = AzureBlobStorageTestAccount.createMock();
     fs = testAccount.getFileSystem();
     backingStore = testAccount.getMockStorage().getBackingStore();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     testAccount.cleanup();
     fs = null;
@@ -67,7 +67,8 @@ public class TestWasbFsck extends AbstractWasbTestWithTimeout {
    * Tests that we recover files properly
    */
   @Test
-  @Ignore  /* flush() no longer does anything  @@TODO: reinstate an appropriate test of fsck recovery*/
+  @Disabled
+  /* flush() no longer does anything  @@TODO: reinstate an appropriate test of fsck recovery*/
   public void testRecover() throws Exception {
     Path danglingFile = new Path("/crashedInTheMiddle");
 

+ 4 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AbstractAzureScaleTest.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.fs.azure.integration;
 
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -32,17 +34,18 @@ import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.*;
  * tests if not.
  *
  */
+@Timeout(AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS)
 public abstract class AbstractAzureScaleTest
     extends AbstractWasbTestBase implements Sizes {
 
   protected static final Logger LOG =
       LoggerFactory.getLogger(AbstractAzureScaleTest.class);
 
-  @Override
   protected int getTestTimeoutMillis() {
     return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS;
   }
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();

+ 10 - 11
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java

@@ -26,7 +26,7 @@ import java.io.OutputStreamWriter;
 import java.net.URI;
 import java.util.List;
 
-import org.junit.Assert;
+import org.junit.jupiter.api.Assertions;
 import org.junit.Assume;
 import org.junit.internal.AssumptionViolatedException;
 import org.slf4j.Logger;
@@ -42,7 +42,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
 
-import static org.junit.Assume.assumeTrue;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
 import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX_REGEX;
 import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_TEST_ACCOUNT_NAME_WITH_DOMAIN;
@@ -56,7 +56,7 @@ import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
  * Utilities for the Azure tests. Based on {@code S3ATestUtils}, so
  * (initially) has unused method.
  */
-public final class AzureTestUtils extends Assert {
+public final class AzureTestUtils extends Assertions {
   private static final Logger LOG = LoggerFactory.getLogger(
       AzureTestUtils.class);
 
@@ -343,10 +343,10 @@ public final class AzureTestUtils extends Assert {
    * @param expectedClass class
    * @param obj object to check
    */
-  public static void assertInstanceOf(Class<?> expectedClass, Object obj) {
-    Assert.assertTrue(String.format("Expected instance of class %s, but is %s.",
-        expectedClass, obj.getClass()),
-        expectedClass.isAssignableFrom(obj.getClass()));
+  public static void assertInstanceOf2(Class<?> expectedClass, Object obj) {
+    Assertions.assertTrue(
+       expectedClass.isAssignableFrom(obj.getClass()), String.format("Expected instance of class %s, but is %s.",
+        expectedClass, obj.getClass()));
   }
 
   /**
@@ -381,7 +381,7 @@ public final class AzureTestUtils extends Assert {
   public static void assertOptionEquals(Configuration conf,
       String key,
       String expected) {
-    assertEquals("Value of " + key, expected, conf.get(key));
+    assertEquals(expected, conf.get(key), "Value of " + key);
   }
 
   /**
@@ -445,7 +445,6 @@ public final class AzureTestUtils extends Assert {
    * field.
    * @param testAccount test account to clean up
    * @return null
-   * @throws Execption cleanup problems
    */
   public static AzureBlobStorageTestAccount cleanup(
       AzureBlobStorageTestAccount testAccount) throws Exception {
@@ -496,8 +495,8 @@ public final class AzureTestUtils extends Assert {
     if (accountName == null) {
       accountName = conf.get(WASB_TEST_ACCOUNT_NAME_WITH_DOMAIN);
     }
-    assumeTrue("Account for WASB is missing or it is not in correct format",
-            accountName != null && !accountName.endsWith(WASB_ACCOUNT_NAME_DOMAIN_SUFFIX_REGEX));
+    assumeTrue(accountName != null && !accountName.endsWith(WASB_ACCOUNT_NAME_DOMAIN_SUFFIX_REGEX),
+        "Account for WASB is missing or it is not in correct format");
     return accountName;
   }
 

+ 1 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/CleanupTestContainers.java

@@ -23,7 +23,7 @@ import java.util.EnumSet;
 import com.microsoft.azure.storage.CloudStorageAccount;
 import com.microsoft.azure.storage.blob.CloudBlobClient;
 import com.microsoft.azure.storage.blob.CloudBlobContainer;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.fs.azure.AbstractWasbTestBase;
 import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;

+ 17 - 16
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/ITestAzureHugeFiles.java

@@ -23,11 +23,11 @@ import java.io.IOException;
 import java.util.EnumSet;
 import java.util.Iterator;
 
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.FixMethodOrder;
-import org.junit.Test;
-import org.junit.runners.MethodSorters;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestMethodOrder;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.TestInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,6 +43,7 @@ import org.apache.hadoop.io.IOUtils;
 
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.*;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
 
 /**
@@ -60,7 +61,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
  * ordering.</b>
  */
 
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+@TestMethodOrder(MethodOrderer.Alphanumeric.class)
 public class ITestAzureHugeFiles extends AbstractAzureScaleTest {
 
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -80,6 +81,7 @@ public class ITestAzureHugeFiles extends AbstractAzureScaleTest {
 
   private Path testPath;
 
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -151,8 +153,8 @@ public class ITestAzureHugeFiles extends AbstractAzureScaleTest {
     assertPathExists(getFileSystem(), "huge file not created", hugefile);
     try {
       FileStatus status = getFileSystem().getFileStatus(hugefile);
-      Assume.assumeTrue("Not a file: " + status, status.isFile());
-      Assume.assumeTrue("File " + hugefile + " is empty", status.getLen() > 0);
+      assumeTrue(status.isFile(), "Not a file: " + status);
+      assumeTrue(status.getLen() > 0, "File " + hugefile + " is empty");
       return status;
     } catch (FileNotFoundException e) {
       skip("huge file not created: " + hugefile);
@@ -201,9 +203,9 @@ public class ITestAzureHugeFiles extends AbstractAzureScaleTest {
         timeout, uploadTime, KEY_TEST_TIMEOUT, uploadTime * 2),
         uploadTime < timeout);
 */
-    assertEquals("File size set in " + KEY_HUGE_FILESIZE + " = " + filesize
-            + " is not a multiple of " + UPLOAD_BLOCKSIZE,
-        0, filesize % UPLOAD_BLOCKSIZE);
+    assertEquals(0, filesize % UPLOAD_BLOCKSIZE,
+        "File size set in " + KEY_HUGE_FILESIZE + " = " + filesize
+        + " is not a multiple of " + UPLOAD_BLOCKSIZE);
 
     byte[] data = SOURCE_DATA;
 
@@ -254,7 +256,7 @@ public class ITestAzureHugeFiles extends AbstractAzureScaleTest {
     ContractTestUtils.assertPathExists(fs, "Huge file", hugefile);
     FileStatus status = fs.getFileStatus(hugefile);
     ContractTestUtils.assertIsFile(hugefile, status);
-    assertEquals("File size in " + status, filesize, status.getLen());
+    assertEquals(filesize, status.getLen(), "File size in " + status);
   }
 
   @Test
@@ -345,8 +347,7 @@ public class ITestAzureHugeFiles extends AbstractAzureScaleTest {
   public void test_060_openAndReadWholeFileBlocks() throws Throwable {
     FileStatus status = assumeHugeFileExists();
     int blockSize = S_1M;
-    describe("Open the test file and read it in blocks of size %d",
-        blockSize);
+    describe("Open the test file and read it in blocks of size %d", blockSize);
     long len =  status.getLen();
     FSDataInputStream in = openDataFile();
     NanoTimer timer2 = null;
@@ -398,8 +399,8 @@ public class ITestAzureHugeFiles extends AbstractAzureScaleTest {
         if (bandwidthInBytes(blockTimer, blockSize) < minimumBandwidth) {
           LOG.warn("Bandwidth {} too low on block {}: resetting connection",
               bw, blockId);
-          Assert.assertTrue("Bandwidth of " + bw + " too low after "
-              + resetCount + " attempts", resetCount <= maxResetCount);
+          assertTrue(resetCount <= maxResetCount, "Bandwidth of " + bw + " too low after "
+              + resetCount + " attempts");
           resetCount++;
           // reset the connection
         }

+ 58 - 61
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/ITestAzureFileSystemInstrumentation.java

@@ -48,7 +48,7 @@ import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsTag;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentMatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -155,37 +155,36 @@ public class ITestAzureFileSystemInstrumentation extends AbstractWasbTestBase {
     base = assertWebResponsesInRange(base, 2, 15);
     getBandwidthGaugeUpdater().triggerUpdate(true);
     long bytesWritten = AzureMetricsTestUtil.getCurrentBytesWritten(getInstrumentation());
-    assertTrue("The bytes written in the last second " + bytesWritten +
-        " is pretty far from the expected range of around " + FILE_SIZE +
-        " bytes plus a little overhead.",
-        bytesWritten > (FILE_SIZE / 2) && bytesWritten < (FILE_SIZE * 2));
+    assertTrue(bytesWritten > (FILE_SIZE / 2) && bytesWritten < (FILE_SIZE * 2),
+        "The bytes written in the last second " + bytesWritten
+        + " is pretty far from the expected range of around " + FILE_SIZE
+        + " bytes plus a little overhead.");
     long totalBytesWritten = AzureMetricsTestUtil.getCurrentTotalBytesWritten(getInstrumentation());
-    assertTrue("The total bytes written  " + totalBytesWritten +
-        " is pretty far from the expected range of around " + FILE_SIZE +
-        " bytes plus a little overhead.",
-        totalBytesWritten >= FILE_SIZE && totalBytesWritten < (FILE_SIZE * 2));
+    assertTrue(totalBytesWritten >= FILE_SIZE && totalBytesWritten < (FILE_SIZE * 2),
+        "The total bytes written  " + totalBytesWritten
+         + " is pretty far from the expected range of around " + FILE_SIZE
+         + " bytes plus a little overhead.");
     long uploadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_UPLOAD_RATE);
     LOG.info("Upload rate: " + uploadRate + " bytes/second.");
     long expectedRate = (FILE_SIZE * 1000L) / uploadDurationMs;
-    assertTrue("The upload rate " + uploadRate +
-        " is below the expected range of around " + expectedRate +
-        " bytes/second that the unit test observed. This should never be" +
-        " the case since the test underestimates the rate by looking at " +
-        " end-to-end time instead of just block upload time.",
-        uploadRate >= expectedRate);
+    assertTrue(uploadRate >= expectedRate, "The upload rate " + uploadRate
+        + " is below the expected range of around " + expectedRate
+        + " bytes/second that the unit test observed. This should never be"
+        + " the case since the test underestimates the rate by looking at "
+        + " end-to-end time instead of just block upload time.");
     long uploadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
         WASB_UPLOAD_LATENCY);
     LOG.info("Upload latency: {}", uploadLatency);
     long expectedLatency = uploadDurationMs; // We're uploading less than a block.
-    assertTrue("The upload latency " + uploadLatency +
-        " should be greater than zero now that I've just uploaded a file.",
-        uploadLatency > 0);
-    assertTrue("The upload latency " + uploadLatency +
-        " is more than the expected range of around " + expectedLatency +
-        " milliseconds that the unit test observed. This should never be" +
-        " the case since the test overestimates the latency by looking at " +
-        " end-to-end time instead of just block upload time.",
-        uploadLatency <= expectedLatency);
+    assertTrue(uploadLatency > 0,
+        "The upload latency " + uploadLatency
+        + " should be greater than zero now that I've just uploaded a file.");
+    assertTrue(uploadLatency <= expectedLatency,
+        "The upload latency " + uploadLatency
+        + " is more than the expected range of around " + expectedLatency
+        + " milliseconds that the unit test observed. This should never be"
+        + " the case since the test overestimates the latency by looking at "
+        + " end-to-end time instead of just block upload time.");
 
     // Read the file
     start = new Date();
@@ -207,32 +206,32 @@ public class ITestAzureFileSystemInstrumentation extends AbstractWasbTestBase {
     long totalBytesRead = AzureMetricsTestUtil.getCurrentTotalBytesRead(getInstrumentation());
     assertEquals(FILE_SIZE, totalBytesRead);
     long bytesRead = AzureMetricsTestUtil.getCurrentBytesRead(getInstrumentation());
-    assertTrue("The bytes read in the last second " + bytesRead +
-        " is pretty far from the expected range of around " + FILE_SIZE +
-        " bytes plus a little overhead.",
-        bytesRead > (FILE_SIZE / 2) && bytesRead < (FILE_SIZE * 2));
+    assertTrue(bytesRead > (FILE_SIZE / 2) && bytesRead < (FILE_SIZE * 2),
+        "The bytes read in the last second " + bytesRead
+        + " is pretty far from the expected range of around " + FILE_SIZE
+        + " bytes plus a little overhead.");
     long downloadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_DOWNLOAD_RATE);
     LOG.info("Download rate: " + downloadRate + " bytes/second.");
     expectedRate = (FILE_SIZE * 1000L) / downloadDurationMs;
-    assertTrue("The download rate " + downloadRate +
-        " is below the expected range of around " + expectedRate +
-        " bytes/second that the unit test observed. This should never be" +
-        " the case since the test underestimates the rate by looking at " +
-        " end-to-end time instead of just block download time.",
-        downloadRate >= expectedRate);
+    assertTrue(downloadRate >= expectedRate,
+        "The download rate " + downloadRate
+        + " is below the expected range of around " + expectedRate
+        + " bytes/second that the unit test observed. This should never be"
+        + " the case since the test underestimates the rate by looking at "
+        + " end-to-end time instead of just block download time.");
     long downloadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
         WASB_DOWNLOAD_LATENCY);
     LOG.info("Download latency: " + downloadLatency);
     expectedLatency = downloadDurationMs; // We're downloading less than a block.
-    assertTrue("The download latency " + downloadLatency +
-        " should be greater than zero now that I've just downloaded a file.",
-        downloadLatency > 0);
-    assertTrue("The download latency " + downloadLatency +
-        " is more than the expected range of around " + expectedLatency +
-        " milliseconds that the unit test observed. This should never be" +
-        " the case since the test overestimates the latency by looking at " +
-        " end-to-end time instead of just block download time.",
-        downloadLatency <= expectedLatency);
+    assertTrue(downloadLatency > 0,
+        "The download latency " + downloadLatency
+         + " should be greater than zero now that I've just downloaded a file.");
+    assertTrue(downloadLatency <= expectedLatency,
+        "The download latency " + downloadLatency
+         + " is more than the expected range of around " + expectedLatency
+         + " milliseconds that the unit test observed. This should never be"
+         + " the case since the test overestimates the latency by looking at "
+         + " end-to-end time instead of just block download time.");
 
     assertNoErrors();
   }
@@ -265,18 +264,18 @@ public class ITestAzureFileSystemInstrumentation extends AbstractWasbTestBase {
     base = assertWebResponsesInRange(base, 20, 50);
     getBandwidthGaugeUpdater().triggerUpdate(true);
     long totalBytesWritten = AzureMetricsTestUtil.getCurrentTotalBytesWritten(getInstrumentation());
-    assertTrue("The total bytes written  " + totalBytesWritten +
-        " is pretty far from the expected range of around " + FILE_SIZE +
-        " bytes plus a little overhead.",
-        totalBytesWritten >= FILE_SIZE && totalBytesWritten < (FILE_SIZE * 2));
+    assertTrue(totalBytesWritten >= FILE_SIZE && totalBytesWritten < (FILE_SIZE * 2),
+        "The total bytes written  " + totalBytesWritten
+        + " is pretty far from the expected range of around " + FILE_SIZE
+        + " bytes plus a little overhead.");
     long uploadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_UPLOAD_RATE);
     LOG.info("Upload rate: " + uploadRate + " bytes/second.");
     long uploadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
         WASB_UPLOAD_LATENCY);
     LOG.info("Upload latency: " + uploadLatency);
-    assertTrue("The upload latency " + uploadLatency +
-        " should be greater than zero now that I've just uploaded a file.",
-        uploadLatency > 0);
+    assertTrue(uploadLatency > 0,
+        "The upload latency " + uploadLatency
+         + " should be greater than zero now that I've just uploaded a file.");
 
     // Read the file
     InputStream inputStream = getFileSystem().open(filePath);
@@ -300,9 +299,9 @@ public class ITestAzureFileSystemInstrumentation extends AbstractWasbTestBase {
     long downloadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
         WASB_DOWNLOAD_LATENCY);
     LOG.info("Download latency: " + downloadLatency);
-    assertTrue("The download latency " + downloadLatency +
-        " should be greater than zero now that I've just downloaded a file.",
-        downloadLatency > 0);
+    assertTrue(downloadLatency > 0,
+        "The download latency " + downloadLatency
+        + " should be greater than zero now that I've just downloaded a file.");
   }
 
   @Test
@@ -418,10 +417,10 @@ public class ITestAzureFileSystemInstrumentation extends AbstractWasbTestBase {
       try {
         outputStream.write(new byte[FILE_SIZE]);
         outputStream.close();
-        assertTrue("Should've thrown", false);
+        assertTrue(false, "Should've thrown");
       } catch (AzureException ex) {
-        assertTrue("Unexpected exception: " + ex,
-          ex.getMessage().contains("lease"));
+        assertTrue(
+         ex.getMessage().contains("lease"), "Unexpected exception: " + ex);
       }
       assertEquals(1, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_CLIENT_ERRORS));
       assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_SERVER_ERRORS));
@@ -482,11 +481,9 @@ public class ITestAzureFileSystemInstrumentation extends AbstractWasbTestBase {
       long inclusiveUpperLimit) {
     long currentResponses = getCurrentWebResponses();
     long justOperation = currentResponses - base;
-    assertTrue(String.format(
-        "Web responses expected in range [%d, %d], but was %d.",
-        inclusiveLowerLimit, inclusiveUpperLimit, justOperation),
-        justOperation >= inclusiveLowerLimit &&
-        justOperation <= inclusiveUpperLimit);
+    assertTrue(justOperation >= inclusiveLowerLimit && justOperation <= inclusiveUpperLimit,
+        String.format("Web responses expected in range [%d, %d], but was %d.",
+        inclusiveLowerLimit, inclusiveUpperLimit, justOperation));
     return currentResponses;
   }  
 

+ 5 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestBandwidthGaugeUpdater.java

@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.fs.azure.metrics;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.Date;
 
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestBandwidthGaugeUpdater {
   @Test
@@ -42,10 +42,9 @@ public class TestBandwidthGaugeUpdater {
         new Date(), 200);
     updater.triggerUpdate(true);
     long currentBytes = AzureMetricsTestUtil.getCurrentBytesWritten(instrumentation);
-    assertTrue(
+    assertTrue(currentBytes > 18 && currentBytes < 22,
         "We expect around (200/10 = 20) bytes written as the gauge value." +
-        "Got " + currentBytes,
-        currentBytes > 18 && currentBytes < 22);
+        "Got " + currentBytes);
     updater.close();
   }
 

+ 5 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestNativeAzureFileSystemMetricsSystem.java

@@ -18,12 +18,13 @@
 
 package org.apache.hadoop.fs.azure.metrics;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
-import org.junit.*;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests that the WASB-specific metrics system is working correctly.
@@ -73,8 +74,8 @@ public class TestNativeAzureFileSystemMetricsSystem {
    */
   private void assertFilesCreated(AzureBlobStorageTestAccount account,
       String name, int expected) {
-    assertEquals("Files created in account " + name,
-        expected, getFilesCreated(account));
+    assertEquals(expected, getFilesCreated(account),
+        "Files created in account " + name);
   }
 
   @Test

+ 2 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestRollingWindowAverage.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.azure.metrics;
 
-import static org.junit.Assert.assertEquals;
-import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import org.junit.jupiter.api.Test;
 
 public class TestRollingWindowAverage {
   /**