diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index e31df5eec65bb..b25ce6a6b1058 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -18,6 +18,10 @@ package org.apache.hadoop.fs.azurebfs; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.junit.jupiter.api.Assumptions.assumeFalse; + import java.io.IOException; import java.net.URI; import java.util.Hashtable; @@ -28,10 +32,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assume; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,7 +75,6 @@ import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.Assume.assumeTrue; /** * Base for AzureBlobFileSystem Integration tests. @@ -111,8 +112,8 @@ protected AbstractAbfsIntegrationTest() throws Exception { // check if accountName is set using different config key accountName = rawConfig.get(FS_AZURE_ABFS_ACCOUNT_NAME); } - assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT_NAME, - accountName != null && !accountName.isEmpty()); + assumeTrue(accountName != null && !accountName.isEmpty(), + "Not set: " + FS_AZURE_ABFS_ACCOUNT_NAME); final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName(); URI defaultUri = null; @@ -188,7 +189,7 @@ public TracingContext getTestTracingContext(AzureBlobFileSystem fs, FSOperationType.TEST_OP, needsPrimaryReqId, format, null); } - @Before + @BeforeEach public void setup() throws Exception { //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem. createFileSystem(); @@ -221,7 +222,7 @@ public void setup() throws Exception { } } - @After + @AfterEach public void teardown() throws Exception { try { IOUtils.closeStream(wasb); @@ -565,23 +566,22 @@ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled( */ protected long assertAbfsStatistics(AbfsStatistic statistic, long expectedValue, Map metricMap) { - assertEquals("Mismatch in " + statistic.getStatName(), expectedValue, - (long) metricMap.get(statistic.getStatName())); + assertEquals(expectedValue, (long) metricMap.get(statistic.getStatName()), + "Mismatch in " + statistic.getStatName()); return expectedValue; } protected void assumeValidTestConfigPresent(final Configuration conf, final String key) { String configuredValue = conf.get(accountProperty(key, accountName), conf.get(key, "")); - Assume.assumeTrue(String.format("Missing Required Test Config: %s.", key), - !configuredValue.isEmpty()); + assumeTrue(!configuredValue.isEmpty(), + String.format("Missing Required Test Config: %s.", key)); } protected void assumeValidAuthConfigsPresent() { final AuthType currentAuthType = getAuthType(); - Assume.assumeFalse( - "SAS Based Authentication Not Allowed For Integration Tests", - currentAuthType == AuthType.SAS); + assumeFalse(currentAuthType == AuthType.SAS, + "SAS Based Authentication Not Allowed For Integration Tests"); if (currentAuthType == AuthType.SharedKey) { assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_ACCOUNT_KEY); } else { @@ -612,7 +612,7 @@ public AbfsServiceType getIngressServiceType() { * @param path path to create. Can be relative or absolute. */ protected void createAzCopyFolder(Path path) throws Exception { - Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.BLOB); + assumeTrue(getAbfsServiceType() == AbfsServiceType.BLOB); assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_FIXED_SAS_TOKEN); String sasToken = getRawConfiguration().get(FS_AZURE_TEST_FIXED_SAS_TOKEN); AzcopyToolHelper azcopyHelper = AzcopyToolHelper.getInstance(sasToken); @@ -624,7 +624,7 @@ protected void createAzCopyFolder(Path path) throws Exception { * @param path path to create. Can be relative or absolute. */ protected void createAzCopyFile(Path path) throws Exception { - Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.BLOB); + assumeTrue(getAbfsServiceType() == AbfsServiceType.BLOB); assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_FIXED_SAS_TOKEN); String sasToken = getRawConfiguration().get(FS_AZURE_TEST_FIXED_SAS_TOKEN); AzcopyToolHelper azcopyHelper = AzcopyToolHelper.getInstance(sasToken); @@ -642,8 +642,8 @@ private String getAzcopyAbsolutePath(Path path) throws IOException { * Otherwise, the test will be skipped. */ protected void assumeBlobServiceType() { - Assume.assumeTrue("Blob service type is required for this test", - getAbfsServiceType() == AbfsServiceType.BLOB); + assumeTrue(getAbfsServiceType() == AbfsServiceType.BLOB, + "Blob service type is required for this test"); } /** @@ -651,8 +651,8 @@ protected void assumeBlobServiceType() { * Otherwise, the test will be skipped. */ protected void assumeDfsServiceType() { - Assume.assumeTrue("DFS service type is required for this test", - getAbfsServiceType() == AbfsServiceType.DFS); + assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS, + "DFS service type is required for this test"); } /** @@ -670,7 +670,7 @@ protected void assumeHnsEnabled() throws IOException { * @throws IOException if an error occurs while checking the account type. */ protected void assumeHnsEnabled(String errorMessage) throws IOException { - Assume.assumeTrue(errorMessage, getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem()), errorMessage); } /** @@ -688,7 +688,7 @@ protected void assumeHnsDisabled() throws IOException { * @throws IOException if an error occurs while checking the account type. */ protected void assumeHnsDisabled(String message) throws IOException { - Assume.assumeFalse(message, getIsNamespaceEnabled(getFileSystem())); + assumeFalse(getIsNamespaceEnabled(getFileSystem()), message); } /** @@ -699,7 +699,7 @@ protected void assumeHnsDisabled(String message) throws IOException { protected void assertPathDns(Path path) { String expectedDns = getAbfsServiceType() == AbfsServiceType.BLOB ? ABFS_BLOB_DOMAIN_NAME : ABFS_DFS_DOMAIN_NAME; - Assertions.assertThat(path.toString()) + assertThat(path.toString()) .describedAs("Path does not contain expected DNS") .contains(expectedDns); } @@ -745,19 +745,23 @@ protected void checkFuturesForExceptions(List> futures, int exceptionV protected void assumeRecoveryThroughClientTransactionID(boolean isCreate) throws IOException { // Assumes that recovery through client transaction ID is enabled. - Assume.assumeTrue("Recovery through client transaction ID is not enabled", - getConfiguration().getIsClientTransactionIdEnabled()); + assumeTrue(getConfiguration().getIsClientTransactionIdEnabled(), + "Recovery through client transaction ID is not enabled"); // Assumes that service type is DFS. assumeDfsServiceType(); // Assumes that namespace is enabled for the given AzureBlobFileSystem. assumeHnsEnabled(); if (isCreate) { // Assume that create client is DFS client. - Assume.assumeTrue("Ingress service type is not DFS", - AbfsServiceType.DFS.equals(getIngressServiceType())); + assumeTrue(AbfsServiceType.DFS.equals(getIngressServiceType()), + "Ingress service type is not DFS"); // Assume that append blob is not enabled in DFS client. - Assume.assumeFalse("Append blob is enabled in DFS client", - isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), + "Append blob is enabled in DFS client"); } } + + protected void assumeNotNull(Object objects) { + assumeTrue(objects != null); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java index 14c9bff7bf8d2..17be79b1bffc5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java @@ -18,18 +18,21 @@ package org.apache.hadoop.fs.azurebfs; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azure.integration.AzureTestConstants; +import static org.apache.hadoop.fs.azure.integration.AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS; import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled; /** * Integration tests at bigger scale; configurable as to * size, off by default. */ +@Timeout(SCALE_TEST_TIMEOUT_MILLIS) public class AbstractAbfsScaleTest extends AbstractAbfsIntegrationTest { protected static final Logger LOG = @@ -39,11 +42,6 @@ public AbstractAbfsScaleTest() throws Exception { super(); } - @Override - protected int getTestTimeoutMillis() { - return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS; - } - @Override public void setup() throws Exception { super.setup(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java index 0485422871ecc..ddd40ed3c6328 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java @@ -19,15 +19,15 @@ import java.io.IOException; -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.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.test.TestName; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; @@ -37,27 +37,22 @@ * Base class for any ABFS test with timeouts & named threads. * This class does not attempt to bind to Azure. */ -public class AbstractAbfsTestWithTimeout extends Assert { +@Timeout(TEST_TIMEOUT) +public class AbstractAbfsTestWithTimeout extends Assertions { private static final Logger LOG = LoggerFactory.getLogger(AbstractAbfsTestWithTimeout.class); /** * The name of the current method. */ - @Rule - 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()); + @RegisterExtension + protected TestName methodName = new TestName(); /** * 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"); } @@ -65,7 +60,7 @@ public static void nameTestThread() { /** * Name the thread to the current test method. */ - @Before + @BeforeEach public void nameThread() { Thread.currentThread().setName("JUnit-" + methodName.getMethodName()); } @@ -110,15 +105,15 @@ protected boolean validateContent(AzureBlobFileSystem fs, Path path, while (valueOfContentAtPos != -1 && pos < lenOfOriginalByteArray) { if (originalByteArray[pos] != valueOfContentAtPos) { - assertEquals("Mismatch in content validation at position {}", pos, - originalByteArray[pos], valueOfContentAtPos); + assertEquals(originalByteArray[pos], valueOfContentAtPos, + "Mismatch in content validation at position " + pos); return false; } valueOfContentAtPos = (byte) in.read(); pos++; } if (valueOfContentAtPos != -1) { - assertEquals("Expected end of file", -1, valueOfContentAtPos); + assertEquals(-1, valueOfContentAtPos, "Expected end of file"); return false; } return true; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestABFSJceksFiltering.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestABFSJceksFiltering.java index e1b6b39521acd..da32419f6aaab 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestABFSJceksFiltering.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestABFSJceksFiltering.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.security.alias.CredentialProviderFactory; import org.apache.hadoop.conf.Configuration; @@ -35,7 +35,7 @@ public void testIncompatibleCredentialProviderIsExcluded() throws Exception { rawConfig.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, "jceks://abfs@a@b.c.d/tmp/a.jceks,jceks://file/tmp/secret.jceks"); try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(rawConfig)) { - assertNotNull("filesystem", fs); + assertNotNull(fs, "filesystem"); String providers = fs.getConf().get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH); assertEquals("jceks://file/tmp/secret.jceks", providers); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index c180689b267ab..a911882420c8e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -27,10 +27,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -43,6 +41,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; /** * Test continuation token which has equal sign. @@ -55,7 +54,7 @@ public ITestAbfsClient() throws Exception { super(); } - @Ignore("HADOOP-16845: Invalid continuation tokens are ignored by the ADLS " + @Disabled("HADOOP-16845: Invalid continuation tokens are ignored by the ADLS " + "Gen2 service, so we are disabling this test until the service is fixed.") @Test public void testContinuationTokenHavingEqualSign() throws Exception { @@ -66,13 +65,13 @@ public void testContinuationTokenHavingEqualSign() throws Exception { AbfsRestOperation op = abfsClient .listPath("/", true, LIST_MAX_RESULTS, "===========", getTestTracingContext(fs, true), null).getOp(); - Assert.assertTrue(false); + assertTrue(false); } catch (AbfsRestOperationException ex) { - Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); + assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); } } - @Ignore("Enable this to verify the log warning message format for HostNotFoundException") + @Disabled("Enable this to verify the log warning message format for HostNotFoundException") @Test public void testUnknownHost() throws Exception { // When hitting hostName not found exception, the retry will take about 14 mins until failed. @@ -113,7 +112,7 @@ public void testListPathWithValidListMaxResultsValues() if (continuationToken == null) { // Listing is complete and number of objects should be same as expected - Assertions.assertThat(list) + assertThat(list) .describedAs("AbfsClient.listPath() should return %d items" + " when listMaxResults is %d, directory contains %d items and " + "listing is complete", @@ -121,7 +120,7 @@ public void testListPathWithValidListMaxResultsValues() .hasSize(expectedListResultsSize); } else { // Listing is incomplete and number of objects can be less than expected - Assertions.assertThat(list) + assertThat(list) .describedAs("AbfsClient.listPath() should return %d items" + " or less when listMaxResults is %d, directory contains" + " %d items and listing is incomplete", @@ -148,7 +147,7 @@ public void testListPathWithValueGreaterThanServerMaximum() if (continuationToken == null) { // Listing is complete and number of objects should be same as expected - Assertions.assertThat(list) + assertThat(list) .describedAs("AbfsClient.listPath() should return %d items" + " when listMaxResults is %d directory contains %d items and " + "listing is complete", LIST_MAX_RESULTS_SERVER, @@ -156,7 +155,7 @@ public void testListPathWithValueGreaterThanServerMaximum() .hasSize(LIST_MAX_RESULTS_SERVER); } else { // Listing is incomplete and number of objects can be less than expected - Assertions.assertThat(list) + assertThat(list) .describedAs("AbfsClient.listPath() should return %d items" + " or less when listMaxResults is %d, directory contains" + " %d items and listing is complete", LIST_MAX_RESULTS_SERVER, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 781cd701400f8..4dca80639b842 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -18,6 +18,9 @@ package org.apache.hadoop.fs.azurebfs; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -28,12 +31,6 @@ import java.util.List; import java.util.Random; -import org.assertj.core.api.Assertions; -import org.assertj.core.api.Assumptions; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -63,6 +60,8 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Lists; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CPK_IN_NON_HNS_ACCOUNT_ERROR_MESSAGE; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; @@ -85,55 +84,48 @@ import static org.apache.hadoop.fs.permission.AclEntryType.USER; import static org.apache.hadoop.fs.permission.FsAction.ALL; -@RunWith(Parameterized.class) public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest { public static final String SERVER_FILE_CONTENT = "123"; private final byte[] cpk = new byte[ENCRYPTION_KEY_LEN]; - private final String cpkSHAEncoded; + private String cpkSHAEncoded; private static final String BLOCK_ID = "MF8tNDE1MjkzOTE4AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"; private List fileSystemsOpenedInTest = new ArrayList<>(); // Encryption type used by filesystem while creating file - @Parameterized.Parameter public EncryptionType fileEncryptionType; // Encryption type used by filesystem to call different operations - @Parameterized.Parameter(1) public EncryptionType requestEncryptionType; - @Parameterized.Parameter(2) public FSOperationType operation; - @Parameterized.Parameter(3) public boolean responseHeaderServerEnc; - @Parameterized.Parameter(4) public boolean responseHeaderReqServerEnc; - @Parameterized.Parameter(5) public boolean isExceptionCase; /** * Boolean value to indicate that the server response would have header related * to CPK and the test would need to assert its value. */ - @Parameterized.Parameter(6) public boolean isCpkResponseHdrExpected; /** * Boolean value to indicate that the server response would have fields related * to CPK and the test would need to assert its value. */ - @Parameterized.Parameter(7) public Boolean isCpkResponseKeyExpected = false; - @Parameterized.Parameter(8) public Boolean fileSystemListStatusResultToBeUsedForOpeningFile = false; - @Parameterized.Parameters(name = "{0} mode, {2}") + protected ITestAbfsCustomEncryption() throws Exception { + super(); + } + public static Iterable params() { return Arrays.asList(new Object[][] { {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, false, false, true, false, false}, @@ -178,14 +170,35 @@ public static Iterable params() { }); } - public ITestAbfsCustomEncryption() throws Exception { + public void initITestAbfsCustomEncryption(EncryptionType pFileEncryptionType, + EncryptionType pRequestEncryptionType, FSOperationType pOperation, + boolean pResponseHeaderServerEnc, boolean pResponseHeaderReqServerEnc, + boolean pIsExceptionCase, boolean pIsCpkResponseHdrExpected, + boolean pFileSystemListStatusResultToBeUsedForOpeningFile) throws Exception { new Random().nextBytes(cpk); cpkSHAEncoded = EncodingHelper.getBase64EncodedString( EncodingHelper.getSHA256Hash(cpk)); + fileEncryptionType = pFileEncryptionType; + requestEncryptionType = pRequestEncryptionType; + operation = pOperation; + responseHeaderServerEnc = pResponseHeaderServerEnc; + responseHeaderReqServerEnc = pResponseHeaderReqServerEnc; + isExceptionCase = pIsExceptionCase; + isCpkResponseHdrExpected = pIsCpkResponseHdrExpected; + fileSystemListStatusResultToBeUsedForOpeningFile = + pFileSystemListStatusResultToBeUsedForOpeningFile; } - @Test - public void testCustomEncryptionCombinations() throws Exception { + @ParameterizedTest(name = "{0} mode, {2}") + @MethodSource("params") + public void testCustomEncryptionCombinations(EncryptionType pFileEncryptionType, + EncryptionType pRequestEncryptionType, FSOperationType pOperation, + boolean pResponseHeaderServerEnc, boolean pResponseHeaderReqServerEnc, + boolean pIsExceptionCase, boolean pIsCpkResponseHdrExpected, + boolean pFileSystemListStatusResultToBeUsedForOpeningFile) throws Exception { + initITestAbfsCustomEncryption(pFileEncryptionType, pRequestEncryptionType, + pOperation, pResponseHeaderServerEnc, pResponseHeaderReqServerEnc, pIsExceptionCase, + pIsCpkResponseHdrExpected, pFileSystemListStatusResultToBeUsedForOpeningFile); try (AzureBlobFileSystem fs = getOrCreateFS()) { validateCpkResponseHeadersForCombination(fs); } @@ -208,10 +221,10 @@ private void validateCpkResponseHeadersForCombination(final AzureBlobFileSystem String expectedKeySHA = EncodingHelper.getBase64EncodedString( EncodingHelper.getSHA256Hash( ecp.getEncryptionKeyForTest(encryptionContext))); - Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) + assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) .isEqualTo(expectedKeySHA); } else { // GLOBAL_KEY - Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) + assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) .isEqualTo(cpkSHAEncoded); } } else { @@ -221,18 +234,18 @@ private void validateCpkResponseHeadersForCombination(final AzureBlobFileSystem String expectedKeySHA = EncodingHelper.getBase64EncodedString( EncodingHelper.getSHA256Hash( ecp.getEncryptionKeyForTest(encryptionContext))); - Assertions.assertThat(httpOp.getListResultSchema().paths().get(0) + assertThat(httpOp.getListResultSchema().paths().get(0) .getCustomerProvidedKeySha256()).isEqualTo(expectedKeySHA); } } else { - Assertions.assertThat( + assertThat( httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) .isEqualTo(null); } } - Assertions.assertThat(httpOp.getResponseHeader(X_MS_SERVER_ENCRYPTED)) + assertThat(httpOp.getResponseHeader(X_MS_SERVER_ENCRYPTED)) .isEqualTo(responseHeaderServerEnc? "true" : null); - Assertions.assertThat(httpOp.getResponseHeader(X_MS_REQUEST_SERVER_ENCRYPTED)) + assertThat(httpOp.getResponseHeader(X_MS_REQUEST_SERVER_ENCRYPTED)) .isEqualTo(responseHeaderReqServerEnc? "true" : null); } @@ -301,10 +314,10 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, * 4.Assert for no exception and get response. */ FileStatus status = fs.listStatus(testPath)[0]; - Assertions.assertThat(status) + assertThat(status) .isInstanceOf(VersionedFileStatus.class); - Assertions.assertThat( + assertThat( ((VersionedFileStatus) status).getEncryptionContext()) .isNotNull(); @@ -312,8 +325,8 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, new OpenFileParameters().withMandatoryKeys(new HashSet<>()) .withStatus(fs.listStatus(testPath)[0])).get()) { byte[] readBuffer = new byte[3]; - Assertions.assertThat(in.read(readBuffer)).isGreaterThan(0); - Assertions.assertThat(readBuffer).isEqualTo(SERVER_FILE_CONTENT.getBytes()); + assertThat(in.read(readBuffer)).isGreaterThan(0); + assertThat(readBuffer).isEqualTo(SERVER_FILE_CONTENT.getBytes()); return null; } } @@ -424,7 +437,7 @@ private AzureBlobFileSystem getAzureBlobFileSystem(final Configuration conf) { fileSystemsOpenedInTest.add(fs); // Default for this config should be true here as FNS Accounts would have failed initialization. // This is needed to make sure test runs even if test config is missing. - Assertions.assertThat( + assertThat( getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, true)) .describedAs("Encryption tests should run only on namespace enabled account") @@ -434,14 +447,14 @@ private AzureBlobFileSystem getAzureBlobFileSystem(final Configuration conf) { GenericTestUtils.assertExceptionContains( CPK_IN_NON_HNS_ACCOUNT_ERROR_MESSAGE, ex, "Exception message should contain the expected message"); - Assertions.assertThat( + assertThat( getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false)) .describedAs("Encryption tests should run only on namespace enabled account") .isFalse(); //Skip the test - Assumptions.assumeThat(true).isFalse(); + assumeThat(true).isFalse(); return null; } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java index 0997b3dbd44d4..7c277bb00adb5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java @@ -20,8 +20,7 @@ import java.io.IOException; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +38,7 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.assertj.core.api.Assertions.assertThat; public class ITestAbfsDurationTrackers extends AbstractAbfsIntegrationTest { @@ -101,7 +101,7 @@ public void testAbfsHttpCallsDurations() throws IOException { */ private void assertDurationTracker(IOStatistics ioStatistics) { for (AbfsStatistic abfsStatistic : HTTP_DURATION_TRACKER_LIST) { - Assertions.assertThat(lookupMeanStatistic(ioStatistics, + assertThat(lookupMeanStatistic(ioStatistics, abfsStatistic.getStatName() + StoreStatisticNames.SUFFIX_MEAN).mean()) .describedAs("The DurationTracker Named " + abfsStatistic.getStatName() + " Doesn't match the expected value.") diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsHugeFiles.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsHugeFiles.java index 510e0a7596b47..12a1ca0b46382 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsHugeFiles.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsHugeFiles.java @@ -18,20 +18,16 @@ package org.apache.hadoop.fs.azurebfs; -import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.Random; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.store.DataBlocks; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import static org.apache.hadoop.fs.azure.integration.AzureTestConstants.AZURE_SCALE_HUGE_FILE_UPLOAD; import static org.apache.hadoop.fs.azure.integration.AzureTestConstants.AZURE_SCALE_HUGE_FILE_UPLOAD_DEFAULT; @@ -43,7 +39,6 @@ /** * Testing Huge file for AbfsOutputStream. */ -@RunWith(Parameterized.class) public class ITestAbfsHugeFiles extends AbstractAbfsScaleTest { private static final int ONE_MB = 1024 * 1024; private static final int EIGHT_MB = 8 * ONE_MB; @@ -62,8 +57,10 @@ public class ITestAbfsHugeFiles extends AbstractAbfsScaleTest { // Block Factory to be used in this test. private String blockFactoryName; - @Parameterized.Parameters(name = "size [{0}] ; blockFactoryName " - + "[{1}]") + public ITestAbfsHugeFiles() throws Exception { + super(); + } + public static Collection sizes() { return Arrays.asList(new Object[][] { { DEFAULT_WRITE_BUFFER_SIZE, DataBlocks.DATA_BLOCKS_BUFFER_DISK }, @@ -75,13 +72,13 @@ public static Collection sizes() { }); } - public ITestAbfsHugeFiles(int size, String blockFactoryName) + public void initITestAbfsHugeFiles(int pSize, String pBlockFactoryName) throws Exception { - this.size = size; - this.blockFactoryName = blockFactoryName; + this.size = pSize; + this.blockFactoryName = pBlockFactoryName; + setUp(); } - @Before public void setUp() throws Exception { Configuration configuration = getRawConfiguration(); configuration.unset(DATA_BLOCKS_BUFFER); @@ -92,8 +89,10 @@ public void setUp() throws Exception { /** * Testing Huge files written at once on AbfsOutputStream. */ - @Test - public void testHugeFileWrite() throws IOException { + @ParameterizedTest(name = "size [{0}] ; blockFactoryName [{1}]") + @MethodSource("sizes") + public void testHugeFileWrite(int pSize, String pBlockFactoryName) throws Exception { + initITestAbfsHugeFiles(pSize, pBlockFactoryName); AzureBlobFileSystem fs = getFileSystem(); Path filePath = path(getMethodName()); final byte[] b = new byte[size]; @@ -103,15 +102,17 @@ public void testHugeFileWrite() throws IOException { } // Verify correct length was uploaded. Don't want to verify contents // here, as this would increase the test time significantly. - assertEquals("Mismatch in content length of file uploaded", size, - fs.getFileStatus(filePath).getLen()); + assertEquals(size, fs.getFileStatus(filePath).getLen(), + "Mismatch in content length of file uploaded"); } /** * Testing Huge files written in chunks of 8M in lots of writes. */ - @Test - public void testLotsOfWrites() throws IOException { + @ParameterizedTest(name = "size [{0}] ; blockFactoryName [{1}]") + @MethodSource("sizes") + public void testLotsOfWrites(int pSize, String pBlockFactoryName) throws Exception { + initITestAbfsHugeFiles(pSize, pBlockFactoryName); assume("If the size isn't a multiple of 8M this test would not pass, so " + "skip", size % EIGHT_MB == 0); @@ -128,7 +129,7 @@ public void testLotsOfWrites() throws IOException { } // Verify correct length was uploaded. Don't want to verify contents // here, as this would increase the test time significantly. - assertEquals("Mismatch in content length of file uploaded", size, - fs.getFileStatus(filePath).getLen()); + assertEquals(size, fs.getFileStatus(filePath).getLen(), + "Mismatch in content length of file uploaded"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsIdentityTransformer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsIdentityTransformer.java index 5868d083e12e9..181efb07fe441 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsIdentityTransformer.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsIdentityTransformer.java @@ -25,7 +25,7 @@ import org.apache.hadoop.util.Lists; import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer; import org.apache.hadoop.fs.permission.AclEntry; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; @@ -73,8 +73,8 @@ public void testDaemonServiceSettingIdentity() throws IOException { resetIdentityConfig(config); // Default config IdentityTransformer identityTransformer = getTransformerWithDefaultIdentityConfig(config); - assertEquals("Identity should not change for default config", - DAEMON, identityTransformer.transformUserOrGroupForSetRequest(DAEMON)); + assertEquals(DAEMON, identityTransformer.transformUserOrGroupForSetRequest(DAEMON), + "Identity should not change for default config"); // Add service principal id config.set(FS_AZURE_OVERRIDE_OWNER_SP, SERVICE_PRINCIPAL_ID); @@ -82,20 +82,20 @@ public void testDaemonServiceSettingIdentity() throws IOException { // case 1: substitution list doesn't contain daemon config.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, "a,b,c,d"); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("Identity should not change when substitution list doesn't contain daemon", - DAEMON, identityTransformer.transformUserOrGroupForSetRequest(DAEMON)); + assertEquals(DAEMON, identityTransformer.transformUserOrGroupForSetRequest(DAEMON), + "Identity should not change when substitution list doesn't contain daemon"); // case 2: substitution list contains daemon name config.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, DAEMON + ",a,b,c,d"); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("Identity should be replaced to servicePrincipalId", - SERVICE_PRINCIPAL_ID, identityTransformer.transformUserOrGroupForSetRequest(DAEMON)); + assertEquals(SERVICE_PRINCIPAL_ID, identityTransformer.transformUserOrGroupForSetRequest(DAEMON), + "Identity should be replaced to servicePrincipalId"); // case 3: substitution list is * config.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, ASTERISK); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("Identity should be replaced to servicePrincipalId", - SERVICE_PRINCIPAL_ID, identityTransformer.transformUserOrGroupForSetRequest(DAEMON)); + assertEquals(SERVICE_PRINCIPAL_ID, identityTransformer.transformUserOrGroupForSetRequest(DAEMON), + "Identity should be replaced to servicePrincipalId"); } @Test @@ -103,8 +103,8 @@ public void testFullyQualifiedNameSettingIdentity() throws IOException { Configuration config = this.getRawConfiguration(); // Default config IdentityTransformer identityTransformer = getTransformerWithDefaultIdentityConfig(config); - assertEquals("short name should not be converted to full name by default", - SHORT_NAME, identityTransformer.transformUserOrGroupForSetRequest(SHORT_NAME)); + assertEquals(SHORT_NAME, identityTransformer.transformUserOrGroupForSetRequest(SHORT_NAME), + "short name should not be converted to full name by default"); resetIdentityConfig(config); @@ -112,8 +112,8 @@ public void testFullyQualifiedNameSettingIdentity() throws IOException { config.setBoolean(FS_AZURE_FILE_OWNER_ENABLE_SHORTNAME, true); config.set(FS_AZURE_FILE_OWNER_DOMAINNAME, DOMAIN); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("short name should be converted to full name", - FULLY_QUALIFIED_NAME, identityTransformer.transformUserOrGroupForSetRequest(SHORT_NAME)); + assertEquals(FULLY_QUALIFIED_NAME, identityTransformer.transformUserOrGroupForSetRequest(SHORT_NAME), + "short name should be converted to full name"); } @Test @@ -128,8 +128,8 @@ public void testNoOpForSettingOidAsIdentity() throws IOException { IdentityTransformer identityTransformer = getTransformerWithCustomizedIdentityConfig(config); final String principalId = UUID.randomUUID().toString(); - assertEquals("Identity should not be changed when owner is already a principal id ", - principalId, identityTransformer.transformUserOrGroupForSetRequest(principalId)); + assertEquals(principalId, identityTransformer.transformUserOrGroupForSetRequest(principalId), + "Identity should not be changed when owner is already a principal id "); } @Test @@ -141,8 +141,8 @@ public void testNoOpWhenSettingSuperUserAsdentity() throws IOException { config.set(FS_AZURE_FILE_OWNER_DOMAINNAME, DOMAIN); // Default config IdentityTransformer identityTransformer = getTransformerWithDefaultIdentityConfig(config); - assertEquals("Identity should not be changed because it is not in substitution list", - SUPER_USER, identityTransformer.transformUserOrGroupForSetRequest(SUPER_USER)); + assertEquals(SUPER_USER, identityTransformer.transformUserOrGroupForSetRequest(SUPER_USER), + "Identity should not be changed because it is not in substitution list"); } @Test @@ -152,14 +152,14 @@ public void testIdentityReplacementForSuperUserGetRequest() throws IOException { // with default config, identityTransformer should do $superUser replacement IdentityTransformer identityTransformer = getTransformerWithDefaultIdentityConfig(config); - assertEquals("$superuser should be replaced with local user by default", - localUser, identityTransformer.transformIdentityForGetRequest(SUPER_USER, true, localUser)); + assertEquals(localUser, identityTransformer.transformIdentityForGetRequest(SUPER_USER, true, localUser), + "$superuser should be replaced with local user by default"); // Disable $supeuser replacement config.setBoolean(FS_AZURE_SKIP_SUPER_USER_REPLACEMENT, true); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("$superuser should not be replaced", - SUPER_USER, identityTransformer.transformIdentityForGetRequest(SUPER_USER, true, localUser)); + assertEquals( + SUPER_USER, identityTransformer.transformIdentityForGetRequest(SUPER_USER, true, localUser), "$superuser should not be replaced"); } @Test @@ -169,47 +169,53 @@ public void testIdentityReplacementForDaemonServiceGetRequest() throws IOExcepti // Default config IdentityTransformer identityTransformer = getTransformerWithDefaultIdentityConfig(config); - assertEquals("By default servicePrincipalId should not be converted for GetFileStatus(), listFileStatus(), getAcl()", - SERVICE_PRINCIPAL_ID, identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser)); + assertEquals(SERVICE_PRINCIPAL_ID, + identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser), + "By default servicePrincipalId should not be converted for GetFileStatus(), listFileStatus(), getAcl()"); resetIdentityConfig(config); // 1. substitution list doesn't contain currentUser config.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, "a,b,c,d"); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("servicePrincipalId should not be replaced if local daemon user is not in substitution list", - SERVICE_PRINCIPAL_ID, identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser)); + assertEquals(SERVICE_PRINCIPAL_ID, + identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser), + "servicePrincipalId should not be replaced if local daemon user is not in substitution list"); resetIdentityConfig(config); // 2. substitution list contains currentUser(daemon name) but the service principal id in config doesn't match config.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, localUser + ",a,b,c,d"); config.set(FS_AZURE_OVERRIDE_OWNER_SP, UUID.randomUUID().toString()); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("servicePrincipalId should not be replaced if it is not equal to the SPN set in config", - SERVICE_PRINCIPAL_ID, identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser)); + assertEquals(SERVICE_PRINCIPAL_ID, + identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser), + "servicePrincipalId should not be replaced if it is not equal to the SPN set in config"); resetIdentityConfig(config); // 3. substitution list contains currentUser(daemon name) and the service principal id in config matches config.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, localUser + ",a,b,c,d"); config.set(FS_AZURE_OVERRIDE_OWNER_SP, SERVICE_PRINCIPAL_ID); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("servicePrincipalId should be transformed to local use", - localUser, identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser)); + assertEquals(localUser, + identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser), + "servicePrincipalId should be transformed to local use"); resetIdentityConfig(config); // 4. substitution is "*" but the service principal id in config doesn't match the input config.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, ASTERISK); config.set(FS_AZURE_OVERRIDE_OWNER_SP, UUID.randomUUID().toString()); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("servicePrincipalId should not be replaced if it is not equal to the SPN set in config", - SERVICE_PRINCIPAL_ID, identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser)); + assertEquals(SERVICE_PRINCIPAL_ID, + identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser), + "servicePrincipalId should not be replaced if it is not equal to the SPN set in config"); resetIdentityConfig(config); // 5. substitution is "*" and the service principal id in config match the input config.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, ASTERISK); config.set(FS_AZURE_OVERRIDE_OWNER_SP, SERVICE_PRINCIPAL_ID); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("servicePrincipalId should be transformed to local user", - localUser, identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser)); + assertEquals(localUser, + identityTransformer.transformIdentityForGetRequest(SERVICE_PRINCIPAL_ID, true, localUser), + "servicePrincipalId should be transformed to local user"); } @Test @@ -219,17 +225,20 @@ public void testIdentityReplacementForKinitUserGetRequest() throws IOException { // Default config IdentityTransformer identityTransformer = getTransformerWithDefaultIdentityConfig(config); - assertEquals("full name should not be transformed if shortname is not enabled", - FULLY_QUALIFIED_NAME, identityTransformer.transformIdentityForGetRequest(FULLY_QUALIFIED_NAME, true, localUser)); + assertEquals(FULLY_QUALIFIED_NAME, + identityTransformer.transformIdentityForGetRequest(FULLY_QUALIFIED_NAME, true, localUser), + "full name should not be transformed if shortname is not enabled"); // add config to get short name config.setBoolean(FS_AZURE_FILE_OWNER_ENABLE_SHORTNAME, true); identityTransformer = getTransformerWithCustomizedIdentityConfig(config); - assertEquals("should convert the full owner name to shortname ", - SHORT_NAME, identityTransformer.transformIdentityForGetRequest(FULLY_QUALIFIED_NAME, true, localUser)); + assertEquals(SHORT_NAME, + identityTransformer.transformIdentityForGetRequest(FULLY_QUALIFIED_NAME, true, localUser), + "should convert the full owner name to shortname "); - assertEquals("group name should not be converted to shortname ", - FULLY_QUALIFIED_NAME, identityTransformer.transformIdentityForGetRequest(FULLY_QUALIFIED_NAME, false, localGroup)); + assertEquals(FULLY_QUALIFIED_NAME, + identityTransformer.transformIdentityForGetRequest(FULLY_QUALIFIED_NAME, false, localGroup), + "group name should not be converted to shortname "); } @Test @@ -350,9 +359,9 @@ private IdentityTransformer getTransformerWithCustomizedIdentityConfig(Configura } private void checkAclEntriesList(List aclEntries, List expected) { - assertTrue("list size not equals", aclEntries.size() == expected.size()); + assertTrue(aclEntries.size() == expected.size(), "list size not equals"); for (int i = 0; i < aclEntries.size(); i++) { - assertEquals("Identity doesn't match", expected.get(i).getName(), aclEntries.get(i).getName()); + assertEquals(expected.get(i).getName(), aclEntries.get(i).getName(), "Identity doesn't match"); } } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index afc92c111a913..6a309459045ee 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -20,8 +20,7 @@ import java.io.IOException; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,6 +36,7 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.assertj.core.api.Assertions.assertThat; public class ITestAbfsInputStreamStatistics extends AbstractAbfsIntegrationTest { @@ -160,18 +160,17 @@ public void testSeekStatistics() throws IOException { * would be equal to OPERATIONS. * */ - assertEquals("Mismatch in seekOps value", 2 * OPERATIONS, - stats.getSeekOperations()); - assertEquals("Mismatch in backwardSeekOps value", OPERATIONS, - stats.getBackwardSeekOperations()); - assertEquals("Mismatch in forwardSeekOps value", OPERATIONS, - stats.getForwardSeekOperations()); - assertEquals("Mismatch in bytesBackwardsOnSeek value", - OPERATIONS * ONE_MB, stats.getBytesBackwardsOnSeek()); - assertEquals("Mismatch in bytesSkippedOnSeek value", - 0, stats.getBytesSkippedOnSeek()); - assertEquals("Mismatch in seekInBuffer value", OPERATIONS, - stats.getSeekInBuffer()); + assertEquals(2 * OPERATIONS, stats.getSeekOperations(), + "Mismatch in seekOps value"); + assertEquals(OPERATIONS, + stats.getBackwardSeekOperations(), "Mismatch in backwardSeekOps value"); + assertEquals(OPERATIONS, + stats.getForwardSeekOperations(), "Mismatch in forwardSeekOps value"); + assertEquals(OPERATIONS * ONE_MB, stats.getBytesBackwardsOnSeek(), + "Mismatch in bytesBackwardsOnSeek value"); + assertEquals(0, stats.getBytesSkippedOnSeek(), "Mismatch in bytesSkippedOnSeek value"); + assertEquals(OPERATIONS, + stats.getSeekInBuffer(), "Mismatch in seekInBuffer value"); in.close(); // Verifying whether stats are readable after stream is closed. @@ -230,12 +229,12 @@ public void testReadStatistics() throws IOException { * total remote read ops is 1. * */ - assertEquals("Mismatch in bytesRead value", OPERATIONS, - stats.getBytesRead()); - assertEquals("Mismatch in readOps value", OPERATIONS, - stats.getReadOperations()); - assertEquals("Mismatch in remoteReadOps value", 1, - stats.getRemoteReadOperations()); + assertEquals(OPERATIONS, + stats.getBytesRead(), "Mismatch in bytesRead value"); + assertEquals(OPERATIONS, + stats.getReadOperations(), "Mismatch in readOps value"); + assertEquals(1, + stats.getRemoteReadOperations(), "Mismatch in remoteReadOps value"); in.close(); // Verifying if stats are still readable after stream is closed. @@ -288,8 +287,9 @@ public void testWithNullStreamStatistics() throws IOException { getTestTracingContext(fs, false)); // Verifying that AbfsInputStream Operations works with null statistics. - assertNotEquals("AbfsInputStream read() with null statistics should " - + "work", -1, in.read()); + assertNotEquals(-1, in.read(), + "AbfsInputStream read() with null statistics should " + + "work"); in.seek(ONE_KB); // Verifying toString() with no StreamStatistics. @@ -364,11 +364,11 @@ public void testReadAheadCounters() throws IOException { * value that we measure at some point of the operation. * */ - Assertions.assertThat(stats.getReadAheadBytesRead()).describedAs( + assertThat(stats.getReadAheadBytesRead()).describedAs( "Mismatch in readAheadBytesRead counter value") .isGreaterThanOrEqualTo(in.getBytesFromReadAhead()); - Assertions.assertThat(stats.getRemoteBytesRead()).describedAs( + assertThat(stats.getRemoteBytesRead()).describedAs( "Mismatch in remoteBytesRead counter value") .isGreaterThanOrEqualTo(in.getBytesFromRemoteRead()); @@ -402,7 +402,7 @@ public void testActionHttpGetRequest() throws IOException { IOStatistics ioStatistics = extractStatistics(fs); LOG.info("AbfsInputStreamStats info: {}", ioStatisticsToPrettyString(ioStatistics)); - Assertions.assertThat( + assertThat( lookupMeanStatistic(ioStatistics, AbfsStatistic.HTTP_GET_REQUEST.getStatName() + StoreStatisticNames.SUFFIX_MEAN).mean()) @@ -420,6 +420,6 @@ public void testActionHttpGetRequest() throws IOException { * @param statistic the name of operation or statistic being asserted. */ private void checkInitValue(long actualValue, String statistic) { - assertEquals("Mismatch in " + statistic + " value", 0, actualValue); + assertEquals(0, actualValue, "Mismatch in " + statistic + " value"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java index 6f7fe0ce2cab2..4f2ab70863003 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java @@ -29,10 +29,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,12 +41,17 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.test.LambdaTestUtils; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; + /** * Test ListStatusRemoteIterator operation. */ @@ -68,10 +70,10 @@ public void testAbfsIteratorWithHasNext() throws Exception { setPageSize(10); final List fileNames = createFilesUnderDirectory(testDir); - ListingSupport listingSupport = Mockito.spy(getFileSystem().getAbfsStore()); + ListingSupport listingSupport = spy(getFileSystem().getAbfsStore()); RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, listingSupport, getTestTracingContext(getFileSystem(), true)); - Assertions.assertThat(fsItr) + assertThat(fsItr) .describedAs("RemoteIterator should be instance of " + "AbfsListStatusRemoteIterator by default") .isInstanceOf(AbfsListStatusRemoteIterator.class); @@ -83,7 +85,7 @@ public void testAbfsIteratorWithHasNext() throws Exception { } verifyIteratorResultCount(itrCount, fileNames); int minNumberOfInvocations = TEST_FILES_NUMBER / 10; - verify(listingSupport, Mockito.atLeast(minNumberOfInvocations)) + verify(listingSupport, atLeast(minNumberOfInvocations)) .listStatus(any(Path.class), nullable(String.class), anyList(), anyBoolean(), nullable(String.class), @@ -96,10 +98,10 @@ public void testAbfsIteratorWithoutHasNext() throws Exception { setPageSize(10); final List fileNames = createFilesUnderDirectory(testDir); - ListingSupport listingSupport = Mockito.spy(getFileSystem().getAbfsStore()); + ListingSupport listingSupport = spy(getFileSystem().getAbfsStore()); RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, listingSupport, getTestTracingContext(getFileSystem(), true)); - Assertions.assertThat(fsItr) + assertThat(fsItr) .describedAs("RemoteIterator should be instance of " + "AbfsListStatusRemoteIterator by default") .isInstanceOf(AbfsListStatusRemoteIterator.class); @@ -112,7 +114,7 @@ public void testAbfsIteratorWithoutHasNext() throws Exception { LambdaTestUtils.intercept(NoSuchElementException.class, fsItr::next); verifyIteratorResultCount(itrCount, fileNames); int minNumberOfInvocations = TEST_FILES_NUMBER / 10; - verify(listingSupport, Mockito.atLeast(minNumberOfInvocations)) + verify(listingSupport, atLeast(minNumberOfInvocations)) .listStatus(any(Path.class), nullable(String.class), anyList(), anyBoolean(), nullable(String.class), @@ -128,7 +130,7 @@ public void testWithAbfsIteratorDisabled() throws Exception { RemoteIterator fsItr = getFileSystem().listStatusIterator(testDir); - Assertions.assertThat(fsItr) + assertThat(fsItr) .describedAs("RemoteIterator should not be instance of " + "AbfsListStatusRemoteIterator when it is disabled") .isNotInstanceOf(AbfsListStatusRemoteIterator.class); @@ -150,7 +152,7 @@ public void testWithAbfsIteratorDisabledWithoutHasNext() throws Exception { RemoteIterator fsItr = getFileSystem().listStatusIterator( testDir); - Assertions.assertThat(fsItr).describedAs( + assertThat(fsItr).describedAs( "RemoteIterator should not be instance of " + "AbfsListStatusRemoteIterator when it is disabled") .isNotInstanceOf(AbfsListStatusRemoteIterator.class); @@ -170,8 +172,8 @@ public void testNextWhenNoMoreElementsPresent() throws Exception { RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, getFileSystem().getAbfsStore(), getTestTracingContext(getFileSystem(), true)); - fsItr = Mockito.spy(fsItr); - Mockito.doReturn(false).when(fsItr).hasNext(); + fsItr = spy(fsItr); + doReturn(false).when(fsItr).hasNext(); LambdaTestUtils.intercept(NoSuchElementException.class, fsItr::next); } @@ -182,7 +184,7 @@ public void testHasNextForEmptyDir() throws Exception { setPageSize(10); RemoteIterator fsItr = getFileSystem() .listStatusIterator(testDir); - Assertions.assertThat(fsItr.hasNext()) + assertThat(fsItr.hasNext()) .describedAs("hasNext returns false for empty directory") .isFalse(); } @@ -195,9 +197,9 @@ public void testHasNextForFile() throws Exception { getFileSystem().create(testFile); setPageSize(10); RemoteIterator fsItr = fs.listStatusIterator(testFile); - Assertions.assertThat(fsItr.hasNext()) + assertThat(fsItr.hasNext()) .describedAs("hasNext returns true for file").isTrue(); - Assertions.assertThat(fsItr.next().getPath().toString()) + assertThat(fsItr.next().getPath().toString()) .describedAs("next returns the file itself") .endsWith(testFileName); } @@ -227,16 +229,15 @@ private void verifyIteratorResultContent(FileStatus fileStatus, List fileNames) { assertPathDns(fileStatus.getPath()); String pathStr = fileStatus.getPath().toString(); - Assert.assertTrue( - String.format("Could not remove path %s from filenames %s", pathStr, - fileNames), fileNames.remove(pathStr)); + assertTrue(fileNames.remove(pathStr), + String.format("Could not remove path %s from filenames %s", pathStr, fileNames)); } private void verifyIteratorResultCount(int itrCount, List fileNames) { - Assertions.assertThat(itrCount).describedAs( + assertThat(itrCount).describedAs( "Number of iterations should be equal to the files created") .isEqualTo(TEST_FILES_NUMBER); - Assertions.assertThat(fileNames) + assertThat(fileNames) .describedAs("After removing every item found from the iterator, " + "there should be no more elements in the fileNames") .hasSize(0); @@ -291,7 +292,7 @@ private List createFilesUnderDirectory(Path rootPath) tasks.add(es.submit(() -> { touch(filePath); synchronized (fileNames) { - Assert.assertTrue(fileNames.add(filePath.toString())); + assertTrue(fileNames.add(filePath.toString())); } return null; })); @@ -303,7 +304,7 @@ private List createFilesUnderDirectory(Path rootPath) es.shutdownNow(); } LOG.debug(fileNames.toString()); - Assertions.assertThat(fileNames) + assertThat(fileNames) .describedAs("File creation incorrect or fileNames not added to list") .hasSize(ITestAbfsListStatusRemoteIterator.TEST_FILES_NUMBER); return fileNames; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java index d871befa43005..901e3fb090020 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java @@ -21,18 +21,13 @@ import java.io.IOException; import java.util.Date; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; import org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider; -import static org.junit.Assume.assumeThat; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.Matchers.isEmptyOrNullString; -import static org.hamcrest.Matchers.isEmptyString; import static org.apache.hadoop.fs.azurebfs.constants.AuthConfigurations.DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY; import static org.apache.hadoop.fs.azurebfs.constants.AuthConfigurations.DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT; @@ -40,6 +35,8 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; /** * Test MsiTokenProvider. @@ -54,14 +51,10 @@ public ITestAbfsMsiTokenProvider() throws Exception { @Test public void test() throws IOException { AbfsConfiguration conf = getConfiguration(); - assumeThat(conf.get(FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT), - not(isEmptyOrNullString())); - assumeThat(conf.get(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT), - not(isEmptyOrNullString())); - assumeThat(conf.get(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID), - not(isEmptyOrNullString())); - assumeThat(conf.get(FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY), - not(isEmptyOrNullString())); + assumeThat(conf.get(FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT)).isNotEmpty(); + assumeThat(conf.get(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT)).isNotEmpty(); + assumeThat(conf.get(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID)).isNotEmpty(); + assumeThat(conf.get(FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY)).isNotEmpty(); String tenantGuid = conf .getPasswordString(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT); @@ -77,8 +70,8 @@ public void test() throws IOException { AzureADToken token = null; token = tokenProvider.getToken(); - assertThat(token.getAccessToken(), not(isEmptyString())); - assertThat(token.getExpiry().after(new Date()), is(true)); + assertThat(token.getAccessToken()).isNotEqualTo(""); + assertThat(token.getExpiry().after(new Date())).isEqualTo(true); } private String getTrimmedPasswordString(AbfsConfiguration conf, String key, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java index e66afbcaa7492..e29bfc5f624b2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java @@ -23,7 +23,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java index 8be997ce69cf3..b962afbb81788 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java @@ -20,8 +20,7 @@ import java.io.IOException; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +33,7 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.assertj.core.api.Assertions.assertThat; /** * Test AbfsOutputStream statistics. @@ -67,8 +67,8 @@ public void testAbfsOutputStreamUploadingBytes() throws IOException { getAbfsOutputStreamStatistics(outForSomeBytes); //Test for zero bytes To upload. - assertEquals("Mismatch in bytes to upload", 0, - abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload()); + assertEquals(0, abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload(), + "Mismatch in bytes to upload"); outForSomeBytes.write(testBytesToUpload.getBytes()); outForSomeBytes.flush(); @@ -76,14 +76,14 @@ public void testAbfsOutputStreamUploadingBytes() throws IOException { getAbfsOutputStreamStatistics(outForSomeBytes); //Test for bytes to upload. - assertEquals("Mismatch in bytes to upload", - testBytesToUpload.getBytes().length, - abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload()); + assertEquals(testBytesToUpload.getBytes().length, + abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload(), + "Mismatch in bytes to upload"); //Test for successful bytes uploaded. - assertEquals("Mismatch in successful bytes uploaded", - testBytesToUpload.getBytes().length, - abfsOutputStreamStatisticsForUploadBytes.getBytesUploadSuccessful()); + assertEquals(testBytesToUpload.getBytes().length, + abfsOutputStreamStatisticsForUploadBytes.getBytesUploadSuccessful(), + "Mismatch in successful bytes uploaded"); } @@ -99,14 +99,14 @@ public void testAbfsOutputStreamUploadingBytes() throws IOException { getAbfsOutputStreamStatistics(outForLargeBytes); //Test for bytes to upload. - assertEquals("Mismatch in bytes to upload", - OPERATIONS * (testBytesToUpload.getBytes().length), - abfsOutputStreamStatistics.getBytesToUpload()); + assertEquals(OPERATIONS * (testBytesToUpload.getBytes().length), + abfsOutputStreamStatistics.getBytesToUpload(), + "Mismatch in bytes to upload"); //Test for successful bytes uploaded. - assertEquals("Mismatch in successful bytes uploaded", - OPERATIONS * (testBytesToUpload.getBytes().length), - abfsOutputStreamStatistics.getBytesUploadSuccessful()); + assertEquals(OPERATIONS * (testBytesToUpload.getBytes().length), + abfsOutputStreamStatistics.getBytesUploadSuccessful(), + "Mismatch in successful bytes uploaded"); } } @@ -137,8 +137,8 @@ public void testAbfsOutputStreamQueueShrink() throws IOException { getAbfsOutputStreamStatistics(outForOneOp); //Test for shrinking queue zero time. - assertEquals("Mismatch in queue shrunk operations", 0, - abfsOutputStreamStatistics.getQueueShrunkOps()); + assertEquals(0, abfsOutputStreamStatistics.getQueueShrunkOps(), + "Mismatch in queue shrunk operations"); } @@ -168,9 +168,9 @@ public void testAbfsOutputStreamQueueShrink() throws IOException { * write operations done to get the number of queue shrinks done. * */ - assertEquals("Mismatch in queue shrunk operations", - OPERATIONS - outForLargeOps.getWriteOperationsSize(), - abfsOutputStreamStatistics.getQueueShrunkOps()); + assertEquals(OPERATIONS - outForLargeOps.getWriteOperationsSize(), + abfsOutputStreamStatistics.getQueueShrunkOps(), + "Mismatch in queue shrunk operations"); } } @@ -196,8 +196,9 @@ public void testAbfsOutputStreamWriteBuffer() throws IOException { getAbfsOutputStreamStatistics(outForOneOp); //Test for zero time writing buffer to service. - assertEquals("Mismatch in write current buffer operations", 0, - abfsOutputStreamStatistics.getWriteCurrentBufferOperations()); + assertEquals(0, + abfsOutputStreamStatistics.getWriteCurrentBufferOperations(), + "Mismatch in write current buffer operations"); outForOneOp.write(testWriteBuffer.getBytes()); outForOneOp.flush(); @@ -205,8 +206,9 @@ public void testAbfsOutputStreamWriteBuffer() throws IOException { abfsOutputStreamStatistics = getAbfsOutputStreamStatistics(outForOneOp); //Test for one time writing buffer to service. - assertEquals("Mismatch in write current buffer operations", 1, - abfsOutputStreamStatistics.getWriteCurrentBufferOperations()); + assertEquals(1, + abfsOutputStreamStatistics.getWriteCurrentBufferOperations(), + "Mismatch in write current buffer operations"); } try ( @@ -225,9 +227,9 @@ public void testAbfsOutputStreamWriteBuffer() throws IOException { AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics = getAbfsOutputStreamStatistics(outForLargeOps); //Test for 10 times writing buffer to service. - assertEquals("Mismatch in write current buffer operations", - OPERATIONS, - abfsOutputStreamStatistics.getWriteCurrentBufferOperations()); + assertEquals(OPERATIONS, + abfsOutputStreamStatistics.getWriteCurrentBufferOperations(), + "Mismatch in write current buffer operations"); } } @@ -250,7 +252,7 @@ public void testAbfsOutputStreamDurationTrackerPutRequest() throws IOException { IOStatistics ioStatistics = extractStatistics(fs); LOG.info("AbfsOutputStreamStats info: {}", ioStatisticsToPrettyString(ioStatistics)); - Assertions.assertThat( + assertThat( lookupMeanStatistic(ioStatistics, AbfsStatistic.HTTP_PUT_REQUEST.getStatName() + StoreStatisticNames.SUFFIX_MEAN).mean()) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java index ad4b0b1049d6d..d7a60884b5cb1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java @@ -38,13 +38,13 @@ import static org.apache.hadoop.fs.azurebfs.enums.AbfsReadFooterMetricsEnum.TOTAL_FILES; import static org.apache.hadoop.fs.azurebfs.enums.AbfsReadFooterMetricsEnum.AVG_FIRST_OFFSET_DIFF; import static org.apache.hadoop.fs.azurebfs.enums.AbfsReadFooterMetricsEnum.AVG_SECOND_OFFSET_DIFF; +import static org.junit.jupiter.api.Assumptions.assumeTrue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.utils.MetricFormat; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Random; @@ -73,8 +73,8 @@ private void checkPrerequisites(){ private void checkIfConfigIsSet(String configKey){ AbfsConfiguration conf = getConfiguration(); String value = conf.get(configKey); - Assume.assumeTrue(configKey + " config is mandatory for the test to run", - value != null && value.trim().length() > 1); + assumeTrue(value != null && value.trim().length() > 1, + configKey + " config is mandatory for the test to run"); } private static final String TEST_PATH = "/testfile"; @@ -127,7 +127,7 @@ private void writeDataToFile(AzureBlobFileSystem fs, Path testPath, byte[] data) */ private void assertMetricsEquality(AzureBlobFileSystem fs, String expectedMetrics) { AbfsReadFooterMetrics actualMetrics = fs.getAbfsClient().getAbfsCounters().getAbfsReadFooterMetrics(); - assertNotNull("AbfsReadFooterMetrics is null", actualMetrics); + assertNotNull(actualMetrics, "AbfsReadFooterMetrics is null"); assertEquals("The computed metrics differs from the actual metrics", expectedMetrics, actualMetrics.toString()); } @@ -188,7 +188,7 @@ public void testReadFooterMetrics() throws Exception { IOSTATISTICS_LOGGING_LEVEL_INFO, statisticsSource); // Ensure data is read successfully and matches the written data. - assertNotEquals("data read in final read()", -1, result); + assertNotEquals(-1, result, "data read in final read()"); assertArrayEquals(readBuffer, b); // Get non-Parquet metrics and assert metrics equality. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index c32c0147fe7da..90c5cb0d7f86f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -21,10 +21,6 @@ import java.util.Arrays; import java.util.Random; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -34,6 +30,8 @@ import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; @@ -47,16 +45,18 @@ * Uses package-private methods in AbfsConfiguration, which is why it is in * this package. */ -@RunWith(Parameterized.class) public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest { private static final String TEST_PATH = "/testfile"; + public ITestAbfsReadWriteAndSeek() throws Exception { + super(); + } + /** * Parameterize on read buffer size and readahead. * For test performance, a full x*y test matrix is not used. * @return the test parameters */ - @Parameterized.Parameters(name = "Size={0}-readahead={1}-Client={2}") public static Iterable sizes() { return Arrays.asList(new Object[][]{ { @@ -112,19 +112,23 @@ public static Iterable sizes() { }); } - private final int size; - private final boolean readaheadEnabled; - private final HttpOperationType httpOperationType; + private int size; + private boolean readaheadEnabled; + private HttpOperationType httpOperationType; - public ITestAbfsReadWriteAndSeek(final int size, - final boolean readaheadEnabled, final HttpOperationType httpOperationType) throws Exception { - this.size = size; - this.readaheadEnabled = readaheadEnabled; - this.httpOperationType = httpOperationType; + public void initITestAbfsReadWriteAndSeek(int pSize, + boolean pReadaheadEnabled, HttpOperationType pHttpOperationType) + throws Exception { + this.size = pSize; + this.readaheadEnabled = pReadaheadEnabled; + this.httpOperationType = pHttpOperationType; } - @Test - public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception { + @ParameterizedTest(name = "Size={0}-readahead={1}-Client={2}") + @MethodSource("sizes") + public void testReadAndWriteWithDifferentBufferSizesAndSeek(int pSize, + boolean pReadaheadEnabled, HttpOperationType pHttpOperationType) throws Exception { + initITestAbfsReadWriteAndSeek(pSize, pReadaheadEnabled, pHttpOperationType); testReadWriteAndSeek(size); } @@ -172,12 +176,15 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { } logIOStatisticsAtLevel(LOG, IOSTATISTICS_LOGGING_LEVEL_INFO, statisticsSource); - assertNotEquals("data read in final read()", -1, result); + assertNotEquals(-1, result, "data read in final read()"); assertArrayEquals(readBuffer, b); } - @Test - public void testReadAheadRequestID() throws java.io.IOException { + @ParameterizedTest(name = "Size={0}-readahead={1}-Client={2}") + @MethodSource("sizes") + public void testReadAheadRequestID(int pSize, + boolean pReadaheadEnabled, HttpOperationType pHttpOperationType) throws Exception { + initITestAbfsReadWriteAndSeek(pSize, pReadaheadEnabled, pHttpOperationType); final AzureBlobFileSystem fs = getFileSystem(); final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); int bufferSize = MIN_BUFFER_SIZE; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java index 0151faa5f412e..92f8af694b07d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java @@ -20,8 +20,7 @@ import java.io.IOException; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType; @@ -40,6 +39,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; /** * Verify the AbfsRestOperationException error message format. @@ -64,35 +64,35 @@ public void testAbfsRestOperationExceptionFormat() throws IOException { String[] errorFields = errorMessage.split(","); // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId) - Assertions.assertThat(errorFields) + assertThat(errorFields) .describedAs( "Number of Fields in exception message are not as expected") .hasSize(5); // Check status message, status code, HTTP Request Type and URL. if (getAbfsServiceType() == AbfsServiceType.BLOB) { - Assertions.assertThat(errorFields[0].trim()) + assertThat(errorFields[0].trim()) .describedAs("Error Message Field in exception message is wrong") .contains( "Operation failed: \"The specified blob does not exist.\""); } else { - Assertions.assertThat(errorFields[0].trim()) + assertThat(errorFields[0].trim()) .describedAs("Error Message Field in exception message is wrong") .isEqualTo( "Operation failed: \"The specified path does not exist.\""); } - Assertions.assertThat(errorFields[1].trim()) + assertThat(errorFields[1].trim()) .describedAs("Status Code Field in exception message " + "should be \"404\"") .isEqualTo("404"); - Assertions.assertThat(errorFields[2].trim()) + assertThat(errorFields[2].trim()) .describedAs("Http Rest Method Field in exception message " + "should be \"HEAD\"") .isEqualTo("HEAD"); - Assertions.assertThat(errorFields[3].trim()) + assertThat(errorFields[3].trim()) .describedAs("Url Field in exception message" + " should start with \"http\"") .startsWith("http"); - Assertions.assertThat(errorFields[4].trim()) + assertThat(errorFields[4].trim()) .describedAs("ActivityId Field in exception message " + "should start with \"rId:\"") .startsWith("rId:"); @@ -106,76 +106,76 @@ public void testAbfsRestOperationExceptionFormat() throws IOException { if (getAbfsServiceType() == AbfsServiceType.DFS) { // verify its format // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId), StorageErrorCode, StorageErrorMessage. - Assertions.assertThat(errorFields) + assertThat(errorFields) .describedAs( "Number of Fields in exception message are not as expected") .hasSize(7); - Assertions.assertThat(errorFields[0].trim()) + assertThat(errorFields[0].trim()) .describedAs("Error Message Field in exception message is wrong") .isEqualTo( "Operation failed: \"The specified path does not exist.\""); - Assertions.assertThat(errorFields[1].trim()) + assertThat(errorFields[1].trim()) .describedAs("Status Code Field in exception message" + " should be \"404\"") .isEqualTo("404"); - Assertions.assertThat(errorFields[2].trim()) + assertThat(errorFields[2].trim()) .describedAs("Http Rest Method Field in exception message" + " should be \"GET\"") .isEqualTo("GET"); - Assertions.assertThat(errorFields[3].trim()) + assertThat(errorFields[3].trim()) .describedAs("Url Field in exception message" + " should start with \"http\"") .startsWith("http"); - Assertions.assertThat(errorFields[4].trim()) + assertThat(errorFields[4].trim()) .describedAs("ActivityId Field in exception message" + " should start with \"rId:\"") .startsWith("rId:"); // Check storage error code and storage error message. - Assertions.assertThat(errorFields[5].trim()) + assertThat(errorFields[5].trim()) .describedAs("StorageErrorCode Field in exception message" + " should be \"PathNotFound\"") .isEqualTo("PathNotFound"); - Assertions.assertThat(errorFields[6].trim()) + assertThat(errorFields[6].trim()) .describedAs("StorageErrorMessage Field in exception message" + " should contain \"RequestId\"") .contains("RequestId"); - Assertions.assertThat(errorFields[6].trim()) + assertThat(errorFields[6].trim()) .describedAs("StorageErrorMessage Field in exception message" + " should contain \"Time\"") .contains("Time"); } else { // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId) - Assertions.assertThat(errorFields) + assertThat(errorFields) .describedAs( "Number of Fields in exception message are not as expected") .hasSize(5); // Check status message, status code, HTTP Request Type and URL. if (getAbfsStore(fs).getAbfsConfiguration().enableAbfsListIterator()) { - Assertions.assertThat(errorFields[0].trim()) + assertThat(errorFields[0].trim()) .describedAs( "Error Message Field in exception message is wrong") .contains( "Operation failed: \"The specified container does not exist.\""); } else { - Assertions.assertThat(errorFields[0].trim()) + assertThat(errorFields[0].trim()) .describedAs( "Error Message Field in exception message is wrong") .contains( "Operation failed: \"The specified blob does not exist.\""); } - Assertions.assertThat(errorFields[1].trim()) + assertThat(errorFields[1].trim()) .describedAs("Status Code Field in exception message " + "should be \"404\"") .isEqualTo("404"); - Assertions.assertThat(errorFields[2].trim()) + assertThat(errorFields[2].trim()) .describedAs("Http Rest Method Field in exception message " + "should be \"HEAD\"") .isEqualTo("HEAD"); - Assertions.assertThat(errorFields[3].trim()) + assertThat(errorFields[3].trim()) .describedAs("Url Field in exception message" + " should start with \"http\"") .startsWith("http"); - Assertions.assertThat(errorFields[4].trim()) + assertThat(errorFields[4].trim()) .describedAs("ActivityId Field in exception message " + "should start with \"rId:\"") .startsWith("rId:"); @@ -206,7 +206,7 @@ public void testWithDifferentCustomTokenFetchRetry(int numOfRetries) throws Exce }); // Number of retries done should be as configured - Assertions.assertThat(retryTestTokenProvider.getRetryCount()) + assertThat(retryTestTokenProvider.getRetryCount()) .describedAs("Number of token fetch retries done does not " + "match with fs.azure.custom.token.fetch.retry.count configured") .isEqualTo(numOfRetries); @@ -222,12 +222,12 @@ public void testAuthFailException() throws Exception { }); String errorDesc = "Should throw RestOp exception on AAD failure"; - Assertions.assertThat(e.getStatusCode()) + assertThat(e.getStatusCode()) .describedAs("Incorrect status code: " + errorDesc).isEqualTo(-1); - Assertions.assertThat(e.getErrorCode()) + assertThat(e.getErrorCode()) .describedAs("Incorrect error code: " + errorDesc) .isEqualTo(AzureServiceErrorCode.UNKNOWN); - Assertions.assertThat(e.getErrorMessage()) + assertThat(e.getErrorMessage()) .describedAs("Incorrect error message: " + errorDesc) .contains("Auth failure: "); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java index 8f692b4477ba4..c48ada71b7309 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.util.Map; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; @@ -42,7 +42,7 @@ public class ITestAbfsStatistics extends AbstractAbfsIntegrationTest { public ITestAbfsStatistics() throws Exception { } - @Before + @BeforeEach public void setUp() throws Exception { super.setup(); // Setting IOStats to INFO level, to see the IOStats after close(). @@ -209,12 +209,10 @@ public void testOpenAppendRenameExists() throws IOException { assertAbfsStatistics(AbfsStatistic.CALL_RENAME, 1, metricMap); //Testing if file exists at path. - assertTrue(String.format("File with name %s should exist", - destCreateFilePath), - fs.exists(destCreateFilePath)); - assertFalse(String.format("File with name %s should not exist", - createFilePath), - fs.exists(createFilePath)); + assertTrue(fs.exists(destCreateFilePath), String.format("File with name %s should exist", + destCreateFilePath)); + assertFalse(fs.exists(createFilePath), String.format("File with name %s should not exist", + createFilePath)); metricMap = fs.getInstrumentationMap(); //Testing exists() calls. @@ -242,12 +240,12 @@ public void testOpenAppendRenameExists() throws IOException { assertTrue(fs.rename(createFilePath, destCreateFilePath)); //check if first name is existing and 2nd is not existing. - assertTrue(String.format("File with name %s should exist", - destCreateFilePath), - fs.exists(destCreateFilePath)); - assertFalse(String.format("File with name %s should not exist", - createFilePath), - fs.exists(createFilePath)); + assertTrue(fs.exists(destCreateFilePath), + String.format("File with name %s should exist", + destCreateFilePath)); + assertFalse(fs.exists(createFilePath), + String.format("File with name %s should not exist", + createFilePath)); } @@ -273,6 +271,6 @@ Testing exists() calls and rename calls. Since both were called 2 */ private void checkInitialValue(String statName, long statValue, long expectedInitialValue) { - assertEquals("Mismatch in " + statName, expectedInitialValue, statValue); + assertEquals(expectedInitialValue, statValue, "Mismatch in " + statName); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java index f62ced9b00ba6..7c5afb2396ae3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,9 +95,9 @@ public void testAbfsStreamOps() throws Exception { * different setups. * */ - assertTrue(String.format("The actual value of %d was not equal to the " - + "expected value of 2 or 3", statistics.getReadOps()), - statistics.getReadOps() == 2 || statistics.getReadOps() == 3); + assertTrue(statistics.getReadOps() == 2 || statistics.getReadOps() == 3, + String.format("The actual value of %d was not equal to the " + + "expected value of 2 or 3", statistics.getReadOps())); } finally { IOUtils.cleanupWithLogger(LOG, inForOneOperation, @@ -105,9 +105,8 @@ public void testAbfsStreamOps() throws Exception { } //Validating if content is being written in the smallOperationsFile - assertTrue("Mismatch in content validation", - validateContent(fs, smallOperationsFile, - testReadWriteOps.getBytes())); + assertTrue(validateContent(fs, smallOperationsFile, + testReadWriteOps.getBytes()), "Mismatch in content validation"); FSDataOutputStream outForLargeOperations = null; FSDataInputStream inForLargeOperations = null; @@ -137,9 +136,10 @@ public void testAbfsStreamOps() throws Exception { if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(largeOperationsFile).toString())) { // for appendblob data is already flushed, so there might be more data to read. - assertTrue(String.format("The actual value of %d was not equal to the " - + "expected value", statistics.getReadOps()), - statistics.getReadOps() >= largeValue || statistics.getReadOps() <= (largeValue + 4)); + assertTrue(statistics.getReadOps() >= largeValue + || statistics.getReadOps() <= (largeValue + 4), + String.format("The actual value of %d was not equal to the " + + "expected value", statistics.getReadOps())); } else { //Test for 1000000 read operations assertReadWriteOps("read", largeValue, statistics.getReadOps()); @@ -150,9 +150,9 @@ public void testAbfsStreamOps() throws Exception { outForLargeOperations); } //Validating if content is being written in largeOperationsFile - assertTrue("Mismatch in content validation", - validateContent(fs, largeOperationsFile, - largeOperationsValidationString.toString().getBytes())); + assertTrue(validateContent(fs, largeOperationsFile, + largeOperationsValidationString.toString().getBytes()), + "Mismatch in content validation"); } @@ -166,7 +166,7 @@ public void testAbfsStreamOps() throws Exception { private void assertReadWriteOps(String operation, long expectedValue, long actualValue) { - assertEquals("Mismatch in " + operation + " operations", expectedValue, - actualValue); + assertEquals(expectedValue, actualValue, + "Mismatch in " + operation + " operations"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index 920c4964a559f..232c6fbf0e020 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -34,10 +34,7 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; @@ -86,7 +83,22 @@ import static org.apache.hadoop.fs.store.DataBlocks.DataBlock.DestState.Closed; import static org.apache.hadoop.fs.store.DataBlocks.DataBlock.DestState.Writing; import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.mockito.ArgumentMatchers.anyString; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * Test append operations. @@ -108,12 +120,14 @@ public ITestAzureBlobFileSystemAppend() throws Exception { super(); } - @Test(expected = FileNotFoundException.class) + @Test public void testAppendDirShouldFail() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Path filePath = path(TEST_FILE_PATH); - fs.mkdirs(filePath); - fs.append(filePath, 0).close(); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Path filePath = path(TEST_FILE_PATH); + fs.mkdirs(filePath); + fs.append(filePath, 0).close(); + }); } @Test @@ -128,22 +142,25 @@ public void testAppendWithLength0() throws Exception { } - @Test(expected = FileNotFoundException.class) + @Test public void testAppendFileAfterDelete() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Path filePath = path(TEST_FILE_PATH); - ContractTestUtils.touch(fs, filePath); - fs.delete(filePath, false); - - fs.append(filePath).close(); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Path filePath = path(TEST_FILE_PATH); + ContractTestUtils.touch(fs, filePath); + fs.delete(filePath, false); + fs.append(filePath).close(); + }); } - @Test(expected = FileNotFoundException.class) + @Test public void testAppendDirectory() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Path folderPath = path(TEST_FOLDER_PATH); - fs.mkdirs(folderPath); - fs.append(folderPath).close(); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Path folderPath = path(TEST_FOLDER_PATH); + fs.mkdirs(folderPath); + fs.append(folderPath).close(); + }); } @Test @@ -166,36 +183,36 @@ public void testCloseOfDataBlockOnAppendComplete() throws Exception { for (String blockBufferType : blockBufferTypes) { Configuration configuration = new Configuration(getRawConfiguration()); configuration.set(DATA_BLOCKS_BUFFER, blockBufferType); - try (AzureBlobFileSystem fs = Mockito.spy( + try (AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(configuration))) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - Mockito.doReturn(store).when(fs).getAbfsStore(); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + doReturn(store).when(fs).getAbfsStore(); DataBlocks.DataBlock[] dataBlock = new DataBlocks.DataBlock[1]; - Mockito.doAnswer(getBlobFactoryInvocation -> { - DataBlocks.BlockFactory factory = Mockito.spy( + doAnswer(getBlobFactoryInvocation -> { + DataBlocks.BlockFactory factory = spy( (DataBlocks.BlockFactory) getBlobFactoryInvocation.callRealMethod()); - Mockito.doAnswer(factoryCreateInvocation -> { - dataBlock[0] = Mockito.spy( + doAnswer(factoryCreateInvocation -> { + dataBlock[0] = spy( (DataBlocks.DataBlock) factoryCreateInvocation.callRealMethod()); return dataBlock[0]; }) .when(factory) - .create(Mockito.anyLong(), Mockito.anyInt(), Mockito.any( + .create(anyLong(), anyInt(), any( BlockUploadStatistics.class)); return factory; }).when(store).getBlockFactory(); try (OutputStream os = fs.create( new Path(getMethodName() + "_" + blockBufferType))) { os.write(new byte[1]); - Assertions.assertThat(dataBlock[0].getState()) + assertThat(dataBlock[0].getState()) .describedAs( "On write of data in outputStream, state should become Writing") .isEqualTo(Writing); os.close(); - Mockito.verify(dataBlock[0], Mockito.times(1)).close(); - Assertions.assertThat(dataBlock[0].getState()) + verify(dataBlock[0], times(1)).close(); + assertThat(dataBlock[0].getState()) .describedAs( - "On close of outputStream, state should become Closed") + "On close of outputStream, state should become Closed") .isEqualTo(Closed); } } @@ -210,7 +227,7 @@ public void testCloseOfDataBlockOnAppendComplete() throws Exception { */ @Test public void testCreateOverDfsAppendOverBlob() throws IOException { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); final AzureBlobFileSystem fs = getFileSystem(); Path testPath = path(TEST_FILE_PATH); AzureBlobFileSystemStore.Permissions permissions @@ -227,7 +244,7 @@ public void testCreateOverDfsAppendOverBlob() throws IOException { AzureIngressHandler ingressHandler = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler(); AbfsClient client = ingressHandler.getClient(); - Assertions.assertThat(client) + assertThat(client) .as("Blob client was not used before fallback") .isInstanceOf(AbfsBlobClient.class); outputStream.write(TEN); @@ -239,7 +256,7 @@ public void testCreateOverDfsAppendOverBlob() throws IOException { AzureIngressHandler ingressHandlerFallback = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler(); AbfsClient clientFallback = ingressHandlerFallback.getClient(); - Assertions.assertThat(clientFallback) + assertThat(clientFallback) .as("DFS client was not used after fallback") .isInstanceOf(AbfsDfsClient.class); } @@ -249,7 +266,7 @@ public void testCreateOverDfsAppendOverBlob() throws IOException { */ @Test public void testMultipleAppendsQualifyForSwitch() throws Exception { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); final AzureBlobFileSystem fs = getFileSystem(); Path testPath = path(TEST_FILE_PATH); AzureBlobFileSystemStore.Permissions permissions @@ -304,7 +321,7 @@ public void testMultipleAppendsQualifyForSwitch() throws Exception { AzureIngressHandler ingressHandlerFallback = ((AbfsOutputStream) out1.getWrappedStream()).getIngressHandler(); AbfsClient clientFallback = ingressHandlerFallback.getClient(); - Assertions.assertThat(clientFallback) + assertThat(clientFallback) .as("DFS client was not used after fallback") .isInstanceOf(AbfsDfsClient.class); } @@ -314,7 +331,7 @@ public void testMultipleAppendsQualifyForSwitch() throws Exception { */ @Test public void testParallelWritesOnDfsAndBlob() throws Exception { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); final AzureBlobFileSystem fs = getFileSystem(); Path testPath = path(TEST_FILE_PATH); Path testPath1 = path(TEST_FILE_PATH1); @@ -371,7 +388,7 @@ public void testParallelWritesOnDfsAndBlob() throws Exception { */ @Test public void testCreateOverBlobAppendOverDfs() throws IOException { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); assumeDfsServiceType(); Configuration conf = getRawConfiguration(); conf.setBoolean(FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK, true); @@ -418,10 +435,10 @@ public void testCreateAppendBlobOverBlobEndpointAppendOverDfs() conf.setBoolean(FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK, true); conf.set(FS_AZURE_INGRESS_SERVICE_TYPE, String.valueOf(AbfsServiceType.DFS)); - try (AzureBlobFileSystem fs = Mockito.spy( + try (AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(conf))) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - Mockito.doReturn(true).when(store).isAppendBlobKey(anyString()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + doReturn(true).when(store).isAppendBlobKey(anyString()); // Set abfsStore as our mocked value. Field privateField = AzureBlobFileSystem.class.getDeclaredField( @@ -463,9 +480,9 @@ public void testCreateAppendBlobOverBlobEndpointAppendOverDfs() public void testCreateAppendBlobOverDfsEndpointAppendOverBlob() throws IOException, NoSuchFieldException, IllegalAccessException { assumeHnsEnabled("FNS does not support append blob creation for DFS endpoint"); - final AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - Mockito.doReturn(true).when(store).isAppendBlobKey(anyString()); + final AzureBlobFileSystem fs = spy(getFileSystem()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + doReturn(true).when(store).isAppendBlobKey(anyString()); // Set abfsStore as our mocked value. Field privateField = AzureBlobFileSystem.class.getDeclaredField( @@ -487,7 +504,7 @@ public void testCreateAppendBlobOverDfsEndpointAppendOverBlob() AzureIngressHandler ingressHandler = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler(); AbfsClient client = ingressHandler.getClient(); - Assertions.assertThat(client) + assertThat(client) .as("Blob client was not used before fallback") .isInstanceOf(AbfsBlobClient.class); outputStream.write(TEN); @@ -499,7 +516,7 @@ public void testCreateAppendBlobOverDfsEndpointAppendOverBlob() AzureIngressHandler ingressHandlerFallback = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler(); AbfsClient clientFallback = ingressHandlerFallback.getClient(); - Assertions.assertThat(clientFallback) + assertThat(clientFallback) .as("DFS client was not used after fallback") .isInstanceOf(AbfsDfsClient.class); } @@ -529,11 +546,11 @@ public void testValidateIngressHandler() throws IOException { FSDataOutputStream outputStream = fs.append(testPath); AzureIngressHandler ingressHandler = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler(); - Assertions.assertThat(ingressHandler) + assertThat(ingressHandler) .as("Blob Ingress handler instance is not correct") .isInstanceOf(AzureBlobIngressHandler.class); AbfsClient client = ingressHandler.getClient(); - Assertions.assertThat(client) + assertThat(client) .as("Blob client was not used correctly") .isInstanceOf(AbfsBlobClient.class); @@ -549,86 +566,96 @@ public void testValidateIngressHandler() throws IOException { FSDataOutputStream outputStream1 = fs.append(testPath1); AzureIngressHandler ingressHandler1 = ((AbfsOutputStream) outputStream1.getWrappedStream()).getIngressHandler(); - Assertions.assertThat(ingressHandler1) + assertThat(ingressHandler1) .as("DFS Ingress handler instance is not correct") .isInstanceOf(AzureDFSIngressHandler.class); AbfsClient client1 = ingressHandler1.getClient(); - Assertions.assertThat(client1) + assertThat(client1) .as("Dfs client was not used correctly") .isInstanceOf(AbfsDfsClient.class); } } - @Test(expected = FileNotFoundException.class) + @Test public void testAppendImplicitDirectory() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Path folderPath = new Path(TEST_FOLDER_PATH); - fs.mkdirs(folderPath); - fs.append(folderPath.getParent()); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Path folderPath = new Path(TEST_FOLDER_PATH); + fs.mkdirs(folderPath); + fs.append(folderPath.getParent()); + }); } - @Test(expected = FileNotFoundException.class) + @Test public void testAppendFileNotExists() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Path folderPath = new Path(TEST_FOLDER_PATH); - fs.append(folderPath); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Path folderPath = new Path(TEST_FOLDER_PATH); + fs.append(folderPath); + }); } /** * Create directory over dfs endpoint and append over blob endpoint. * Should return error as append is not supported for directory. * **/ - @Test(expected = IOException.class) + @Test public void testCreateExplicitDirectoryOverDfsAppendOverBlob() - throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); - final Path folderPath = path(TEST_FOLDER_PATH); - AzureBlobFileSystemStore.Permissions permissions - = new AzureBlobFileSystemStore.Permissions(false, - FsPermission.getDefault(), FsPermission.getUMask(fs.getConf())); - fs.getAbfsStore().getClientHandler().getDfsClient(). - createPath(makeQualified(folderPath).toUri().getPath(), false, false, - permissions, false, null, - null, getTestTracingContext(fs, true)); - FSDataOutputStream outputStream = fs.append(folderPath); - outputStream.write(TEN); - outputStream.hsync(); + throws IOException { + assertThrows(IOException.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Path folderPath = path(TEST_FOLDER_PATH); + AzureBlobFileSystemStore.Permissions permissions = + new AzureBlobFileSystemStore.Permissions(false, + FsPermission.getDefault(), FsPermission.getUMask(fs.getConf())); + fs.getAbfsStore().getClientHandler().getDfsClient(). + createPath(makeQualified(folderPath).toUri().getPath(), false, false, + permissions, false, null, + null, getTestTracingContext(fs, true)); + FSDataOutputStream outputStream = fs.append(folderPath); + outputStream.write(TEN); + outputStream.hsync(); + }); } /** * Recreate file between append and flush. Etag mismatch happens. **/ - @Test(expected = IOException.class) + @Test public void testRecreateAppendAndFlush() throws IOException { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); - final AzureBlobFileSystem fs = getFileSystem(); - final Path filePath = path(TEST_FILE_PATH); - fs.create(filePath); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB); - FSDataOutputStream outputStream = fs.append(filePath); - outputStream.write(TEN); - try (AzureBlobFileSystem fs1 - = (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()); - FSDataOutputStream outputStream1 = fs1.create(filePath)) { - outputStream.hsync(); - } - } + assertThrows(IOException.class, () -> { + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); + final AzureBlobFileSystem fs = getFileSystem(); + final Path filePath = path(TEST_FILE_PATH); + fs.create(filePath); + assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB); + FSDataOutputStream outputStream = fs.append(filePath); + outputStream.write(TEN); + try (AzureBlobFileSystem fs1 = + (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()); + FSDataOutputStream outputStream1 = fs1.create(filePath)) { + outputStream.hsync(); + } + }); + } /** * Recreate directory between append and flush. Etag mismatch happens. **/ - @Test(expected = IOException.class) + @Test public void testRecreateDirectoryAppendAndFlush() throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); - final Path filePath = path(TEST_FILE_PATH); - fs.create(filePath); - FSDataOutputStream outputStream = fs.append(filePath); - outputStream.write(TEN); - try (AzureBlobFileSystem fs1 - = (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration())) { - fs1.mkdirs(filePath); - outputStream.hsync(); - } + assertThrows(IOException.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Path filePath = path(TEST_FILE_PATH); + fs.create(filePath); + FSDataOutputStream outputStream = fs.append(filePath); + outputStream.write(TEN); + try (AzureBlobFileSystem fs1 = + (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration())) { + fs1.mkdirs(filePath); + outputStream.hsync(); + } + }); } /** @@ -735,14 +762,14 @@ public void testParallelWriteDifferentContentLength() throws Exception { **/ @Test public void testParallelWriteOutputStreamClose() throws Exception { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); AzureBlobFileSystem fs = getFileSystem(); final Path secondarytestfile = new Path("secondarytestfile"); ExecutorService executorService = Executors.newFixedThreadPool(2); List> futures = new ArrayList<>(); FSDataOutputStream out1 = fs.create(secondarytestfile); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB); + assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB); AbfsOutputStream outputStream1 = (AbfsOutputStream) out1.getWrappedStream(); String fileETag = outputStream1.getIngressHandler().getETag(); final byte[] b1 = new byte[8 * ONE_MB]; @@ -806,12 +833,12 @@ public void testParallelWriteOutputStreamClose() throws Exception { **/ @Test public void testEtagMismatch() throws Exception { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); AzureBlobFileSystem fs = getFileSystem(); final Path filePath = path(TEST_FILE_PATH); FSDataOutputStream out1 = fs.create(filePath); FSDataOutputStream out2 = fs.create(filePath); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB); + assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB); out2.write(TEN); out2.hsync(); out1.write(TEN); @@ -822,7 +849,7 @@ public void testEtagMismatch() throws Exception { public void testAppendWithLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE_PATH); - final AzureBlobFileSystem fs = Mockito.spy( + final AzureBlobFileSystem fs = spy( getCustomFileSystem(testFilePath.getParent(), 1)); FsPermission permission = new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL); @@ -863,26 +890,26 @@ public void testAppendImplicitDirectoryAzcopy() throws Exception { */ @Test public void testIntermittentAppendFailureToBeReported() throws Exception { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); - try (AzureBlobFileSystem fs = Mockito.spy( + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); + try (AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) { assumeHnsDisabled(); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); assumeBlobServiceType(); - AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler()); - AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient()); + AbfsClientHandler clientHandler = spy(store.getClientHandler()); + AbfsBlobClient blobClient = spy(clientHandler.getBlobClient()); - Mockito.doReturn(clientHandler).when(store).getClientHandler(); - Mockito.doReturn(blobClient).when(clientHandler).getBlobClient(); - Mockito.doReturn(blobClient).when(clientHandler).getIngressClient(); + doReturn(clientHandler).when(store).getClientHandler(); + doReturn(blobClient).when(clientHandler).getBlobClient(); + doReturn(blobClient).when(clientHandler).getIngressClient(); - Mockito.doThrow( + doThrow( new AbfsRestOperationException(HTTP_UNAVAILABLE, "", "", new Exception())) .when(blobClient) - .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any( - AppendRequestParameters.class), Mockito.any(), Mockito.any(), - Mockito.any(TracingContext.class)); + .append(anyString(), any(byte[].class), any( + AppendRequestParameters.class), any(), any(), + any(TracingContext.class)); byte[] bytes = new byte[1024 * 1024 * 8]; new Random().nextBytes(bytes); @@ -944,14 +971,14 @@ public void testIntermittentAppendFailureToBeReported() throws Exception { private FSDataOutputStream createMockedOutputStream(AzureBlobFileSystem fs, Path path, AbfsClient client) throws IOException { - AbfsOutputStream abfsOutputStream = Mockito.spy( + AbfsOutputStream abfsOutputStream = spy( (AbfsOutputStream) fs.create(path).getWrappedStream()); - AzureIngressHandler ingressHandler = Mockito.spy( + AzureIngressHandler ingressHandler = spy( abfsOutputStream.getIngressHandler()); - Mockito.doReturn(ingressHandler).when(abfsOutputStream).getIngressHandler(); - Mockito.doReturn(client).when(ingressHandler).getClient(); + doReturn(ingressHandler).when(abfsOutputStream).getIngressHandler(); + doReturn(client).when(ingressHandler).getClient(); - FSDataOutputStream fsDataOutputStream = Mockito.spy( + FSDataOutputStream fsDataOutputStream = spy( new FSDataOutputStream(abfsOutputStream, null)); return fsDataOutputStream; } @@ -963,30 +990,30 @@ private FSDataOutputStream createMockedOutputStream(AzureBlobFileSystem fs, */ @Test public void testWriteAsyncOpFailedAfterCloseCalled() throws Exception { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); - try (AzureBlobFileSystem fs = Mockito.spy( + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); + try (AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler()); - AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient()); - AbfsDfsClient dfsClient = Mockito.spy(clientHandler.getDfsClient()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + AbfsClientHandler clientHandler = spy(store.getClientHandler()); + AbfsBlobClient blobClient = spy(clientHandler.getBlobClient()); + AbfsDfsClient dfsClient = spy(clientHandler.getDfsClient()); AbfsClient client = clientHandler.getIngressClient(); if (clientHandler.getIngressClient() instanceof AbfsBlobClient) { - Mockito.doReturn(blobClient).when(clientHandler).getBlobClient(); - Mockito.doReturn(blobClient).when(clientHandler).getIngressClient(); + doReturn(blobClient).when(clientHandler).getBlobClient(); + doReturn(blobClient).when(clientHandler).getIngressClient(); } else { - Mockito.doReturn(dfsClient).when(clientHandler).getDfsClient(); - Mockito.doReturn(dfsClient).when(clientHandler).getIngressClient(); + doReturn(dfsClient).when(clientHandler).getDfsClient(); + doReturn(dfsClient).when(clientHandler).getIngressClient(); } - Mockito.doReturn(clientHandler).when(store).getClientHandler(); + doReturn(clientHandler).when(store).getClientHandler(); byte[] bytes = new byte[1024 * 1024 * 8]; new Random().nextBytes(bytes); AtomicInteger count = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { count.incrementAndGet(); while (count.get() < 2) { // No operation inside the loop @@ -996,11 +1023,11 @@ public void testWriteAsyncOpFailedAfterCloseCalled() throws Exception { new Exception()); }) .when(client instanceof AbfsBlobClient ? blobClient : dfsClient) - .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any( - AppendRequestParameters.class), Mockito.any(), Mockito.any(), - Mockito.any(TracingContext.class)); + .append(anyString(), any(byte[].class), any( + AppendRequestParameters.class), any(), any(), + any(TracingContext.class)); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { count.incrementAndGet(); while (count.get() < 2) { // No operation inside the loop @@ -1010,9 +1037,9 @@ public void testWriteAsyncOpFailedAfterCloseCalled() throws Exception { new Exception()); }) .when(client instanceof AbfsBlobClient ? blobClient : dfsClient) - .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any( - AppendRequestParameters.class), Mockito.any(), Mockito.any(), - Mockito.any(TracingContext.class)); + .append(anyString(), any(byte[].class), any( + AppendRequestParameters.class), any(), any(), + any(TracingContext.class)); FSDataOutputStream os = createMockedOutputStream(fs, new Path("/test/file"), @@ -1067,23 +1094,23 @@ private String generateBlockId(AbfsOutputStream os, long position) { */ @Test public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Exception { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); // Create a spy of AzureBlobFileSystem - try (AzureBlobFileSystem fs = Mockito.spy( + try (AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) { assumeHnsDisabled(); // Create a spy of AzureBlobFileSystemStore - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); assumeBlobServiceType(); // Create spies for the client handler and blob client - AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler()); - AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient()); + AbfsClientHandler clientHandler = spy(store.getClientHandler()); + AbfsBlobClient blobClient = spy(clientHandler.getBlobClient()); // Set up the spies to return the mocked objects - Mockito.doReturn(clientHandler).when(store).getClientHandler(); - Mockito.doReturn(blobClient).when(clientHandler).getBlobClient(); - Mockito.doReturn(blobClient).when(clientHandler).getIngressClient(); + doReturn(clientHandler).when(store).getClientHandler(); + doReturn(blobClient).when(clientHandler).getBlobClient(); + doReturn(blobClient).when(clientHandler).getIngressClient(); AtomicInteger flushCount = new AtomicInteger(0); FSDataOutputStream os = createMockedOutputStream(fs, new Path("/test/file"), blobClient); @@ -1096,56 +1123,56 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep String blockId = generateBlockId(out, 0); String blockListXml = generateBlockListXml(blockId); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { // Set up the mock for the flush operation AbfsClientTestUtil.setMockAbfsRestOperationForFlushOperation(blobClient, eTag, blockListXml, (httpOperation) -> { - Mockito.doAnswer(invocation -> { + doAnswer(invocation -> { // Call the real processResponse method invocation.callRealMethod(); int currentCount = flushCount.incrementAndGet(); if (currentCount == 1) { - Mockito.when(httpOperation.getStatusCode()) + when(httpOperation.getStatusCode()) .thenReturn( HTTP_INTERNAL_ERROR); // Status code 500 for Internal Server Error - Mockito.when(httpOperation.getStorageErrorMessage()) + when(httpOperation.getStorageErrorMessage()) .thenReturn("CONNECTION_RESET"); // Error message throw new IOException("Connection Reset"); } return null; }).when(httpOperation).processResponse( - Mockito.nullable(byte[].class), - Mockito.anyInt(), - Mockito.anyInt() + nullable(byte[].class), + anyInt(), + anyInt() ); return httpOperation; }); return answer.callRealMethod(); }).when(blobClient).flush( - Mockito.any(byte[].class), - Mockito.anyString(), - Mockito.anyBoolean(), - Mockito.nullable(String.class), - Mockito.nullable(String.class), - Mockito.anyString(), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.any(TracingContext.class) + any(byte[].class), + anyString(), + anyBoolean(), + nullable(String.class), + nullable(String.class), + anyString(), + nullable(ContextEncryptionAdapter.class), + any(TracingContext.class) ); out.hsync(); out.close(); - Mockito.verify(blobClient, Mockito.times(1)).flush( - Mockito.any(byte[].class), - Mockito.anyString(), - Mockito.anyBoolean(), - Mockito.nullable(String.class), - Mockito.nullable(String.class), - Mockito.anyString(), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.any(TracingContext.class)); + verify(blobClient, times(1)).flush( + any(byte[].class), + anyString(), + anyBoolean(), + nullable(String.class), + nullable(String.class), + anyString(), + nullable(ContextEncryptionAdapter.class), + any(TracingContext.class)); } } @@ -1163,24 +1190,24 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep */ @Test public void testFlushSuccessWithConnectionResetOnResponseInvalidMd5() throws Exception { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); // Create a spy of AzureBlobFileSystem - try (AzureBlobFileSystem fs = Mockito.spy( + try (AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) { assumeHnsDisabled(); // Create a spy of AzureBlobFileSystemStore - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); assumeBlobServiceType(); // Create spies for the client handler and blob client - AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler()); - AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient()); + AbfsClientHandler clientHandler = spy(store.getClientHandler()); + AbfsBlobClient blobClient = spy(clientHandler.getBlobClient()); // Set up the spies to return the mocked objects - Mockito.doReturn(clientHandler).when(store).getClientHandler(); - Mockito.doReturn(blobClient).when(clientHandler).getBlobClient(); - Mockito.doReturn(blobClient).when(clientHandler).getIngressClient(); + doReturn(clientHandler).when(store).getClientHandler(); + doReturn(blobClient).when(clientHandler).getBlobClient(); + doReturn(blobClient).when(clientHandler).getIngressClient(); AtomicInteger flushCount = new AtomicInteger(0); FSDataOutputStream os = createMockedOutputStream(fs, new Path("/test/file"), blobClient); @@ -1193,48 +1220,48 @@ public void testFlushSuccessWithConnectionResetOnResponseInvalidMd5() throws Exc String blockId = generateBlockId(out, 0); String blockListXml = generateBlockListXml(blockId); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { // Set up the mock for the flush operation AbfsClientTestUtil.setMockAbfsRestOperationForFlushOperation(blobClient, eTag, blockListXml, (httpOperation) -> { - Mockito.doAnswer(invocation -> { + doAnswer(invocation -> { // Call the real processResponse method invocation.callRealMethod(); int currentCount = flushCount.incrementAndGet(); if (currentCount == 1) { - Mockito.when(httpOperation.getStatusCode()) + when(httpOperation.getStatusCode()) .thenReturn( HTTP_INTERNAL_ERROR); // Status code 500 for Internal Server Error - Mockito.when(httpOperation.getStorageErrorMessage()) + when(httpOperation.getStorageErrorMessage()) .thenReturn("CONNECTION_RESET"); // Error message throw new IOException("Connection Reset"); } else if (currentCount == 2) { - Mockito.when(httpOperation.getStatusCode()) + when(httpOperation.getStatusCode()) .thenReturn(HTTP_OK); - Mockito.when(httpOperation.getStorageErrorMessage()) + when(httpOperation.getStorageErrorMessage()) .thenReturn("HTTP_OK"); } return null; }).when(httpOperation).processResponse( - Mockito.nullable(byte[].class), - Mockito.anyInt(), - Mockito.anyInt() + nullable(byte[].class), + anyInt(), + anyInt() ); return httpOperation; }); return answer.callRealMethod(); }).when(blobClient).flush( - Mockito.any(byte[].class), - Mockito.anyString(), - Mockito.anyBoolean(), - Mockito.nullable(String.class), - Mockito.nullable(String.class), - Mockito.anyString(), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.any(TracingContext.class) + any(byte[].class), + anyString(), + anyBoolean(), + nullable(String.class), + nullable(String.class), + anyString(), + nullable(ContextEncryptionAdapter.class), + any(TracingContext.class) ); FSDataOutputStream os1 = createMockedOutputStream(fs, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java index 3e0959d5dd6a0..1a091626630eb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java @@ -22,8 +22,7 @@ import java.io.IOException; import java.util.EnumSet; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.XAttrSetFlag; @@ -36,6 +35,7 @@ import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; /** * Test attribute operations. @@ -111,7 +111,7 @@ public void testGetSetXAttrOnRoot() throws Exception { fs.getXAttr(testPath, attributeName); }); - Assertions.assertThat(ex.getStatusCode()) + assertThat(ex.getStatusCode()) .describedAs("GetXAttr() on root should fail with Bad Request") .isEqualTo(HTTP_BAD_REQUEST); @@ -119,7 +119,7 @@ public void testGetSetXAttrOnRoot() throws Exception { fs.setXAttr(testPath, attributeName, attributeValue, CREATE_FLAG); }); - Assertions.assertThat(ex.getStatusCode()) + assertThat(ex.getStatusCode()) .describedAs("SetXAttr() on root should fail with Bad Request") .isEqualTo(HTTP_BAD_REQUEST); } @@ -235,17 +235,17 @@ private void testGetSetXAttrHelper(final AzureBlobFileSystem fs, } private void assertAttributeNull(byte[] rv) { - Assertions.assertThat(rv) + assertThat(rv) .describedAs("Cannot get attribute before setting it") .isNull(); } private void assertAttributeEqual(byte[] rv, byte[] attributeValue, String decodedAttributeValue) throws Exception { - Assertions.assertThat(rv) + assertThat(rv) .describedAs("Retrieved Attribute Does not Matches in Encoded Form") .containsExactly(attributeValue); - Assertions.assertThat(getFileSystem().getAbfsStore().decodeAttribute(rv)) + assertThat(getFileSystem().getAbfsStore().decodeAttribute(rv)) .describedAs("Retrieved Attribute Does not Matches in Decoded Form") .isEqualTo(decodedAttributeValue); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java index ab01b2e10c4b9..b05d0a2c28348 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java @@ -22,8 +22,7 @@ import java.util.Arrays; import java.util.UUID; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; @@ -44,6 +43,7 @@ import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; import static org.apache.hadoop.fs.permission.AclEntryType.GROUP; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Test Perform Authorization Check operation @@ -58,14 +58,14 @@ public class ITestAzureBlobFileSystemAuthorization extends AbstractAbfsIntegrati public ITestAzureBlobFileSystemAuthorization() throws Exception { // The mock SAS token provider relies on the account key to generate SAS. - Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + assumeTrue(this.getAuthType() == AuthType.SharedKey); } @Override public void setup() throws Exception { boolean isHNSEnabled = getConfiguration().getBoolean( TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); - Assume.assumeTrue(isHNSEnabled); + assumeTrue(isHNSEnabled); loadConfiguredFileSystem(); getConfiguration().set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_AUTHZ_CLASS); getConfiguration().set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SAS.toString()); @@ -216,55 +216,55 @@ public void testGetFileStatusUnauthorized() throws Exception { @Test public void testSetOwnerUnauthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.SetOwner, true); } @Test public void testSetPermissionUnauthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.SetPermissions, true); } @Test public void testModifyAclEntriesUnauthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.ModifyAclEntries, true); } @Test public void testRemoveAclEntriesUnauthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.RemoveAclEntries, true); } @Test public void testRemoveDefaultAclUnauthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.RemoveDefaultAcl, true); } @Test public void testRemoveAclUnauthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.RemoveAcl, true); } @Test public void testSetAclUnauthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.SetAcl, true); } @Test public void testGetAclStatusAuthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.GetAcl, false); } @Test public void testGetAclStatusUnauthorized() throws Exception { - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.GetAcl, true); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java index 2941b96fefa2e..fca7a0012e667 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -18,13 +18,14 @@ package org.apache.hadoop.fs.azurebfs; +import static org.junit.jupiter.api.Assumptions.assumeFalse; + import com.microsoft.azure.storage.CloudStorageAccount; import com.microsoft.azure.storage.blob.CloudBlobClient; import com.microsoft.azure.storage.blob.CloudBlobContainer; import com.microsoft.azure.storage.blob.CloudBlockBlob; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -42,8 +43,8 @@ public ITestAzureBlobFileSystemBackCompat() throws Exception { @Test public void testBlobBackCompat() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeFalse("This test does not support namespace enabled account", - getIsNamespaceEnabled(getFileSystem())); + assumeFalse(getIsNamespaceEnabled(getFileSystem()), + "This test does not support namespace enabled account"); String storageConnectionString = getBlobConnectionString(); CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString); CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCLI.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCLI.java index 6f0d0cc6e1a3b..9d38617094040 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCLI.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCLI.java @@ -20,7 +20,7 @@ import java.util.UUID; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FsShell; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java index 71c77ce82c8e2..d94a8c97b2cfa 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java @@ -17,6 +17,11 @@ */ package org.apache.hadoop.fs.azurebfs; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + import java.io.FileNotFoundException; import java.io.IOException; import java.lang.reflect.Field; @@ -24,9 +29,7 @@ import org.apache.hadoop.fs.azurebfs.enums.Trilean; import org.apache.hadoop.util.Lists; -import org.junit.Assume; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider; @@ -114,34 +117,38 @@ private void setTestFsConf(final String fsConfKey, conf.set(confKeyWithAccountName, confValue); } - @Test(expected = IllegalArgumentException.class) + @Test public void testCheckAccessWithNullPath() throws IOException { - superUserFs.access(null, FsAction.READ); + assertThrows(IllegalArgumentException.class, () -> { + superUserFs.access(null, FsAction.READ); + }); } - @Test(expected = NullPointerException.class) + @Test public void testCheckAccessForFileWithNullFsAction() throws Exception { - Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false", - isHNSEnabled); - Assume.assumeTrue(FS_AZURE_ENABLE_CHECK_ACCESS + " is false", - isCheckAccessEnabled); - // NPE when trying to convert null FsAction enum - superUserFs.access(new Path("test.txt"), null); - } + assertThrows(NullPointerException.class, () -> { + assumeTrue(isHNSEnabled, FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false"); + assumeTrue(isCheckAccessEnabled, FS_AZURE_ENABLE_CHECK_ACCESS + " is false"); + // NPE when trying to convert null FsAction enum + superUserFs.access(new Path("test.txt"), null); + }); +} - @Test(expected = FileNotFoundException.class) + @Test public void testCheckAccessForNonExistentFile() throws Exception { - checkPrerequisites(); - Path nonExistentFile = setupTestDirectoryAndUserAccess( - "/nonExistentFile1.txt", FsAction.ALL); - superUserFs.delete(nonExistentFile, true); - testUserFs.access(nonExistentFile, FsAction.READ); + assertThrows(FileNotFoundException.class, () -> { + checkPrerequisites(); + Path nonExistentFile = setupTestDirectoryAndUserAccess( + "/nonExistentFile1.txt", FsAction.ALL); + superUserFs.delete(nonExistentFile, true); + testUserFs.access(nonExistentFile, FsAction.READ); + }); } @Test public void testWhenCheckAccessConfigIsOff() throws Exception { - Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false", - isHNSEnabled); + assumeTrue(isHNSEnabled, + FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false"); Configuration conf = getRawConfiguration(); conf.setBoolean(FS_AZURE_ENABLE_CHECK_ACCESS, false); FileSystem fs = FileSystem.newInstance(conf); @@ -172,11 +179,10 @@ public void testWhenCheckAccessConfigIsOff() throws Exception { @Test public void testCheckAccessForAccountWithoutNS() throws Exception { - Assume.assumeFalse(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is true", - getConfiguration() - .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, true)); - Assume.assumeTrue(FS_AZURE_ENABLE_CHECK_ACCESS + " is false", - isCheckAccessEnabled); + assumeFalse(getConfiguration(). + getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, true), + FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is true"); + assumeTrue(isCheckAccessEnabled, FS_AZURE_ENABLE_CHECK_ACCESS + " is false"); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID); @@ -194,8 +200,8 @@ public void testCheckAccessForAccountWithoutNS() throws Exception { // or not, and as the account is non HNS the AzureBlobFileSystem#access // acts as noop AzureBlobFileSystemStore mockAbfsStore = - Mockito.mock(AzureBlobFileSystemStore.class); - Mockito.when(mockAbfsStore + mock(AzureBlobFileSystemStore.class); + when(mockAbfsStore .getIsNamespaceEnabled(getTestTracingContext(getFileSystem(), false))) .thenReturn(true); Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField( @@ -311,10 +317,9 @@ public void testFsActionALL() throws Exception { } private void checkPrerequisites() throws Exception { - Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false", - isHNSEnabled); - Assume.assumeTrue(FS_AZURE_ENABLE_CHECK_ACCESS + " is false", - isCheckAccessEnabled); + assumeTrue(isHNSEnabled, + FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false"); + assumeTrue(isCheckAccessEnabled, FS_AZURE_ENABLE_CHECK_ACCESS + " is false"); setTestUserFs(); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET); @@ -324,22 +329,20 @@ private void checkPrerequisites() throws Exception { private void checkIfConfigIsSet(String configKey){ AbfsConfiguration conf = getConfiguration(); String value = conf.get(configKey); - Assume.assumeTrue(configKey + " config is mandatory for the test to run", - value != null && value.trim().length() > 1); + assumeTrue(value != null && value.trim().length() > 1, + configKey + " config is mandatory for the test to run"); } private void assertAccessible(Path testFilePath, FsAction fsAction) throws IOException { - assertTrue( - "Should have been given access " + fsAction + " on " + testFilePath, - isAccessible(testUserFs, testFilePath, fsAction)); + assertTrue(isAccessible(testUserFs, testFilePath, fsAction), + "Should have been given access " + fsAction + " on " + testFilePath); } private void assertInaccessible(Path testFilePath, FsAction fsAction) throws IOException { - assertFalse( - "Should have been denied access " + fsAction + " on " + testFilePath, - isAccessible(testUserFs, testFilePath, fsAction)); + assertFalse(isAccessible(testUserFs, testFilePath, fsAction), + "Should have been denied access " + fsAction + " on " + testFilePath); } private void setExecuteAccessForParentDirs(Path dir) throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java index bf8c14ae5e14d..0967849a287cd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java @@ -25,10 +25,7 @@ import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -47,7 +44,11 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeFalse; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; /** * Test For Verifying Checksum Related Operations @@ -76,7 +77,7 @@ public void testWriteReadWithChecksum() throws Exception { public void testAppendWithChecksumAtDifferentOffsets() throws Exception { AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true); if (!getIsNamespaceEnabled(fs)) { - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); } AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient(); Path path = path("testPath" + getMethodName()); @@ -116,19 +117,19 @@ public void testWriteReadWithChecksumAndOptions() throws Exception { @Test public void testAbfsInvalidChecksumExceptionInAppend() throws Exception { AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true); - AbfsClient spiedClient = Mockito.spy(fs.getAbfsStore().getClientHandler().getIngressClient()); + AbfsClient spiedClient = spy(fs.getAbfsStore().getClientHandler().getIngressClient()); Path path = path("testPath" + getMethodName()); AbfsOutputStream os = (AbfsOutputStream) fs.create(path).getWrappedStream(); byte[] data= generateRandomBytes(MB_4); String invalidMD5Hash = spiedClient.computeMD5Hash( INVALID_MD5_TEXT.getBytes(), 0, INVALID_MD5_TEXT.length()); - Mockito.doReturn(invalidMD5Hash).when(spiedClient).computeMD5Hash(any(), + doReturn(invalidMD5Hash).when(spiedClient).computeMD5Hash(any(), any(Integer.class), any(Integer.class)); AbfsRestOperationException ex = intercept(AbfsInvalidChecksumException.class, () -> { appendWithOffsetHelper(os, spiedClient, path, data, fs, 0, 0); }); - Assertions.assertThat(ex.getErrorCode()) + assertThat(ex.getErrorCode()) .describedAs("Exception Message should contain MD5Mismatch") .isEqualTo(AzureServiceErrorCode.MD5_MISMATCH); fs.close(); @@ -137,14 +138,14 @@ public void testAbfsInvalidChecksumExceptionInAppend() throws Exception { @Test public void testAbfsInvalidChecksumExceptionInRead() throws Exception { AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true); - AbfsClient spiedClient = Mockito.spy(fs.getAbfsStore().getClient()); + AbfsClient spiedClient = spy(fs.getAbfsStore().getClient()); Path path = path("testPath" + getMethodName()); byte[] data = generateRandomBytes(MB_3); createFileWithData(path, data, fs); String invalidMD5Hash = spiedClient.computeMD5Hash( INVALID_MD5_TEXT.getBytes(), 0, INVALID_MD5_TEXT.length()); - Mockito.doReturn(invalidMD5Hash).when(spiedClient).computeMD5Hash(any(), + doReturn(invalidMD5Hash).when(spiedClient).computeMD5Hash(any(), any(Integer.class), any(Integer.class)); intercept(AbfsInvalidChecksumException.class, () -> { @@ -166,9 +167,9 @@ private void testWriteReadWithChecksumInternal(final boolean readAheadEnabled) in.read(bytesRead, 0, dataSize); // Verify that the data read is same as data written - Assertions.assertThat(bytesRead) - .describedAs("Bytes read with checksum enabled are not as expected") - .containsExactly(bytesUploaded); + assertThat(bytesRead) + .describedAs("Bytes read with checksum enabled are not as expected") + .containsExactly(bytesUploaded); } } @@ -182,7 +183,8 @@ private String generateBlockId(AbfsOutputStream os, long position) { String streamIdHash = Integer.toString(streamId.hashCode()); String blockId = String.format("%d_%s", position, streamIdHash); byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH]; - System.arraycopy(blockId.getBytes(), 0, blockIdByteArray, 0, Math.min(BLOCK_ID_LENGTH, blockId.length())); + System.arraycopy(blockId.getBytes(), 0, blockIdByteArray, + 0, Math.min(BLOCK_ID_LENGTH, blockId.length())); return new String(Base64.encodeBase64(blockIdByteArray), StandardCharsets.UTF_8); } @@ -234,7 +236,7 @@ private void readWithOffsetAndPositionHelper(AbfsClient client, Path path, byte[] actual = Arrays.copyOfRange(readBuffer, offset, offset + readLength); byte[] expected = Arrays.copyOfRange(data, position, readLength + position); - Assertions.assertThat(actual) + assertThat(actual) .describedAs("Data read should be same as Data Written") .containsExactly(expected); } @@ -258,9 +260,9 @@ private void testWriteReadWithChecksumAndOptionsInternal( in.read(1, bytesRead, 1, MB_4); // Verify that the data read is same as data written - Assertions.assertThat(Arrays.copyOfRange(bytesRead, 1, MB_4)) - .describedAs("Bytes read with checksum enabled are not as expected") - .containsExactly(Arrays.copyOfRange(bytesUploaded, 1, MB_4)); + assertThat(Arrays.copyOfRange(bytesRead, 1, MB_4)) + .describedAs("Bytes read with checksum enabled are not as expected") + .containsExactly(Arrays.copyOfRange(bytesUploaded, 1, MB_4)); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java index e9b1a27278f24..52b4e5be9530a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java @@ -20,9 +20,7 @@ import java.io.IOException; import java.nio.file.AccessDeniedException; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -45,6 +43,8 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Tests to validate the choice between using a custom SASTokenProvider @@ -66,7 +66,7 @@ public ITestAzureBlobFileSystemChooseSAS() throws Exception { // SAS Token configured might not have permissions for creating file system. // Shared Key must be configured to create one. Once created, a new instance // of same file system will be used with SAS Authentication. - Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + assumeTrue(this.getAuthType() == AuthType.SharedKey); } @Override @@ -153,7 +153,7 @@ public void testBothProviderFixedTokenConfigured() throws Exception { FileSystem.newInstance(testAbfsConfig.getRawConfiguration())) { // Asserting that MockDelegationSASTokenProvider is used. - Assertions.assertThat(testAbfsConfig.getSASTokenProvider()) + assertThat(testAbfsConfig.getSASTokenProvider()) .describedAs("Custom SASTokenProvider Class must be used") .isInstanceOf(MockDelegationSASTokenProvider.class); @@ -196,7 +196,7 @@ public void testFixedSASTokenProviderPreference() throws Exception { testAbfsConfig.set(FS_AZURE_SAS_FIXED_TOKEN, accountAgnosticSAS); // Assert that Container Specific Fixed SAS is used - Assertions.assertThat(getFixedSASToken(testAbfsConfig)) + assertThat(getFixedSASToken(testAbfsConfig)) .describedAs("Container-specific fixed SAS should've been used.") .isEqualTo(containerSAS); @@ -204,14 +204,14 @@ public void testFixedSASTokenProviderPreference() throws Exception { testAbfsConfig.unset( containerProperty(FS_AZURE_SAS_FIXED_TOKEN, this.getFileSystemName(), this.getAccountName())); - Assertions.assertThat(getFixedSASToken(testAbfsConfig)) + assertThat(getFixedSASToken(testAbfsConfig)) .describedAs("Account-specific fixed SAS should've been used.") .isEqualTo(accountSAS); //Assert that Account-Agnostic fixed SAS is used if no other fixed SAS configs are set. testAbfsConfig.unset( accountProperty(FS_AZURE_SAS_FIXED_TOKEN, this.getAccountName())); - Assertions.assertThat(getFixedSASToken(testAbfsConfig)) + assertThat(getFixedSASToken(testAbfsConfig)) .describedAs("Account-agnostic fixed SAS should've been used.") .isEqualTo(accountAgnosticSAS); } @@ -244,7 +244,7 @@ private void testOnlyFixedTokenConfiguredInternal(AbfsConfiguration testAbfsConf FileSystem.newInstance(testAbfsConfig.getRawConfiguration())) { // Asserting that FixedSASTokenProvider is used. - Assertions.assertThat(testAbfsConfig.getSASTokenProvider()) + assertThat(testAbfsConfig.getSASTokenProvider()) .describedAs("FixedSASTokenProvider Class must be used") .isInstanceOf(FixedSASTokenProvider.class); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java index aabaf82b622a8..0390a99c6a519 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java @@ -24,7 +24,7 @@ import java.io.InputStreamReader; import java.io.OutputStreamWriter; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index b719a3217b299..90a0a9b98ca2f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -34,9 +34,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.assertj.core.api.Assertions; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -96,17 +94,19 @@ import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicity.SUFFIX; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; /** @@ -210,7 +210,7 @@ public void testCreateNonRecursiveWhenParentExist() throws Exception { fs.createNonRecursive(new Path(createDirectoryPath, "B"), FsPermission .getDefault(), false, 1024, (short) 1, 1024, null); - Assertions.assertThat(fs.exists(new Path(createDirectoryPath, "B"))) + assertThat(fs.exists(new Path(createDirectoryPath, "B"))) .describedAs("File should be created").isTrue(); fs.close(); } @@ -230,7 +230,7 @@ public void testCreateNonRecursiveWhenParentNotExist() throws Exception { intercept(FileNotFoundException.class, () -> fs.createNonRecursive(new Path("A/B/C"), FsPermission .getDefault(), false, 1024, (short) 1, 1024, null)); - Assertions.assertThat(fs.exists(new Path("A/B/C"))) + assertThat(fs.exists(new Path("A/B/C"))) .describedAs("New File should not be created.").isFalse(); fs.close(); } @@ -244,11 +244,11 @@ public void testCreateNonRecursiveWhenParentNotExist() throws Exception { * @throws IOException in case of failure */ private AzureBlobFileSystem createJsonFile(Path path, Path renameJson) throws IOException { - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); doReturn(store).when(fs).getAbfsStore(); - AbfsClient client = Mockito.spy(store.getClient()); + AbfsClient client = spy(store.getClient()); doReturn(client).when(store).getClient(); fs.setWorkingDirectory(new Path(ROOT_PATH)); fs.mkdirs(new Path(path, "test3")); @@ -258,7 +258,7 @@ private AzureBlobFileSystem createJsonFile(Path path, Path renameJson) throws IO new Path("/hbase/test4"), renameJson, getTestTracingContext(fs, true), fileStatus.getEtag(), client).preRename(); - Assertions.assertThat(fs.exists(renameJson)) + assertThat(fs.exists(renameJson)) .describedAs("Rename Pending Json file should exist.") .isTrue(); return fs; @@ -280,15 +280,15 @@ public void testCreateNonRecursiveWhenParentNotExistAndRenamePendingExist() thro Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX); fs = createJsonFile(path, renameJson); fs.delete(path, true); - Assertions.assertThat(fs.exists(renameJson)).isTrue(); + assertThat(fs.exists(renameJson)).isTrue(); AzureBlobFileSystem finalFs = fs; intercept(FileNotFoundException.class, () -> finalFs.createNonRecursive(new Path(path, "test4"), FsPermission .getDefault(), false, 1024, (short) 1, 1024, null)); - Assertions.assertThat(finalFs.exists(new Path(path, "test4"))) + assertThat(finalFs.exists(new Path(path, "test4"))) .describedAs("New File should not be created.") .isFalse(); - Assertions.assertThat(finalFs.exists(renameJson)) + assertThat(finalFs.exists(renameJson)) .describedAs("Rename Pending Json file should be deleted.") .isFalse(); } finally { @@ -317,16 +317,16 @@ public void testCreateNonRecursiveWhenParentAndRenamePendingExist() throws Excep intercept(FileNotFoundException.class, () -> finalFs.createNonRecursive(new Path(path, "test4"), FsPermission .getDefault(), false, 1024, (short) 1, 1024, null)); - Assertions.assertThat(finalFs.exists(path)) + assertThat(finalFs.exists(path)) .describedAs("Old path should be deleted.") .isFalse(); - Assertions.assertThat(finalFs.exists(new Path(path, "test4"))) + assertThat(finalFs.exists(new Path(path, "test4"))) .describedAs("New File should not be created.") .isFalse(); - Assertions.assertThat(finalFs.exists(renameJson)) + assertThat(finalFs.exists(renameJson)) .describedAs("Rename Pending Json file should be deleted.") .isFalse(); - Assertions.assertThat(finalFs.exists(new Path("/hbase/test4"))) + assertThat(finalFs.exists(new Path("/hbase/test4"))) .describedAs("Rename should be successful.") .isTrue(); } finally { @@ -400,7 +400,7 @@ public void testTryWithResources() throws Throwable { // the exception raised in close() must be in the caught exception's // suppressed list Throwable[] suppressed = fnfe.getSuppressed(); - Assertions.assertThat(suppressed.length) + assertThat(suppressed.length) .describedAs("suppressed count should be 1").isEqualTo(1); Throwable inner = suppressed[0]; if (!(inner instanceof IOException)) { @@ -611,9 +611,9 @@ public void testNegativeScenariosForCreateOverwriteDisabled() = ITestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); - AbfsClientHandler clientHandler = Mockito.mock(AbfsClientHandler.class); + AbfsClientHandler clientHandler = mock(AbfsClientHandler.class); when(clientHandler.getIngressClient()).thenReturn(mockClient); - when(clientHandler.getClient(Mockito.any())).thenReturn(mockClient); + when(clientHandler.getClient(any())).thenReturn(mockClient); AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); @@ -640,11 +640,11 @@ public void testNegativeScenariosForCreateOverwriteDisabled() doCallRealMethod().when(mockClient) .conditionalCreateOverwriteFile(anyString(), - Mockito.nullable(FileSystem.Statistics.class), - Mockito.nullable(AzureBlobFileSystemStore.Permissions.class), + nullable(FileSystem.Statistics.class), + nullable(AzureBlobFileSystemStore.Permissions.class), anyBoolean(), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.nullable(TracingContext.class)); + nullable(ContextEncryptionAdapter.class), + nullable(TracingContext.class)); // mock for overwrite=false doThrow(conflictResponseEx) // Scn1: GFS fails with Http404 @@ -686,14 +686,14 @@ public void testNegativeScenariosForCreateOverwriteDisabled() if (mockClient instanceof AbfsBlobClient) { doReturn(false).when((AbfsBlobClient) mockClient) .isNonEmptyDirectory(anyString(), - Mockito.nullable(TracingContext.class)); + nullable(TracingContext.class)); doNothing().when((AbfsBlobClient) mockClient) .tryMarkerCreation(anyString(), anyBoolean(), - Mockito.nullable(String.class), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.nullable(TracingContext.class)); + nullable(String.class), + nullable(ContextEncryptionAdapter.class), + nullable(TracingContext.class)); // mock for overwrite=true doThrow( @@ -786,13 +786,13 @@ public void testCreateMarkerFailExceptionIsSwallowed() throws Throwable { config.set("fs.azure.enable.conditional.create.overwrite", Boolean.toString(true)); try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(), config)) { - AbfsClient mockClient = Mockito.spy(fs.getAbfsClient()); - AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + AbfsClient mockClient = spy(fs.getAbfsClient()); + AzureBlobFileSystemStore spiedStore = spy(fs.getAbfsStore()); spiedStore.setClient(mockClient); - AbfsClientHandler clientHandler = Mockito.mock(AbfsClientHandler.class); + AbfsClientHandler clientHandler = mock(AbfsClientHandler.class); when(clientHandler.getIngressClient()).thenReturn(mockClient); - when(clientHandler.getClient(Mockito.any())).thenReturn(mockClient); + when(clientHandler.getClient(any())).thenReturn(mockClient); Path testFolder = new Path("/dir1"); createAzCopyFolder(testFolder); @@ -809,23 +809,23 @@ public void testCreateMarkerFailExceptionIsSwallowed() throws Throwable { doCallRealMethod().when(mockClient) .conditionalCreateOverwriteFile(anyString(), - Mockito.nullable(FileSystem.Statistics.class), - Mockito.nullable(AzureBlobFileSystemStore.Permissions.class), + nullable(FileSystem.Statistics.class), + nullable(AzureBlobFileSystemStore.Permissions.class), anyBoolean(), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.nullable(TracingContext.class)); + nullable(ContextEncryptionAdapter.class), + nullable(TracingContext.class)); doCallRealMethod().when((AbfsBlobClient) mockClient) - .tryMarkerCreation(anyString(), anyBoolean(), Mockito.nullable(String.class), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.nullable(TracingContext.class)); + .tryMarkerCreation(anyString(), anyBoolean(), nullable(String.class), + nullable(ContextEncryptionAdapter.class), + nullable(TracingContext.class)); - Mockito.doReturn(new ArrayList<>(Collections.singletonList(testFolder))) + doReturn(new ArrayList<>(Collections.singletonList(testFolder))) .when((AbfsBlobClient) mockClient) - .getMarkerPathsTobeCreated(any(Path.class), Mockito.nullable(TracingContext.class)); + .getMarkerPathsTobeCreated(any(Path.class), nullable(TracingContext.class)); doReturn(false).when((AbfsBlobClient) mockClient) - .isNonEmptyDirectory(anyString(), Mockito.nullable(TracingContext.class)); + .isNonEmptyDirectory(anyString(), nullable(TracingContext.class)); doAnswer(new Answer() { private boolean firstCall = true; @@ -841,9 +841,9 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).doCallRealMethod() .when((AbfsBlobClient) mockClient) .createPathRestOp(anyString(), anyBoolean(), anyBoolean(), - anyBoolean(), Mockito.nullable(String.class), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.nullable(TracingContext.class)); + anyBoolean(), nullable(String.class), + nullable(ContextEncryptionAdapter.class), + nullable(TracingContext.class)); AbfsClientTestUtil.hookOnRestOpsForTracingContextSingularity(mockClient); @@ -857,7 +857,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { Path testPath = new Path("/dir1/testFile"); abfsStore.createFile(testPath, null, true, permission, umask, getTestTracingContext(getFileSystem(), true)); - Assertions.assertThat(fs.exists(testPath)) + assertThat(fs.exists(testPath)) .describedAs("File not created when marker creation failed.") .isTrue(); } @@ -917,7 +917,7 @@ public void testDeleteBeforeFlush() throws Throwable { // the exception raised in close() must be in the caught exception's // suppressed list Throwable[] suppressed = fnfe.getSuppressed(); - assertEquals("suppressed count", 1, suppressed.length); + assertEquals(1, suppressed.length, "suppressed count"); Throwable inner = suppressed[0]; if (!(inner instanceof IOException)) { throw inner; @@ -940,10 +940,10 @@ public void testMkdirsFailsForSubdirectoryOfExistingFile() throws Exception { fs.mkdirs(new Path("a/b/d")); intercept(IOException.class, () -> fs.mkdirs(new Path("a/b/c/d/e"))); - Assertions.assertThat(fs.exists(new Path("a/b/c"))).isTrue(); - Assertions.assertThat(fs.exists(new Path("a/b/d"))).isTrue(); + assertThat(fs.exists(new Path("a/b/c"))).isTrue(); + assertThat(fs.exists(new Path("a/b/d"))).isTrue(); // Asserting directory created still exists as explicit. - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a/b/d"), fs, getTestTracingContext(fs, true))) .describedAs("Path is not an explicit directory") @@ -971,10 +971,10 @@ public void testMkdirSameFolder() throws Exception { public void testCreateDirectoryAndFile() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { fs.mkdirs(new Path("a/b/c")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))).isTrue(); + assertThat(fs.exists(new Path("a/b/c"))).isTrue(); intercept(IOException.class, () -> fs.create(new Path("a/b/c"))); // Asserting that directory still exists as explicit - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), fs, getTestTracingContext(fs, true))) .describedAs("Path is not an explicit directory") @@ -991,7 +991,7 @@ public void testCreateSameFile() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { fs.create(new Path("a/b/c")); fs.create(new Path("a/b/c")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Path does not exist") .isTrue(); } @@ -1055,7 +1055,7 @@ public void testCreationOverwriteFalseWithoutConditionalOverwrite() throws Excep public void testCreateSameFileWithOverwriteFalse() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { fs.create(new Path("a/b/c")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Path does not exist") .isTrue(); intercept(IOException.class, @@ -1071,7 +1071,7 @@ public void testCreateSameFileWithOverwriteFalse() throws Exception { public void testCreateSubPath() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { fs.create(new Path("a/b/c")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Path does not exist") .isTrue(); intercept(IOException.class, @@ -1176,16 +1176,16 @@ public void testParallelCreateOverwriteTrue() public void testCreatePathParentExplicit() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { fs.mkdirs(new Path("a/b/c")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Path does not exist") .isTrue(); fs.create(new Path("a/b/c/d")); - Assertions.assertThat(fs.exists(new Path("a/b/c/d"))) + assertThat(fs.exists(new Path("a/b/c/d"))) .describedAs("Path does not exist") .isTrue(); // asserting that parent stays explicit - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), fs, getTestTracingContext(fs, true))) .describedAs("Path is not an explicit directory") @@ -1198,9 +1198,9 @@ fs, getTestTracingContext(fs, true))) public void testCreateWithAppendBlobEnabled() throws IOException, NoSuchFieldException, IllegalAccessException { Configuration conf = getRawConfiguration(); - try (AzureBlobFileSystem fs = Mockito.spy( + try (AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(conf))) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); doReturn(true).when(store).isAppendBlobKey(anyString()); // Set abfsStore as our mocked value. @@ -1237,12 +1237,12 @@ public void testParentExplicitPathImplicit() throws Exception { intercept(IOException.class, () -> fs.create(sourcePath, false)); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(sourcePath.getParent(), fs, getTestTracingContext(fs, true))) .describedAs("Parent directory should be explicit.") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isImplicitDirectory(sourcePath, fs, getTestTracingContext(fs, true))) .describedAs("Path should be implicit.") @@ -1271,12 +1271,12 @@ public void testParentImplicitPathImplicit() throws Exception { intercept(IOException.class, () -> fs.create(sourcePath, false)); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isImplicitDirectory(parentPath, fs, getTestTracingContext(fs, true))) .describedAs("Parent directory is implicit.") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isImplicitDirectory(sourcePath, fs, getTestTracingContext(fs, true))) .describedAs("Path should also be implicit.") @@ -1305,7 +1305,7 @@ public void testCreateFileExistsImplicitParent() throws Exception { String eTagAfterCreateOverwrite = extractFileEtag(fileName); - Assertions.assertThat(eTag.equals(eTagAfterCreateOverwrite)) + assertThat(eTag.equals(eTagAfterCreateOverwrite)) .describedAs( "New file eTag after create overwrite should be different from old") .isFalse(); @@ -1315,11 +1315,11 @@ public void testCreateFileExistsImplicitParent() throws Exception { String eTagAfterCreate = extractFileEtag(fileName); - Assertions.assertThat(eTagAfterCreateOverwrite.equals(eTagAfterCreate)) + assertThat(eTagAfterCreateOverwrite.equals(eTagAfterCreate)) .describedAs("File eTag should not change as creation fails") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(parentPath, fs, getTestTracingContext(fs, true))) .describedAs("Parent path should also change to explicit.") @@ -1348,7 +1348,7 @@ public void testCreateFileExistsExplicitParent() throws Exception { String eTagAfterCreateOverwrite = extractFileEtag(fileName); - Assertions.assertThat(eTag.equals(eTagAfterCreateOverwrite)) + assertThat(eTag.equals(eTagAfterCreateOverwrite)) .describedAs( "New file eTag after create overwrite should be different from old") .isFalse(); @@ -1358,11 +1358,11 @@ public void testCreateFileExistsExplicitParent() throws Exception { String eTagAfterCreate = extractFileEtag(fileName); - Assertions.assertThat(eTagAfterCreateOverwrite.equals(eTagAfterCreate)) + assertThat(eTagAfterCreateOverwrite.equals(eTagAfterCreate)) .describedAs("File eTag should not change as creation fails") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(parentPath, fs, getTestTracingContext(fs, true))) .describedAs("Parent path should also change to explicit.") @@ -1391,7 +1391,7 @@ public void testCreateFileParentFile() throws Exception { FileStatus status = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path(parentName)), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path is not a file") .isFalse(); } @@ -1421,13 +1421,13 @@ public void testMkdirs() throws Exception { fs.mkdirs(new Path("a/b/c/d")); fs.mkdirs(new Path("a/b/c/e")); - Assertions.assertThat(fs.exists(new Path("a/b"))) + assertThat(fs.exists(new Path("a/b"))) .describedAs("Path a/b does not exist") .isTrue(); - Assertions.assertThat(fs.exists(new Path("a/b/c/d"))) + assertThat(fs.exists(new Path("a/b/c/d"))) .describedAs("Path a/b/c/d does not exist") .isTrue(); - Assertions.assertThat(fs.exists(new Path("a/b/c/e"))) + assertThat(fs.exists(new Path("a/b/c/e"))) .describedAs("Path a/b/c/e does not exist") .isTrue(); @@ -1435,19 +1435,19 @@ public void testMkdirs() throws Exception { FileStatus status = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a/b")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path a/b is not an explicit directory") .isTrue(); FileStatus status1 = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a/b/c/d")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status1.isDirectory()) + assertThat(status1.isDirectory()) .describedAs("Path a/b/c/d is not an explicit directory") .isTrue(); FileStatus status2 = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a/b/c/e")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status2.isDirectory()) + assertThat(status2.isDirectory()) .describedAs("Path a/b/c/e is not an explicit directory") .isTrue(); } @@ -1461,7 +1461,7 @@ public void testMkdirs() throws Exception { public void testMkdirsCreateSubPath() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { fs.mkdirs(new Path("a/b/c")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Path a/b/c does not exist") .isTrue(); intercept(IOException.class, () -> fs.create(new Path("a/b"))); @@ -1470,7 +1470,7 @@ public void testMkdirsCreateSubPath() throws Exception { FileStatus status2 = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a/b/c")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status2.isDirectory()) + assertThat(status2.isDirectory()) .describedAs("Path a/b/c is not an explicit directory") .isTrue(); } @@ -1487,13 +1487,13 @@ public void testMkdirsByLevel() throws Exception { fs.mkdirs(new Path("a/b/c")); fs.mkdirs(new Path("a/b/c/d/e")); - Assertions.assertThat(fs.exists(new Path("a"))) + assertThat(fs.exists(new Path("a"))) .describedAs("Path a does not exist") .isTrue(); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Path a/b/c does not exist") .isTrue(); - Assertions.assertThat(fs.exists(new Path("a/b/c/d/e"))) + assertThat(fs.exists(new Path("a/b/c/d/e"))) .describedAs("Path a/b/c/d/e does not exist") .isTrue(); @@ -1501,19 +1501,19 @@ public void testMkdirsByLevel() throws Exception { FileStatus status = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a/")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path a is not an explicit directory") .isTrue(); FileStatus status1 = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a/b/c")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status1.isDirectory()) + assertThat(status1.isDirectory()) .describedAs("Path a/b/c is not an explicit directory") .isTrue(); FileStatus status2 = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a/b/c/d/e")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status2.isDirectory()) + assertThat(status2.isDirectory()) .describedAs("Path a/b/c/d/e is not an explicit directory") .isTrue(); } @@ -1529,7 +1529,7 @@ public void testMkdirsWithDelete() throws Exception { fs.mkdirs(new Path("a/b/c/d")); fs.delete(new Path("a/b/c/d")); fs.getFileStatus(new Path("a/b/c")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Path a/b/c does not exist") .isTrue(); } @@ -1544,10 +1544,10 @@ public void testMkdirsWithRename() throws Exception { fs.mkdirs(new Path("a/b/c/d")); fs.create(new Path("e/file")); fs.delete(new Path("a/b/c/d")); - Assertions.assertThat(fs.rename(new Path("e"), new Path("a/b/c/d"))) + assertThat(fs.rename(new Path("e"), new Path("a/b/c/d"))) .describedAs("Failed to rename path e to a/b/c/d") .isTrue(); - Assertions.assertThat(fs.exists(new Path("a/b/c/d/file"))) + assertThat(fs.exists(new Path("a/b/c/d/file"))) .describedAs("Path a/b/c/d/file does not exist") .isTrue(); } @@ -1591,14 +1591,14 @@ public void testCreateSameDirectory() throws Exception { fs.mkdirs(new Path("a/b/c")); fs.mkdirs(new Path("a/b/c")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Path a/b/c does not exist") .isTrue(); // Asserting that directories created as explicit FileStatus status = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a/b/c")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path a/b/c is not an explicit directory") .isTrue(); } @@ -1630,7 +1630,7 @@ public void testMkdirOnRootAsParent() throws Exception { FileStatus status = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("a")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path a is not an explicit directory") .isTrue(); } @@ -1649,7 +1649,7 @@ public void testMkdirOnRoot() throws Exception { FileStatus status = fs.getAbfsStore() .getFileStatus(fs.makeQualified(new Path("/")), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path is not an explicit directory") .isTrue(); } @@ -1664,7 +1664,7 @@ public void testCreateUnicode() throws Exception { final Path path = new Path("/file\u0031"); fs.create(path); - Assertions.assertThat(fs.exists(path)) + assertThat(fs.exists(path)) .describedAs("Path with unicode does not exist") .isTrue(); } @@ -1683,7 +1683,7 @@ public void testMkdirUnicode() throws Exception { FileStatus status = fs.getAbfsStore() .getFileStatus(fs.makeQualified(path), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path is not an explicit directory") .isTrue(); } @@ -1719,7 +1719,7 @@ public void testMkdirParallelRequests() throws Exception { FileStatus status = fs.getAbfsStore() .getFileStatus(fs.makeQualified(path), new TracingContext(getTestTracingContext(fs, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path is not an explicit directory") .isTrue(); } @@ -1742,7 +1742,7 @@ public void testCreateSameDirectoryOverwriteFalse() throws Exception { FileStatus status = fs1.getAbfsStore() .getFileStatus(fs1.makeQualified(new Path("a/b/c")), new TracingContext(getTestTracingContext(fs1, true))); - Assertions.assertThat(status.isDirectory()) + assertThat(status.isDirectory()) .describedAs("Path is not an explicit directory") .isTrue(); } @@ -1756,10 +1756,10 @@ public void testCreateDirectoryAndFileRecreation() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { fs.mkdirs(new Path("a/b/c")); fs.create(new Path("a/b/c/d")); - Assertions.assertThat(fs.exists(new Path("a/b/c"))) + assertThat(fs.exists(new Path("a/b/c"))) .describedAs("Directory a/b/c does not exist") .isTrue(); - Assertions.assertThat(fs.exists(new Path("a/b/c/d"))) + assertThat(fs.exists(new Path("a/b/c/d"))) .describedAs("File a/b/c/d does not exist") .isTrue(); intercept(IOException.class, @@ -1778,7 +1778,7 @@ public void testCreateNonRecursiveForAtomicDirectoryFile() throws Exception { .bufferSize(1024) .blockSize(1024) .build(); - Assertions.assertThat(fileSystem.exists(new Path("/hbase/dir/file"))) + assertThat(fileSystem.exists(new Path("/hbase/dir/file"))) .describedAs("File /hbase/dir/file does not exist") .isTrue(); } @@ -1802,7 +1802,7 @@ public void testCreateOnNonExistingPathWithImplicitParentDir() throws Exception fs.create(path); // Asserting that path created by azcopy becomes explicit. - Assertions.assertThat(fs.exists(path)) + assertThat(fs.exists(path)) .describedAs("File dir1/dir2 does not exist") .isTrue(); } @@ -1819,14 +1819,14 @@ public void testMkdirOnNonExistingPathWithImplicitParentDir() throws Exception { fs.mkdirs(path); // Asserting that path created by azcopy becomes explicit. - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(implicitPath, fs, getTestTracingContext(fs, true))) .describedAs("Path created by azcopy did not become explicit") .isTrue(); // Asserting that the directory created by mkdir exists as explicit. - Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(path, + assertThat(DirectoryStateHelper.isExplicitDirectory(path, fs, getTestTracingContext(fs, true))) .describedAs("Directory created by mkdir does not exist as explicit") .isTrue(); @@ -1854,14 +1854,14 @@ public void testMkdirOnExistingExplicitDirWithImplicitParentDir() throws Excepti fs.mkdirs(path); // Asserting that path created by azcopy becomes explicit. - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(implicitPath, fs, getTestTracingContext(fs, true))) .describedAs("Path created by azcopy did not become explicit") .isTrue(); // Asserting that the directory created by mkdir exists as explicit. - Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(path, + assertThat(DirectoryStateHelper.isExplicitDirectory(path, fs, getTestTracingContext(fs, true))) .describedAs("Directory created by mkdir does not exist as explicit") .isTrue(); @@ -1888,14 +1888,14 @@ public void testMkdirOnExistingImplicitDirWithExplicitParentDir() throws Excepti fs.mkdirs(path); // Asserting that the directory created by mkdir exists as explicit. - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(explicitPath, fs, getTestTracingContext(fs, true))) .describedAs("Explicit parent directory does not exist as explicit") .isTrue(); // Asserting that the directory created by mkdir exists as explicit. - Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(path, + assertThat(DirectoryStateHelper.isImplicitDirectory(path, fs, getTestTracingContext(fs, true))) .describedAs("Mkdir created explicit directory") .isTrue(); @@ -1921,14 +1921,14 @@ public void testMkdirOnExistingImplicitDirWithImplicitParentDir() throws Excepti // Creating a directory on existing implicit directory inside an implicit directory fs.mkdirs(path); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isImplicitDirectory(implicitPath, fs, getTestTracingContext(fs, true))) .describedAs("Marker is present for path created by azcopy") .isTrue(); // Asserting that the mkdir didn't create markers for existing directory. - Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(path, + assertThat(DirectoryStateHelper.isImplicitDirectory(path, fs, getTestTracingContext(fs, true))) .describedAs("Marker is present for existing directory") .isTrue(); @@ -1958,7 +1958,7 @@ public void testMkdirOnExistingFileWithImplicitParentDir() throws Exception { }); // Asserting that the file still exists at path. - Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(path, + assertThat(DirectoryStateHelper.isExplicitDirectory(path, fs, getTestTracingContext(fs, true))) .describedAs("File still exists at path") .isFalse(); @@ -1973,7 +1973,7 @@ fs, getTestTracingContext(fs, true))) */ @Test public void testImplicitExplicitFolder() throws Exception { - Configuration configuration = Mockito.spy(getRawConfiguration()); + Configuration configuration = spy(getRawConfiguration()); try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration)) { final Path implicitPath = new Path("a/b/c"); @@ -1987,22 +1987,22 @@ public void testImplicitExplicitFolder() throws Exception { fs.mkdirs(new Path("a/b/c/d")); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isImplicitDirectory(new Path("a"), fs, getTestTracingContext(fs, true))) .describedAs("Directory 'a' should be implicit") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a/b"), fs, getTestTracingContext(fs, true))) .describedAs("Directory 'a/b' should be explicit") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), fs, getTestTracingContext(fs, true))) .describedAs("Directory 'a/b/c' should be explicit") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/d"), fs, getTestTracingContext(fs, true))) .describedAs("Directory 'a/b/c/d' should be explicit") @@ -2018,7 +2018,7 @@ fs, getTestTracingContext(fs, true))) */ @Test public void testImplicitExplicitFolder1() throws Exception { - Configuration configuration = Mockito.spy(getRawConfiguration()); + Configuration configuration = spy(getRawConfiguration()); try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration)) { final Path implicitPath = new Path("a/b/c"); @@ -2036,24 +2036,24 @@ public void testImplicitExplicitFolder1() throws Exception { fs.mkdirs(new Path("a/b/c/d")); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isImplicitDirectory(new Path("a/b"), fs, getTestTracingContext(fs, true))) .describedAs("Directory 'a/b' should be implicit") .isTrue(); // Asserting that the directory created by mkdir exists as explicit. - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a"), fs, getTestTracingContext(fs, true))) .describedAs("Directory 'a' should be explicit") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), fs, getTestTracingContext(fs, true))) .describedAs("Directory 'a/b/c' should be explicit") .isTrue(); - Assertions.assertThat( + assertThat( DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/d"), fs, getTestTracingContext(fs, true))) .describedAs("Directory 'a/b/c/d' should be explicit") @@ -2096,9 +2096,9 @@ public void testCreatePathRetryIdempotency() throws Exception { "/NonOverwriteTest_FileName_" + UUID.randomUUID()); final List headers = new ArrayList<>(); mockRetriedRequest(abfsClient, headers); - AbfsRestOperation getPathRestOp = Mockito.mock(AbfsRestOperation.class); - AbfsHttpOperation op = Mockito.mock(AbfsHttpOperation.class); - Mockito.doAnswer(answer -> { + AbfsRestOperation getPathRestOp = mock(AbfsRestOperation.class); + AbfsHttpOperation op = mock(AbfsHttpOperation.class); + doAnswer(answer -> { String requiredHeader = null; for (AbfsHttpHeader httpHeader : headers) { if (X_MS_CLIENT_TRANSACTION_ID.equalsIgnoreCase( @@ -2109,12 +2109,12 @@ public void testCreatePathRetryIdempotency() throws Exception { } return requiredHeader; }).when(op).getResponseHeader(X_MS_CLIENT_TRANSACTION_ID); - Mockito.doReturn(true).when(getPathRestOp).hasResult(); - Mockito.doReturn(op).when(getPathRestOp).getResult(); - Mockito.doReturn(getPathRestOp).when(abfsClient).getPathStatus( - Mockito.nullable(String.class), Mockito.nullable(Boolean.class), - Mockito.nullable(TracingContext.class), - Mockito.nullable(ContextEncryptionAdapter.class)); + doReturn(true).when(getPathRestOp).hasResult(); + doReturn(op).when(getPathRestOp).getResult(); + doReturn(getPathRestOp).when(abfsClient).getPathStatus( + nullable(String.class), nullable(Boolean.class), + nullable(TracingContext.class), + nullable(ContextEncryptionAdapter.class)); fs.create(nonOverwriteFile, false); } } @@ -2144,11 +2144,11 @@ public void testGetClientTransactionIdAfterCreate() throws Exception { final AbfsHttpOperation getPathStatusOp = abfsDfsClient.getPathStatus(nonOverwriteFile.toUri().getPath(), false, getTestTracingContext(fs, true), null).getResult(); - Assertions.assertThat( + assertThat( getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID)) .describedAs("Client transaction ID should be set during create") .isNotNull(); - Assertions.assertThat( + assertThat( getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID)) .describedAs("Client transaction ID should be equal to the one set in the header") .isEqualTo(clientTransactionId[0]); @@ -2182,11 +2182,11 @@ public void testClientTransactionIdAfterTwoCreateCalls() throws Exception { final AbfsHttpOperation getPathStatusOp = abfsDfsClient.getPathStatus(testPath.toUri().getPath(), false, getTestTracingContext(fs, true), null).getResult(); - Assertions.assertThat( + assertThat( getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID)) .describedAs("Client transaction ID should be set during create") .isNotNull(); - Assertions.assertThat( + assertThat( getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID)) .describedAs("Client transaction ID should be equal to the one set in the header") .isEqualTo(clientTransactionId[0]); @@ -2214,23 +2214,23 @@ public void testFailureInGetPathStatusDuringCreateRecovery() throws Exception { mockAddClientTransactionIdToHeader(abfsDfsClient, clientTransactionId); mockRetriedRequest(abfsDfsClient, new ArrayList<>()); boolean[] flag = new boolean[1]; - Mockito.doAnswer(getPathStatus -> { + doAnswer(getPathStatus -> { if (!flag[0]) { flag[0] = true; throw new AbfsRestOperationException(HTTP_CLIENT_TIMEOUT, "", "", new Exception()); } return getPathStatus.callRealMethod(); }).when(abfsDfsClient).getPathStatus( - Mockito.nullable(String.class), Mockito.nullable(Boolean.class), - Mockito.nullable(TracingContext.class), - Mockito.nullable(ContextEncryptionAdapter.class)); + nullable(String.class), nullable(Boolean.class), + nullable(TracingContext.class), + nullable(ContextEncryptionAdapter.class)); final Path nonOverwriteFile = new Path( "/NonOverwriteTest_FileName_" + UUID.randomUUID()); String errorMessage = intercept(AbfsDriverException.class, () -> fs.create(nonOverwriteFile, false)).getErrorMessage(); - Assertions.assertThat(errorMessage) + assertThat(errorMessage) .describedAs("getPathStatus should fail while recovering") .contains(ERR_CREATE_RECOVERY); } @@ -2244,13 +2244,13 @@ public void testFailureInGetPathStatusDuringCreateRecovery() throws Exception { * @return A mocked {@link AbfsDfsClient} instance associated with the provided file system. */ private AbfsDfsClient mockIngressClientHandler(AzureBlobFileSystem fs) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler()); - AbfsDfsClient abfsDfsClient = (AbfsDfsClient) Mockito.spy( + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + AbfsClientHandler clientHandler = spy(store.getClientHandler()); + AbfsDfsClient abfsDfsClient = (AbfsDfsClient) spy( clientHandler.getClient()); fs.getAbfsStore().setClient(abfsDfsClient); fs.getAbfsStore().setClientHandler(clientHandler); - Mockito.doReturn(abfsDfsClient).when(clientHandler).getIngressClient(); + doReturn(abfsDfsClient).when(clientHandler).getIngressClient(); return abfsDfsClient; } @@ -2277,12 +2277,12 @@ public void answer(final AbfsRestOperation mockedObj, throws AbfsRestOperationException { if (count == 0) { count = 1; - AbfsHttpOperation op = Mockito.mock(AbfsHttpOperation.class); - Mockito.doReturn(HTTP_METHOD_PUT).when(op).getMethod(); - Mockito.doReturn(EMPTY_STRING).when(op).getStorageErrorMessage(); - Mockito.doReturn(true).when(mockedObj).hasResult(); - Mockito.doReturn(op).when(mockedObj).getResult(); - Mockito.doReturn(HTTP_CONFLICT).when(op).getStatusCode(); + AbfsHttpOperation op = mock(AbfsHttpOperation.class); + doReturn(HTTP_METHOD_PUT).when(op).getMethod(); + doReturn(EMPTY_STRING).when(op).getStorageErrorMessage(); + doReturn(true).when(mockedObj).hasResult(); + doReturn(op).when(mockedObj).getResult(); + doReturn(HTTP_CONFLICT).when(op).getStatusCode(); headers.addAll(mockedObj.getRequestHeaders()); throw new AbfsRestOperationException(HTTP_CONFLICT, AzureServiceErrorCode.PATH_CONFLICT.getErrorCode(), EMPTY_STRING, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java index ed6ffc00f3f2f..8651b75b116cc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java @@ -27,10 +27,7 @@ import java.util.List; import java.util.UUID; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,6 +65,12 @@ import static org.apache.hadoop.fs.permission.AclEntryType.GROUP; import static org.apache.hadoop.fs.permission.AclEntryType.USER; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; /** * Test Perform Authorization Check operation @@ -83,14 +86,14 @@ public class ITestAzureBlobFileSystemDelegationSAS extends AbstractAbfsIntegrati public ITestAzureBlobFileSystemDelegationSAS() throws Exception { // These tests rely on specific settings in azure-auth-keys.xml: String sasProvider = getRawConfiguration().get(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE); - Assume.assumeTrue(MockDelegationSASTokenProvider.class.getCanonicalName().equals(sasProvider)); - Assume.assumeNotNull(getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_ID)); - Assume.assumeNotNull(getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_SECRET)); - Assume.assumeNotNull(getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID)); - Assume.assumeNotNull(getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID)); + assumeTrue(MockDelegationSASTokenProvider.class.getCanonicalName().equals(sasProvider)); + assumeNotNull(getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_ID)); + assumeNotNull(getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_SECRET)); + assumeNotNull(getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID)); + assumeNotNull(getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID)); // The test uses shared key to create a random filesystem and then creates another // instance of this filesystem using SAS authorization. - Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + assumeTrue(this.getAuthType() == AuthType.SharedKey); } @Override @@ -114,10 +117,10 @@ public void testCheckAccess() throws Exception { fs.setOwner(rootPath, MockDelegationSASTokenProvider.TEST_OWNER, null); fs.setPermission(rootPath, new FsPermission(FsAction.ALL, FsAction.READ_EXECUTE, FsAction.EXECUTE)); FileStatus rootStatus = fs.getFileStatus(rootPath); - assertEquals("The directory permissions are not expected.", "rwxr-x--x", rootStatus.getPermission().toString()); - assertEquals("The directory owner is not expected.", - MockDelegationSASTokenProvider.TEST_OWNER, - rootStatus.getOwner()); + assertEquals("The directory permissions are not expected.", "rwxr-x--x", + rootStatus.getPermission().toString()); + assertEquals(MockDelegationSASTokenProvider.TEST_OWNER, + rootStatus.getOwner(), "The directory owner is not expected."); Path dirPath = new Path(UUID.randomUUID().toString()); fs.mkdirs(dirPath); @@ -129,8 +132,10 @@ public void testCheckAccess() throws Exception { FileStatus dirStatus = fs.getFileStatus(dirPath); FileStatus fileStatus = fs.getFileStatus(filePath); - assertEquals("The owner is not expected.", MockDelegationSASTokenProvider.TEST_OWNER, dirStatus.getOwner()); - assertEquals("The owner is not expected.", MockDelegationSASTokenProvider.TEST_OWNER, fileStatus.getOwner()); + assertEquals(MockDelegationSASTokenProvider.TEST_OWNER, dirStatus.getOwner(), + "The owner is not expected."); + assertEquals(MockDelegationSASTokenProvider.TEST_OWNER, fileStatus.getOwner(), + "The owner is not expected."); assertEquals("The directory permissions are not expected.", "rwxr-xr-x", dirStatus.getPermission().toString()); assertEquals("The file permissions are not expected.", "r--r-----", fileStatus.getPermission().toString()); @@ -462,16 +467,16 @@ public void testListAndDeleteImplicitPaths() throws Exception { List list = op.getResult() .getListResultSchema() .paths(); - Assertions.assertThat(list).hasSize(2); + assertThat(list).hasSize(2); client.deletePath(implicitDir.toString(), true, "", getTestTracingContext(fs, false)); - Assertions.assertThat(fs.exists(file1)) + assertThat(fs.exists(file1)) .describedAs("Deleted file1 should not exist.").isFalse(); - Assertions.assertThat(fs.exists(file2)) + assertThat(fs.exists(file2)) .describedAs("Deleted file2 should not exist.").isFalse(); - Assertions.assertThat(fs.exists(implicitDir)) + assertThat(fs.exists(implicitDir)) .describedAs("The parent dir should not exist.") .isFalse(); } @@ -485,10 +490,10 @@ public void testListAndDeleteImplicitPaths() throws Exception { * @return the spied AbfsClient for interaction verification */ private AbfsClient addSpyHooksOnClient(final AzureBlobFileSystem fs) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - Mockito.doReturn(store).when(fs).getAbfsStore(); - AbfsClient client = Mockito.spy(store.getClient()); - Mockito.doReturn(client).when(store).getClient(); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = spy(store.getClient()); + doReturn(client).when(store).getClient(); return client; } @@ -501,10 +506,10 @@ private void checkSignatureMaskAssertions(AbfsRestOperation op) { AbfsHttpOperation result = op.getResult(); String url = result.getMaskedUrl(); String encodedUrl = result.getMaskedEncodedUrl(); - Assertions.assertThat(url.substring(url.indexOf("sig="))) + assertThat(url.substring(url.indexOf("sig="))) .describedAs("Signature query param should be masked") .startsWith("sig=XXXXX"); - Assertions.assertThat(encodedUrl.substring(encodedUrl.indexOf("sig%3D"))) + assertThat(encodedUrl.substring(encodedUrl.indexOf("sig%3D"))) .describedAs("Signature query param should be masked") .startsWith("sig%3DXXXXX"); } @@ -513,7 +518,7 @@ private void checkSignatureMaskAssertions(AbfsRestOperation op) { // Test masking of signature for rename operation for Blob public void testSignatureMaskforBlob() throws Exception { assumeBlobServiceType(); - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.getAbfsStore().setClient(client); @@ -521,35 +526,35 @@ public void testSignatureMaskforBlob() throws Exception { String dest = "/testABC" + "/abc.txt"; fs.create(new Path(src)).close(); - Mockito.doAnswer(answer -> { - Path srcCopy = answer.getArgument(0); - Path dstCopy = answer.getArgument(1); - String leaseId = answer.getArgument(2); - TracingContext tracingContext = answer.getArgument(3); - AbfsRestOperation op - = ((AbfsBlobClient) getFileSystem().getAbfsClient()).copyBlob(srcCopy, - dstCopy, leaseId, tracingContext); - checkSignatureMaskAssertions(op); - return answer.callRealMethod(); - }) - .when(client) - .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.any(String.class), Mockito.any(TracingContext.class)); - - Mockito.doAnswer(answer -> { - Path blobPath = answer.getArgument(0); - String leaseId = answer.getArgument(1); - TracingContext tracingContext = answer.getArgument(2); - AbfsRestOperation op - = ((AbfsBlobClient) getFileSystem().getAbfsClient()).deleteBlobPath( - blobPath, - leaseId, tracingContext); - checkSignatureMaskAssertions(op); - return answer.callRealMethod(); - }) - .when(client) - .deleteBlobPath(Mockito.any(Path.class), Mockito.any(String.class), - Mockito.any(TracingContext.class)); + doAnswer(answer -> { + Path srcCopy = answer.getArgument(0); + Path dstCopy = answer.getArgument(1); + String leaseId = answer.getArgument(2); + TracingContext tracingContext = answer.getArgument(3); + AbfsRestOperation op = + ((AbfsBlobClient) getFileSystem().getAbfsClient()).copyBlob(srcCopy, + dstCopy, leaseId, tracingContext); + checkSignatureMaskAssertions(op); + return answer.callRealMethod(); + }) + .when(client) + .copyBlob(any(Path.class), any(Path.class), + any(String.class), any(TracingContext.class)); + + doAnswer(answer -> { + Path blobPath = answer.getArgument(0); + String leaseId = answer.getArgument(1); + TracingContext tracingContext = answer.getArgument(2); + AbfsRestOperation op = + ((AbfsBlobClient) getFileSystem().getAbfsClient()).deleteBlobPath( + blobPath, + leaseId, tracingContext); + checkSignatureMaskAssertions(op); + return answer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(any(Path.class), any(String.class), + any(TracingContext.class)); client.renamePath(src, dest, null, getTestTracingContext(fs, false), null, @@ -591,9 +596,8 @@ public void testSetPermissionForNonOwner() throws Exception { assertEquals("The permissions are not expected.", "rwxr-x---", rootStatus.getPermission().toString()); - assertNotEquals("The owner is not expected.", - MockDelegationSASTokenProvider.TEST_OWNER, - rootStatus.getOwner()); + assertNotEquals(MockDelegationSASTokenProvider.TEST_OWNER, + rootStatus.getOwner(), "The owner is not expected."); // Attempt to set permission without being the owner. intercept(AccessDeniedException.class, @@ -611,9 +615,8 @@ public void testSetPermissionForNonOwner() throws Exception { assertEquals("The permissions are not expected.", "rwxr-x--x", rootStatus.getPermission().toString()); - assertEquals("The directory owner is not expected.", - MockDelegationSASTokenProvider.TEST_OWNER, - rootStatus.getOwner()); + assertEquals(MockDelegationSASTokenProvider.TEST_OWNER, + rootStatus.getOwner(), "The directory owner is not expected."); } @Test @@ -628,9 +631,8 @@ public void testSetPermissionWithoutAgentForNonOwner() throws Exception { assertEquals("The permissions are not expected.", "rw-r--r--", status.getPermission().toString()); - assertNotEquals("The owner is not expected.", - TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID, - status.getOwner()); + assertNotEquals(TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID, + status.getOwner(), "The owner is not expected."); fs.setPermission(path, new FsPermission(FsAction.READ, FsAction.READ, FsAction.NONE)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index e20a2c43279ad..99c136fb985de 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -29,10 +29,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileAlreadyExistsException; @@ -67,11 +64,22 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.mockito.ArgumentMatchers.any; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; /** * Test delete operation. @@ -106,7 +114,7 @@ public void testDeleteRoot() throws Exception { fs.delete(root, true); ls = fs.listStatus(root); - assertEquals("listing size", 0, ls.length); + assertEquals(0, ls.length, "listing size"); } @Test() @@ -180,7 +188,7 @@ public Void call() throws Exception { @Test public void testDeleteIdempotency() throws Exception { - Assume.assumeTrue(DEFAULT_DELETE_CONSIDERED_IDEMPOTENT); + assumeTrue(DEFAULT_DELETE_CONSIDERED_IDEMPOTENT); // Config to reduce the retry and maxBackoff time for test run AbfsConfiguration abfsConfig = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( @@ -207,7 +215,7 @@ public void testDeleteIdempotency() throws Exception { when(op.getResult()).thenReturn(http504Op); when(op.hasResult()).thenReturn(true); - Assertions.assertThat(testClient.deleteIdempotencyCheckOp(op) + assertThat(testClient.deleteIdempotencyCheckOp(op) .getResult() .getStatusCode()) .describedAs( @@ -223,7 +231,7 @@ public void testDeleteIdempotency() throws Exception { when(op.getResult()).thenReturn(http404Op); when(op.hasResult()).thenReturn(true); - Assertions.assertThat(testClient.deleteIdempotencyCheckOp(op) + assertThat(testClient.deleteIdempotencyCheckOp(op) .getResult() .getStatusCode()) .describedAs( @@ -239,7 +247,7 @@ public void testDeleteIdempotency() throws Exception { when(op.getResult()).thenReturn(http400Op); when(op.hasResult()).thenReturn(true); - Assertions.assertThat(testClient.deleteIdempotencyCheckOp(op) + assertThat(testClient.deleteIdempotencyCheckOp(op) .getResult() .getStatusCode()) .describedAs( @@ -295,8 +303,8 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // Idempotency check on Delete always returns success StorageErrorResponseSchema storageErrorResponse = new StorageErrorResponseSchema( "NotFound", "NotFound", "NotFound"); - Mockito.doReturn(storageErrorResponse).when(mockClient).processStorageErrorResponse(any()); - AbfsRestOperation idempotencyRetOp = Mockito.spy(ITestAbfsClient.getRestOp( + doReturn(storageErrorResponse).when(mockClient).processStorageErrorResponse(any()); + AbfsRestOperation idempotencyRetOp = spy(ITestAbfsClient.getRestOp( DeletePath, mockClient, HTTP_METHOD_DELETE, ITestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"), ITestAbfsClient.getTestRequestHeaders(mockClient), getConfiguration())); @@ -307,25 +315,25 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { doReturn(tracingContext).when(idempotencyRetOp).createNewTracingContext(any()); if (mockClient instanceof AbfsBlobClient) { doCallRealMethod().when((AbfsBlobClient) mockClient) - .getBlobDeleteHandler(Mockito.nullable(String.class), - Mockito.anyBoolean(), Mockito.nullable(TracingContext.class)); + .getBlobDeleteHandler(nullable(String.class), + anyBoolean(), nullable(TracingContext.class)); doCallRealMethod().when(mockClient) - .listPath(Mockito.nullable(String.class), Mockito.anyBoolean(), - Mockito.anyInt(), Mockito.nullable(String.class), - Mockito.nullable(TracingContext.class), Mockito.nullable(URI.class)); + .listPath(nullable(String.class), anyBoolean(), + anyInt(), nullable(String.class), + nullable(TracingContext.class), nullable(URI.class)); doCallRealMethod().when((AbfsBlobClient) mockClient) - .listPath(Mockito.nullable(String.class), Mockito.anyBoolean(), - Mockito.anyInt(), Mockito.nullable(String.class), - Mockito.nullable(TracingContext.class), Mockito.nullable(URI.class), Mockito.anyBoolean()); + .listPath(nullable(String.class), anyBoolean(), + anyInt(), nullable(String.class), + nullable(TracingContext.class), nullable(URI.class), anyBoolean()); doCallRealMethod().when((AbfsBlobClient) mockClient) - .getPathStatus(Mockito.nullable(String.class), Mockito.nullable(TracingContext.class), - Mockito.nullable(ContextEncryptionAdapter.class), Mockito.anyBoolean()); + .getPathStatus(nullable(String.class), nullable(TracingContext.class), + nullable(ContextEncryptionAdapter.class), anyBoolean()); } when(mockClient.deletePath("/NonExistingPath", false, null, tracingContext)) .thenCallRealMethod(); - Assertions.assertThat(mockClient.deletePath( + assertThat(mockClient.deletePath( "/NonExistingPath", false, null, @@ -344,12 +352,12 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { public void deleteBlobDirParallelThreadToDeleteOnDifferentTracingContext() throws Exception { Configuration configuration = getRawConfiguration(); - AzureBlobFileSystem fs = Mockito.spy( + AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(configuration)); - AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); - AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + AzureBlobFileSystemStore spiedStore = spy(fs.getAbfsStore()); + AbfsClient spiedClient = spy(fs.getAbfsClient()); - Mockito.doReturn(spiedStore).when(fs).getAbfsStore(); + doReturn(spiedStore).when(fs).getAbfsStore(); spiedStore.setClient(spiedClient); fs.mkdirs(new Path("/testDir")); @@ -383,14 +391,14 @@ public void testDeleteFileInImplicitDir() throws Exception { fs.delete(file1, false); fs.delete(file2, true); - Assertions.assertThat(fs.exists(implicitDir)) + assertThat(fs.exists(implicitDir)) .describedAs("The directory should exist.") .isTrue(); - Assertions.assertThat(fs.exists(file1)) + assertThat(fs.exists(file1)) .describedAs("Deleted file should not be present.").isFalse(); - Assertions.assertThat(fs.exists(file2)) + assertThat(fs.exists(file2)) .describedAs("Deleted file should not be present.").isFalse(); - Assertions.assertThat(fs.exists(implicitDir)) + assertThat(fs.exists(implicitDir)) .describedAs("The parent dir should exist.") .isTrue(); } @@ -410,7 +418,7 @@ public void testDeleteEmptyExplicitDir() throws Exception { fs.mkdirs(p1); fs.delete(p1, false); - Assertions.assertThat(fs.exists(p1)) + assertThat(fs.exists(p1)) .describedAs("The deleted directory should not exist.") .isFalse(); } @@ -440,7 +448,7 @@ public void testDeleteNonEmptyExplicitDir() throws Exception { intercept(FileAlreadyExistsException.class, () -> fs.delete(p2, false)); - Assertions.assertThat(!fs.exists(p1)) + assertThat(!fs.exists(p1)) .describedAs("FileStatus of the deleted directory should not exist.") .isTrue(); } @@ -456,7 +464,7 @@ public void testDeleteNonExistingPath() throws Exception { AzureBlobFileSystem fs = getFileSystem(); Path p = new Path("/nonExistingPath"); - Assertions.assertThat(fs.delete(p, true)) + assertThat(fs.delete(p, true)) .describedAs("Delete operation on non-existing path should return false") .isFalse(); } @@ -474,7 +482,7 @@ public void testExceptionForDeletedFile() throws Exception { fs.create(testFile); fs.delete(testFile, false); - Assertions.assertThat(fs.delete(testFile, true)) + assertThat(fs.delete(testFile, true)) .describedAs("Delete operation on deleted path should return false.") .isFalse(); } @@ -500,10 +508,10 @@ public void testDeleteImplicitDir() throws Exception { intercept(FileAlreadyExistsException.class, () -> fs.delete(new Path("/testDir/dir1"), false)); fs.delete(new Path("/testDir/dir1"), true); - Assertions.assertThat(!fs.exists(new Path("/testDir/dir1"))) + assertThat(!fs.exists(new Path("/testDir/dir1"))) .describedAs("FileStatus of the deleted directory should not exist") .isTrue(); - Assertions.assertThat(!fs.exists(new Path("/testDir/dir1/file1"))) + assertThat(!fs.exists(new Path("/testDir/dir1/file1"))) .describedAs("Child of a deleted directory should not be present") .isTrue(); } @@ -520,13 +528,13 @@ public void testDeleteImplicitDirWithSingleListResults() throws Exception { getRawConfiguration()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); - AbfsBlobClient spiedClient = Mockito.spy(client); + AbfsBlobClient spiedClient = spy(client); fs.getAbfsStore().setClient(spiedClient); fs.mkdirs(new Path("/testDir/dir1")); for (int i = 0; i < 10; i++) { fs.create(new Path("/testDir/dir1/file" + i)); } - Mockito.doAnswer(answer -> { + doAnswer(answer -> { String path = answer.getArgument(0); boolean recursive = answer.getArgument(1); String continuation = answer.getArgument(3); @@ -534,13 +542,13 @@ public void testDeleteImplicitDirWithSingleListResults() throws Exception { return client.listPath(path, recursive, 1, continuation, context, null); }) .when(spiedClient) - .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class), Mockito.nullable(URI.class)); + .listPath(anyString(), anyBoolean(), anyInt(), + nullable(String.class), + any(TracingContext.class), nullable(URI.class)); client.deleteBlobPath(new Path("/testDir/dir1"), null, getTestTracingContext(fs, true)); fs.delete(new Path("/testDir/dir1"), true); - Assertions.assertThat(fs.exists(new Path("/testDir/dir1"))) + assertThat(fs.exists(new Path("/testDir/dir1"))) .describedAs("FileStatus of the deleted directory should not exist") .isFalse(); } @@ -561,13 +569,13 @@ public void testDeleteExplicitDirInImplicitParentDir() throws Exception { client.deleteBlobPath(new Path("/testDir/"), null, getTestTracingContext(fs, true)); fs.delete(new Path("/testDir/dir1"), true); - Assertions.assertThat(fs.exists(new Path("/testDir/dir1"))) + assertThat(fs.exists(new Path("/testDir/dir1"))) .describedAs("Deleted directory should not exist") .isFalse(); - Assertions.assertThat(fs.exists(new Path("/testDir/dir1/file1"))) + assertThat(fs.exists(new Path("/testDir/dir1/file1"))) .describedAs("Child of a deleted directory should not be present") .isFalse(); - Assertions.assertThat(fs.exists(new Path("/testDir"))) + assertThat(fs.exists(new Path("/testDir"))) .describedAs("Parent Implicit directory should exist") .isTrue(); } @@ -581,21 +589,21 @@ public void testDeleteExplicitDirInImplicitParentDir() throws Exception { */ @Test public void testDeleteParallelBlobFailure() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); - AbfsBlobClient client = Mockito.spy((AbfsBlobClient) fs.getAbfsClient()); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsBlobClient client = spy((AbfsBlobClient) fs.getAbfsClient()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); store.setClient(client); - Mockito.doReturn(store).when(fs).getAbfsStore(); + doReturn(store).when(fs).getAbfsStore(); fs.mkdirs(new Path("/testDir")); fs.create(new Path("/testDir/file1")); fs.create(new Path("/testDir/file2")); fs.create(new Path("/testDir/file3")); - Mockito.doThrow( + doThrow( new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", new Exception())) .when(client) - .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), nullable(String.class), + any(TracingContext.class)); LambdaTestUtils.intercept( AccessDeniedException.class, () -> { @@ -614,7 +622,7 @@ public void testDeleteParallelBlobFailure() throws Exception { public void testDeleteRootWithNonRecursion() throws Exception { AzureBlobFileSystem fs = getFileSystem(); fs.mkdirs(new Path("/testDir")); - Assertions.assertThat(fs.delete(new Path(ROOT_PATH), false)).isFalse(); + assertThat(fs.delete(new Path(ROOT_PATH), false)).isFalse(); } /** @@ -627,8 +635,8 @@ public void testDeleteRootWithNonRecursion() throws Exception { @Test public void testProducerStopOnDeleteFailure() throws Exception { assumeBlobServiceType(); - Configuration configuration = Mockito.spy(getRawConfiguration()); - AzureBlobFileSystem fs = Mockito.spy( + Configuration configuration = spy(getRawConfiguration()); + AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.get(configuration)); fs.mkdirs(new Path("/src")); ExecutorService executorService = Executors.newFixedThreadPool(10); @@ -646,20 +654,20 @@ public void testProducerStopOnDeleteFailure() throws Exception { future.get(); } AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); - AbfsBlobClient spiedClient = Mockito.spy(client); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsBlobClient spiedClient = spy(client); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); store.setClient(spiedClient); - Mockito.doReturn(store).when(fs).getAbfsStore(); + doReturn(store).when(fs).getAbfsStore(); final int[] deleteCallInvocation = new int[1]; - Mockito.doAnswer(answer -> { + doAnswer(answer -> { throw new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", new Exception()); }).when(spiedClient) - .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), nullable(String.class), + any(TracingContext.class)); AbfsClientTestUtil.mockGetDeleteBlobHandler(spiedClient, (blobDeleteHandler) -> { - Mockito.doAnswer(answer -> { + doAnswer(answer -> { try { answer.callRealMethod(); } catch (AbfsRestOperationException ex) { @@ -674,7 +682,7 @@ public void testProducerStopOnDeleteFailure() throws Exception { return null; }); final int[] listCallInvocation = new int[1]; - Mockito.doAnswer(answer -> { + doAnswer(answer -> { if (listCallInvocation[0] == 1) { while (deleteCallInvocation[0] == 0) {} } @@ -682,15 +690,15 @@ public void testProducerStopOnDeleteFailure() throws Exception { return answer.callRealMethod(); }) .when(spiedClient) - .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), - Mockito.nullable(String.class), Mockito.any(TracingContext.class), Mockito.nullable(URI.class)); + .listPath(anyString(), anyBoolean(), anyInt(), + nullable(String.class), any(TracingContext.class), nullable(URI.class)); intercept(AccessDeniedException.class, () -> { fs.delete(new Path("/src"), true); }); - Mockito.verify(spiedClient, Mockito.times(1)) - .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), - Mockito.nullable(String.class), Mockito.any(TracingContext.class), Mockito.nullable(URI.class)); + verify(spiedClient, times(1)) + .listPath(anyString(), anyBoolean(), anyInt(), + nullable(String.class), any(TracingContext.class), nullable(URI.class)); } /** @@ -703,13 +711,13 @@ public void testProducerStopOnDeleteFailure() throws Exception { */ @Test public void testDeleteEmitDeletionCountInClientRequestId() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); - AbfsBlobClient spiedClient = Mockito.spy(client); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsBlobClient spiedClient = spy(client); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); store.setClient(spiedClient); - Mockito.doReturn(store).when(fs).getAbfsStore(); + doReturn(store).when(fs).getAbfsStore(); String dirPathStr = "/testDir/dir1"; fs.mkdirs(new Path(dirPathStr)); ExecutorService executorService = Executors.newFixedThreadPool(5); @@ -728,8 +736,8 @@ public void testDeleteEmitDeletionCountInClientRequestId() throws Exception { fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.DELETE, false, 0); fs.registerListener(tracingHeaderValidator); - Mockito.doAnswer(answer -> { - Mockito.doAnswer(deleteAnswer -> { + doAnswer(answer -> { + doAnswer(deleteAnswer -> { if (dirPathStr.equalsIgnoreCase( ((Path) deleteAnswer.getArgument(0)).toUri().getPath())) { tracingHeaderValidator.setOperatedBlobCount(BLOB_COUNT); @@ -740,14 +748,14 @@ public void testDeleteEmitDeletionCountInClientRequestId() throws Exception { return deleteAnswer.callRealMethod(); }) .when(spiedClient) - .deleteBlobPath(Mockito.any(Path.class), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), + nullable(String.class), + any(TracingContext.class)); return answer.callRealMethod(); }) .when(store) - .delete(Mockito.any(Path.class), Mockito.anyBoolean(), - Mockito.any(TracingContext.class)); + .delete(any(Path.class), anyBoolean(), + any(TracingContext.class)); fs.delete(new Path(dirPathStr), true); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java index efb488988e39f..b9ab03bb05d24 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java @@ -27,8 +27,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient; import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -46,6 +45,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; /** * Test end to end between ABFS client and ABFS server. @@ -83,36 +83,37 @@ public void testReadWriteBytesToFile() throws Exception { } } - @Test (expected = IOException.class) + @Test public void testOOBWritesAndReadFail() throws Exception { - Configuration conf = this.getRawConfiguration(); - conf.setBoolean(AZURE_TOLERATE_CONCURRENT_APPEND, false); - final AzureBlobFileSystem fs = getFileSystem(); - int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize(); - - byte[] bytesToRead = new byte[readBufferSize]; - final byte[] b = new byte[2 * readBufferSize]; - new Random().nextBytes(b); - - final Path testFilePath = path(methodName.getMethodName()); - try(FSDataOutputStream writeStream = fs.create(testFilePath)) { - writeStream.write(b); - writeStream.flush(); + assertThrows(IOException.class, () -> { + Configuration conf = this.getRawConfiguration(); + conf.setBoolean(AZURE_TOLERATE_CONCURRENT_APPEND, false); + final AzureBlobFileSystem fs = getFileSystem(); + int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize(); + byte[] bytesToRead = new byte[readBufferSize]; + final byte[] b = new byte[2 * readBufferSize]; + new Random().nextBytes(b); + final Path testFilePath = path(methodName.getMethodName()); + + try(FSDataOutputStream writeStream = fs.create(testFilePath)) { + writeStream.write(b); + writeStream.flush(); + } + + try (FSDataInputStream readStream = fs.open(testFilePath)) { + assertEquals(readBufferSize, + readStream.read(bytesToRead, 0, readBufferSize)); + try (FSDataOutputStream writeStream = fs.create(testFilePath)) { + writeStream.write(b); + writeStream.flush(); + } + + assertEquals(readBufferSize, + readStream.read(bytesToRead, 0, readBufferSize)); + } + }); } - try (FSDataInputStream readStream = fs.open(testFilePath)) { - assertEquals(readBufferSize, - readStream.read(bytesToRead, 0, readBufferSize)); - try (FSDataOutputStream writeStream = fs.create(testFilePath)) { - writeStream.write(b); - writeStream.flush(); - } - - assertEquals(readBufferSize, - readStream.read(bytesToRead, 0, readBufferSize)); - } - } - @Test public void testOOBWritesAndReadSucceed() throws Exception { Configuration conf = this.getRawConfiguration(); @@ -264,11 +265,13 @@ public void testHttpConnectionTimeout() throws Exception { TEST_STABLE_DEFAULT_READ_TIMEOUT_MS); } - @Test(expected = InvalidAbfsRestOperationException.class) + @Test public void testHttpReadTimeout() throws Exception { - // Small read timeout is bound to make the request fail. - testHttpTimeouts(TEST_STABLE_DEFAULT_CONNECTION_TIMEOUT_MS, + assertThrows(InvalidAbfsRestOperationException.class, () -> { + // Small read timeout is bound to make the request fail. + testHttpTimeouts(TEST_STABLE_DEFAULT_CONNECTION_TIMEOUT_MS, TEST_UNSTABLE_READ_TIMEOUT_MS); + }); } public void testHttpTimeouts(int connectionTimeoutMs, int readTimeoutMs) @@ -285,11 +288,11 @@ public void testHttpTimeouts(int connectionTimeoutMs, int readTimeoutMs) // Reduce retry count to reduce test run time conf.setInt(AZURE_MAX_IO_RETRIES, 1); final AzureBlobFileSystem fs = getFileSystem(conf); - Assertions.assertThat( + assertThat( fs.getAbfsStore().getAbfsConfiguration().getHttpConnectionTimeout()) .describedAs("HTTP connection time should be picked from config") .isEqualTo(connectionTimeoutMs); - Assertions.assertThat( + assertThat( fs.getAbfsStore().getAbfsConfiguration().getHttpReadTimeout()) .describedAs("HTTP Read time should be picked from config") .isEqualTo(readTimeoutMs); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java index fccd0632375d3..64fd96ad63989 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java @@ -26,7 +26,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -109,12 +109,12 @@ public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception { } String stats = abfsStatistics.toString(); - assertEquals("Bytes read in " + stats, - remoteData.length, abfsStatistics.getBytesRead()); - assertEquals("bytes written in " + stats, - sourceData.length, abfsStatistics.getBytesWritten()); - assertEquals("bytesRead from read() call", testBufferSize, bytesRead); - assertArrayEquals("round tripped data", sourceData, remoteData); + assertEquals(remoteData.length, abfsStatistics.getBytesRead(), + "Bytes read in " + stats); + assertEquals(sourceData.length, abfsStatistics.getBytesWritten(), + "bytes written in " + stats); + assertEquals(testBufferSize, bytesRead, "bytesRead from read() call"); + assertArrayEquals(sourceData, remoteData, "round tripped data"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java index 2d7298f1e1bd4..59d0908b7c88f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java @@ -22,9 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; -import org.assertj.core.api.Assertions; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -37,8 +35,13 @@ import static org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.mockIngressClientHandler; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; /** * Test FileStatus. @@ -63,7 +66,7 @@ public void testEnsureStatusWorksForRoot() throws Exception { Path root = new Path("/"); FileStatus[] rootls = fs.listStatus(root); - assertEquals("root listing", 0, rootls.length); + assertEquals(0, rootls.length, "root listing"); } @Test @@ -82,23 +85,23 @@ private FileStatus validateStatus(final AzureBlobFileSystem fs, final Path name, String errorInStatus = "error in " + fileStatus + " from " + fs; if (!getIsNamespaceEnabled(fs)) { - assertEquals(errorInStatus + ": owner", - fs.getOwnerUser(), fileStatus.getOwner()); - assertEquals(errorInStatus + ": group", - fs.getOwnerUserPrimaryGroup(), fileStatus.getGroup()); + assertEquals(fs.getOwnerUser(), + fileStatus.getOwner(), errorInStatus + ": owner"); + assertEquals(fs.getOwnerUserPrimaryGroup(), + fileStatus.getGroup(), errorInStatus + ": group"); assertEquals(new FsPermission(FULL_PERMISSION), fileStatus.getPermission()); } else { // When running with namespace enabled account, // the owner and group info retrieved from server will be digit ids. // hence skip the owner and group validation if (isDir) { - assertEquals(errorInStatus + ": permission", - new FsPermission(DEFAULT_DIR_PERMISSION_VALUE), fileStatus.getPermission()); - assertTrue(errorInStatus + "not a directory", fileStatus.isDirectory()); + assertEquals(new FsPermission(DEFAULT_DIR_PERMISSION_VALUE), fileStatus.getPermission(), + errorInStatus + ": permission"); + assertTrue(fileStatus.isDirectory(), errorInStatus + "not a directory"); } else { - assertEquals(errorInStatus + ": permission", - new FsPermission(DEFAULT_FILE_PERMISSION_VALUE), fileStatus.getPermission()); - assertTrue(errorInStatus + "not a file", fileStatus.isFile()); + assertEquals(new FsPermission(DEFAULT_FILE_PERMISSION_VALUE), fileStatus.getPermission(), + errorInStatus + ": permission"); + assertTrue(fileStatus.isFile(), errorInStatus + "not a file"); } } assertPathDns(fileStatus.getPath()); @@ -153,10 +156,10 @@ public void testLastModifiedTime() throws IOException { long createEndTime = System.currentTimeMillis(); FileStatus fStat = fs.getFileStatus(testFilePath); long lastModifiedTime = fStat.getModificationTime(); - assertTrue("lastModifiedTime should be after minCreateStartTime", - minCreateStartTime < lastModifiedTime); - assertTrue("lastModifiedTime should be before createEndTime", - createEndTime > lastModifiedTime); + assertTrue(minCreateStartTime < lastModifiedTime, + "lastModifiedTime should be after minCreateStartTime"); + assertTrue(createEndTime > lastModifiedTime, + "lastModifiedTime should be before createEndTime"); } /** @@ -247,19 +250,19 @@ public void testFileStatusOnImplicitPath() throws Exception { @Test public void testListStatusIsCalledForImplicitPathOnBlobEndpoint() throws Exception { assumeBlobServiceType(); - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - Mockito.doReturn(store).when(fs).getAbfsStore(); - AbfsBlobClient abfsClient = Mockito.spy(store.getClientHandler().getBlobClient()); - Mockito.doReturn(abfsClient).when(store).getClient(); + AzureBlobFileSystem fs = spy(getFileSystem()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + doReturn(store).when(fs).getAbfsStore(); + AbfsBlobClient abfsClient = spy(store.getClientHandler().getBlobClient()); + doReturn(abfsClient).when(store).getClient(); Path implicitPath = path("implicitDir"); createAzCopyFolder(implicitPath); fs.getFileStatus(implicitPath); - Mockito.verify(abfsClient, Mockito.times(1)).getPathStatus(any(), eq(false), any(), any()); - Mockito.verify(abfsClient, Mockito.times(1)).listPath(any(), eq(false), eq(1), any(), any(), any(), eq(false)); + verify(abfsClient, times(1)).getPathStatus(any(), eq(false), any(), any()); + verify(abfsClient, times(1)).listPath(any(), eq(false), eq(1), any(), any(), any(), eq(false)); } /** @@ -268,12 +271,12 @@ public void testListStatusIsCalledForImplicitPathOnBlobEndpoint() throws Excepti * @param isDir */ private void verifyFileStatus(FileStatus fileStatus, boolean isDir) { - Assertions.assertThat(fileStatus).isNotNull(); + assertThat(fileStatus).isNotNull(); if (isDir) { - Assertions.assertThat(fileStatus.getLen()).isEqualTo(0); - Assertions.assertThat(fileStatus.isDirectory()).isTrue(); + assertThat(fileStatus.getLen()).isEqualTo(0); + assertThat(fileStatus.isDirectory()).isTrue(); } else { - Assertions.assertThat(fileStatus.isFile()).isTrue(); + assertThat(fileStatus.isFile()).isTrue(); } assertPathDns(fileStatus.getPath()); } @@ -284,8 +287,8 @@ private void verifyFileStatus(FileStatus fileStatus, boolean isDir) { * @param key */ private void verifyFileNotFound(FileNotFoundException ex, String key) { - Assertions.assertThat(ex).isNotNull(); - Assertions.assertThat(ex.getMessage()).contains(key); + assertThat(ex).isNotNull(); + assertThat(ex.getMessage()).contains(key); } /** @@ -293,7 +296,7 @@ private void verifyFileNotFound(FileNotFoundException ex, String key) { * verifying the correct header and directory state. */ private void testIsDirectory(boolean expected, String... configName) throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(getFileSystem())) { + try (AzureBlobFileSystem fs = spy(getFileSystem())) { assumeBlobServiceType(); AbfsBlobClient abfsBlobClient = mockIngressClientHandler(fs); // Mock the operation to modify the headers @@ -311,12 +314,12 @@ private void testIsDirectory(boolean expected, String... configName) throws Exce true, getTestTracingContext(fs, true), null).getResult(); - Assertions.assertThat(abfsBlobClient.checkIsDir(op)) + assertThat(abfsBlobClient.checkIsDir(op)) .describedAs("Directory should be marked as " + expected) .isEqualTo(expected); // Verify the header and directory state - Assertions.assertThat(fileStatus.isDirectory()) + assertThat(fileStatus.isDirectory()) .describedAs("Expected directory state: " + expected) .isEqualTo(expected); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java index 3c21525549bfe..cd1c3aa57cdc8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java @@ -20,8 +20,7 @@ import java.lang.ref.WeakReference; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -57,6 +56,6 @@ public void testFinalize() throws Exception { i++; } - Assert.assertTrue("testFinalizer didn't get cleaned up within maxTries", ref.get() == null); + assertTrue(ref.get() == null, "testFinalizer didn't get cleaned up within maxTries"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index b55032c5132a5..2f2257e23e301 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -35,9 +35,7 @@ import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; -import org.hamcrest.core.IsEqual; -import org.hamcrest.core.IsNot; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -48,6 +46,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APPEND_BLOB_KEY; 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 flush operation. @@ -94,8 +93,8 @@ public void testAbfsOutputStreamAsyncFlushWithRetainUncommittedData() throws Exc while (inputStream.available() != 0) { int result = inputStream.read(r); - assertNotEquals("read returned -1", -1, result); - assertArrayEquals("buffer read from stream", r, b); + assertNotEquals(-1, result, "read returned -1"); + assertArrayEquals(r, b, "buffer read from stream"); } } } @@ -170,7 +169,7 @@ public Void call() throws Exception { es.shutdownNow(); FileStatus fileStatus = fs.getFileStatus(testFilePath); long expectedWrites = (long) TEST_BUFFER_SIZE * FLUSH_TIMES; - assertEquals("Wrong file length in " + testFilePath, expectedWrites, fileStatus.getLen()); + assertEquals(expectedWrites, fileStatus.getLen(), "Wrong file length in " + testFilePath); } @Test @@ -400,15 +399,11 @@ private void validate(InputStream stream, byte[] writeBuffer, boolean isEqual) int numBytesRead = stream.read(readBuffer, 0, readBuffer.length); if (isEqual) { - assertArrayEquals( - "Bytes read do not match bytes written.", - writeBuffer, - readBuffer); + assertArrayEquals(writeBuffer, + readBuffer, "Bytes read do not match bytes written."); } else { - assertThat( - "Bytes read unexpectedly match bytes written.", - readBuffer, - IsNot.not(IsEqual.equalTo(writeBuffer))); + assertThat(readBuffer).isEqualTo(writeBuffer). + as("Bytes read unexpectedly match bytes written."); } } finally { stream.close(); @@ -420,14 +415,11 @@ private void validate(FileSystem fs, Path path, byte[] writeBuffer, boolean isEq byte[] readBuffer = new byte[TEST_FILE_LENGTH]; int numBytesRead = inputStream.read(readBuffer, 0, readBuffer.length); if (isEqual) { - assertArrayEquals( - String.format("Bytes read do not match bytes written to %1$s", filePath), writeBuffer, readBuffer); + assertArrayEquals(writeBuffer, readBuffer, + String.format("Bytes read do not match bytes written to %1$s", filePath)); } else { - assertThat( - String.format("Bytes read unexpectedly match bytes written to %1$s", - filePath), - readBuffer, - IsNot.not(IsEqual.equalTo(writeBuffer))); + assertThat(readBuffer).isNotEqualTo(writeBuffer).as( + String.format("Bytes read unexpectedly match bytes written to %1$s", filePath)); } } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java index 35ce615ba738c..414723b47eb14 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java @@ -25,8 +25,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; @@ -50,6 +49,13 @@ import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.INCORRECT_INGRESS_TYPE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; /** * Test filesystem initialization and creation. @@ -70,52 +76,54 @@ public void setup() throws Exception { public void teardown() { } - @Test (expected = FileNotFoundException.class) + @Test public void ensureFilesystemWillNotBeCreatedIfCreationConfigIsNotSet() throws Exception { - final AzureBlobFileSystem fs = this.createFileSystem(); - FileStatus[] fileStatuses = fs.listStatus(new Path("/")); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = this.createFileSystem(); + FileStatus[] fileStatuses = fs.listStatus(new Path("/")); + }); } @Test public void testGetAclCallOnHnsConfigAbsence() throws Exception { AzureBlobFileSystem fs = ((AzureBlobFileSystem) FileSystem.newInstance( getRawConfiguration())); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - AbfsClient client = Mockito.spy(fs.getAbfsStore().getClient(AbfsServiceType.DFS)); - Mockito.doReturn(client).when(store).getClient(AbfsServiceType.DFS); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + AbfsClient client = spy(fs.getAbfsStore().getClient(AbfsServiceType.DFS)); + doReturn(client).when(store).getClient(AbfsServiceType.DFS); - Mockito.doThrow(TrileanConversionException.class) + doThrow(TrileanConversionException.class) .when(store) .isNamespaceEnabled(); store.setNamespaceEnabled(Trilean.UNKNOWN); TracingContext tracingContext = getSampleTracingContext(fs, true); - Mockito.doReturn(Mockito.mock(AbfsRestOperation.class)) + doReturn(mock(AbfsRestOperation.class)) .when(client) - .getAclStatus(Mockito.anyString(), any(TracingContext.class)); + .getAclStatus(anyString(), any(TracingContext.class)); store.getIsNamespaceEnabled(tracingContext); - Mockito.verify(client, Mockito.times(1)) - .getAclStatus(Mockito.anyString(), any(TracingContext.class)); + verify(client, times(1)) + .getAclStatus(anyString(), any(TracingContext.class)); } @Test public void testNoGetAclCallOnHnsConfigPresence() throws Exception { AzureBlobFileSystem fs = ((AzureBlobFileSystem) FileSystem.newInstance( getRawConfiguration())); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - AbfsClient client = Mockito.spy(fs.getAbfsClient()); - Mockito.doReturn(client).when(store).getClient(); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + AbfsClient client = spy(fs.getAbfsClient()); + doReturn(client).when(store).getClient(); - Mockito.doReturn(true) + doReturn(true) .when(store) .isNamespaceEnabled(); TracingContext tracingContext = getSampleTracingContext(fs, true); store.getIsNamespaceEnabled(tracingContext); - Mockito.verify(client, Mockito.times(0)) - .getAclStatus(Mockito.anyString(), any(TracingContext.class)); + verify(client, times(0)) + .getAclStatus(anyString(), any(TracingContext.class)); } /** @@ -149,8 +157,8 @@ public void testFileSystemInitializationFailsForInvalidIngress() throws Exceptio public void testFileSystemInitFailsIfNotAbleToDetermineAccountType() throws Exception { AzureBlobFileSystem fs = ((AzureBlobFileSystem) FileSystem.newInstance( getRawConfiguration())); - AzureBlobFileSystem mockedFs = Mockito.spy(fs); - Mockito.doThrow( + AzureBlobFileSystem mockedFs = spy(fs); + doThrow( new AbfsRestOperationException(HTTP_UNAVAILABLE, "Throttled", "Throttled", null)).when(mockedFs).getIsNamespaceEnabled(any()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index af1e9e8496dc1..2d15123b5e246 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -21,9 +21,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.RejectedExecutionException; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -42,6 +40,7 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; +import org.junit.jupiter.api.Timeout; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.CONDITION_NOT_MET; @@ -52,6 +51,7 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; +import static org.junit.jupiter.api.Assumptions.assumeFalse; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INFINITE_LEASE_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; @@ -65,8 +65,8 @@ * Test lease operations. */ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest { - private static final int TEST_EXECUTION_TIMEOUT = 30 * 1000; - private static final int LONG_TEST_EXECUTION_TIMEOUT = 90 * 1000; + private static final int TEST_EXECUTION_TIMEOUT = 30; + private static final int LONG_TEST_EXECUTION_TIMEOUT = 90; private static final String TEST_FILE = "testfile"; private final boolean isHNSEnabled; private static final int TEST_BYTES = 20; @@ -87,19 +87,21 @@ private AzureBlobFileSystem getCustomFileSystem(Path infiniteLeaseDirs, int numL return getFileSystem(conf); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testNoInfiniteLease() throws IOException { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getFileSystem(); fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { - Assert.assertFalse("Output stream should not have lease", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); + assertFalse(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should not have lease"); } - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testNoLeaseThreads() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 0); @@ -111,22 +113,24 @@ public void testNoLeaseThreads() throws Exception { }); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testOneWriter() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out = fs.create(testFilePath); - Assert.assertTrue("Output stream should have lease", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); + assertTrue(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should have lease"); out.close(); - Assert.assertFalse("Output stream should not have lease", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertFalse(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should not have lease"); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testSubDir() throws Exception { final Path testFilePath = new Path(new Path(path(methodName.getMethodName()), "subdir"), TEST_FILE); @@ -135,15 +139,16 @@ public void testSubDir() throws Exception { fs.mkdirs(testFilePath.getParent().getParent()); FSDataOutputStream out = fs.create(testFilePath); - Assert.assertTrue("Output stream should have lease", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); + assertTrue(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should have lease"); out.close(); - Assert.assertFalse("Output stream should not have lease", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertFalse(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should not have lease"); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testTwoCreate() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); @@ -162,7 +167,7 @@ public void testTwoCreate() throws Exception { return "Expected second create on infinite lease dir to fail"; }); } - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expectException) throws Exception { @@ -198,30 +203,33 @@ private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expec } } - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testTwoWritersCreateAppendNoInfiniteLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getFileSystem(); - Assume.assumeFalse("Parallel Writes Not Allowed on Append Blobs", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Parallel Writes Not Allowed on Append Blobs"); fs.mkdirs(testFilePath.getParent()); twoWriters(fs, testFilePath, false); } - @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(LONG_TEST_EXECUTION_TIMEOUT) public void testTwoWritersCreateAppendWithInfiniteLeaseEnabled() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); - Assume.assumeFalse("Parallel Writes Not Allowed on Append Blobs", isAppendBlobEnabled()); + assumeFalse(isAppendBlobEnabled(), "Parallel Writes Not Allowed on Append Blobs"); fs.mkdirs(testFilePath.getParent()); twoWriters(fs, testFilePath, true); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testLeaseFreedOnClose() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); @@ -230,15 +238,16 @@ public void testLeaseFreedOnClose() throws Exception { FSDataOutputStream out; out = fs.create(testFilePath); out.write(0); - Assert.assertTrue("Output stream should have lease", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); + assertTrue(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should have lease"); out.close(); - Assert.assertFalse("Output stream should not have lease after close", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertFalse(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should not have lease after close"); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testWriteAfterBreakLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); @@ -271,18 +280,19 @@ public void testWriteAfterBreakLease() throws Exception { return "Expected exception on close after lease break but got " + out; }); - Assert.assertTrue("Output stream lease should be freed", - ((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()); + assertTrue(((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed(), + "Output stream lease should be freed"); try (FSDataOutputStream out2 = fs.append(testFilePath)) { out2.write(2); out2.hsync(); } - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } - @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(LONG_TEST_EXECUTION_TIMEOUT) public void testLeaseFreedAfterBreak() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); @@ -299,34 +309,36 @@ public void testLeaseFreedAfterBreak() throws Exception { return "Expected exception on close after lease break but got " + out; }); - Assert.assertTrue("Output stream lease should be freed", - ((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()); + assertTrue(((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed(), + "Output stream lease should be freed"); - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testInfiniteLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { - Assert.assertTrue("Output stream should have lease", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); + assertTrue(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should have lease"); out.write(0); } - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + assertTrue(fs.getAbfsStore().areLeasesFreed()); try (FSDataOutputStream out = fs.append(testFilePath)) { - Assert.assertTrue("Output stream should have lease", - ((AbfsOutputStream) out.getWrappedStream()).hasLease()); + assertTrue(((AbfsOutputStream) out.getWrappedStream()).hasLease(), + "Output stream should have lease"); out.write(1); } - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testFileSystemClose() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); @@ -334,11 +346,10 @@ public void testFileSystemClose() throws Exception { try (FSDataOutputStream out = fs.create(testFilePath)) { out.write(0); - Assert.assertFalse("Store leases should exist", - fs.getAbfsStore().areLeasesFreed()); + assertFalse(fs.getAbfsStore().areLeasesFreed(), "Store leases should exist"); } fs.close(); - Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); + assertTrue(fs.getAbfsStore().areLeasesFreed(), "Store leases were not freed"); Callable exceptionRaisingCallable = () -> { try (FSDataOutputStream out2 = fs.append(testFilePath)) { @@ -365,7 +376,8 @@ public void testFileSystemClose() throws Exception { } } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test + @Timeout(TEST_EXECUTION_TIMEOUT) public void testAcquireRetry() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); @@ -380,11 +392,11 @@ public void testAcquireRetry() throws Exception { AbfsLease lease = new AbfsLease(fs.getAbfsClient(), testFilePath.toUri().getPath(), true, INFINITE_LEASE_DURATION, null, tracingContext); - Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID()); + assertNotNull(lease.getLeaseID(), "Did not successfully lease file"); listener.setOperation(FSOperationType.RELEASE_LEASE); lease.free(); lease.getTracingContext().setListener(null); - Assert.assertEquals("Unexpected acquire retry count", 0, lease.getAcquireRetryCount()); + assertEquals(0, lease.getAcquireRetryCount(), "Unexpected acquire retry count"); AbfsClient mockClient = spy(fs.getAbfsClient()); @@ -395,9 +407,9 @@ public void testAcquireRetry() throws Exception { lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), true, 5, 1, INFINITE_LEASE_DURATION, null, tracingContext); - Assert.assertNotNull("Acquire lease should have retried", lease.getLeaseID()); + assertNotNull(lease.getLeaseID(), "Acquire lease should have retried"); lease.free(); - Assert.assertEquals("Unexpected acquire retry count", 2, lease.getAcquireRetryCount()); + assertEquals(2, lease.getAcquireRetryCount(), "Unexpected acquire retry count"); doThrow(new AbfsLease.LeaseException("failed to acquire")).when(mockClient) .acquireLease(anyString(), anyInt(), any(), any(TracingContext.class)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index d415dd2533471..55322d8d946f4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -32,9 +32,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.assertj.core.api.Assertions; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.mockito.stubbing.Stubber; import org.apache.hadoop.conf.Configuration; @@ -78,11 +76,17 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.rename; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyList; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** @@ -145,21 +149,21 @@ public Void call() throws Exception { */ @Test public void testListPathTracingContext() throws Exception { - final AzureBlobFileSystem spiedFs = Mockito.spy(getFileSystem()); - final AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore()); - final AbfsClient spiedClient = Mockito.spy(spiedFs.getAbfsClient()); - final TracingContext spiedTracingContext = Mockito.spy( + final AzureBlobFileSystem spiedFs = spy(getFileSystem()); + final AzureBlobFileSystemStore spiedStore = spy(spiedFs.getAbfsStore()); + final AbfsClient spiedClient = spy(spiedFs.getAbfsClient()); + final TracingContext spiedTracingContext = spy( new TracingContext( spiedFs.getClientCorrelationId(), spiedFs.getFileSystemId(), FSOperationType.LISTSTATUS, true, TracingHeaderFormat.ALL_ID_FORMAT, null)); - Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); - Mockito.doReturn(spiedClient).when(spiedStore).getClient(); + doReturn(spiedStore).when(spiedFs).getAbfsStore(); + doReturn(spiedClient).when(spiedStore).getClient(); spiedFs.setWorkingDirectory(new Path("/")); AbfsClientTestUtil.setMockAbfsRestOperationForListOperation(spiedClient, (httpOperation) -> { - Stubber stubber = Mockito.doThrow( + Stubber stubber = doThrow( new SocketTimeoutException(CONNECTION_TIMEOUT_JDK_MESSAGE)); stubber.doNothing().when(httpOperation).processResponse( nullable(byte[].class), nullable(int.class), nullable(int.class)); @@ -172,46 +176,46 @@ public void testListPathTracingContext() throws Exception { spiedStore.listStatus(new Path("/"), "", fileStatuses, true, null, spiedTracingContext); // Assert that there were retries due to SocketTimeoutException - Mockito.verify(spiedClient, Mockito.times(1)) + verify(spiedClient, times(1)) .getRetryPolicy(CONNECTION_TIMEOUT_ABBREVIATION); // Assert that there were 2 paginated ListPath calls were made 1 and 2. // 1. Without continuation token - Mockito.verify(spiedClient, times(1)).listPath( + verify(spiedClient, times(1)).listPath( "/", false, spiedFs.getAbfsStore().getAbfsConfiguration().getListMaxResults(), null, spiedTracingContext, spiedFs.getAbfsStore().getUri()); // 2. With continuation token - Mockito.verify(spiedClient, times(1)).listPath( + verify(spiedClient, times(1)).listPath( "/", false, spiedFs.getAbfsStore().getAbfsConfiguration().getListMaxResults(), TEST_CONTINUATION_TOKEN, spiedTracingContext, spiedFs.getAbfsStore().getUri()); // Assert that none of the API calls used the same tracing header. - Mockito.verify(spiedTracingContext, times(0)).constructHeader(any(), any(), any()); + verify(spiedTracingContext, times(0)).constructHeader(any(), any(), any()); } @Test public void testListPathParsingFailure() throws Exception { assumeBlobServiceType(); - AzureBlobFileSystem spiedFs = Mockito.spy(getFileSystem()); - AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore()); - AbfsBlobClient spiedClient = Mockito.spy(spiedStore.getClientHandler() + AzureBlobFileSystem spiedFs = spy(getFileSystem()); + AzureBlobFileSystemStore spiedStore = spy(spiedFs.getAbfsStore()); + AbfsBlobClient spiedClient = spy(spiedStore.getClientHandler() .getBlobClient()); - Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); - Mockito.doReturn(spiedClient).when(spiedStore).getClient(); + doReturn(spiedStore).when(spiedFs).getAbfsStore(); + doReturn(spiedClient).when(spiedStore).getClient(); - Mockito.doThrow(new SocketException(CONNECTION_RESET_MESSAGE)).when(spiedClient).filterRenamePendingFiles(any(), any()); + doThrow(new SocketException(CONNECTION_RESET_MESSAGE)).when(spiedClient).filterRenamePendingFiles(any(), any()); List fileStatuses = new ArrayList<>(); AbfsDriverException ex = intercept(AbfsDriverException.class, () -> { spiedStore.listStatus(new Path("/"), "", fileStatuses, true, null, getTestTracingContext(spiedFs, true)); }); - Assertions.assertThat(ex.getStatusCode()) + assertThat(ex.getStatusCode()) .describedAs("Expecting Network Error status code") .isEqualTo(-1); - Assertions.assertThat(ex.getErrorMessage()) + assertThat(ex.getErrorMessage()) .describedAs("Expecting COPY_ABORTED error code") .contains(ERR_BLOB_LIST_PARSING); } @@ -226,7 +230,7 @@ public void testListFileVsListDir() throws Exception { Path path = path("/testFile"); try(FSDataOutputStream ignored = fs.create(path)) { FileStatus[] testFiles = fs.listStatus(path); - assertEquals("length of test files", 1, testFiles.length); + assertEquals(1, testFiles.length, "length of test files"); FileStatus status = testFiles[0]; assertIsFileReference(status); } @@ -244,18 +248,21 @@ public void testListFileVsListDir2() throws Exception { ContractTestUtils.touch(fs, testFile0Path); FileStatus[] testFiles = fs.listStatus(testFile0Path); - assertEquals("Wrong listing size of file " + testFile0Path, - 1, testFiles.length); + assertEquals(1, testFiles.length, + "Wrong listing size of file " + testFile0Path); FileStatus file0 = testFiles[0]; - assertEquals("Wrong path for " + file0, new Path(getTestUrl(), - testFolder + "/testFolder2/testFolder3/testFile"), file0.getPath()); + assertEquals(new Path(getTestUrl(), + testFolder + "/testFolder2/testFolder3/testFile"), file0.getPath(), + "Wrong path for " + file0); assertIsFileReference(file0); } - @Test(expected = FileNotFoundException.class) + @Test public void testListNonExistentDir() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - fs.listStatus(new Path("/testFile/")); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + fs.listStatus(new Path("/testFile/")); + }); } @Test @@ -293,23 +300,23 @@ public void testListFiles() throws Exception { () -> fs.listFiles(childF, false).next()); // do some final checks on the status (failing due to version checks) - assertEquals("Path mismatch of " + locatedChildStatus, - childF, locatedChildStatus.getPath()); - assertEquals("locatedstatus.equals(status)", - locatedChildStatus, childStatus); - assertEquals("status.equals(locatedstatus)", - childStatus, locatedChildStatus); + assertEquals(childF, locatedChildStatus.getPath(), + "Path mismatch of " + locatedChildStatus); + assertEquals(locatedChildStatus, childStatus, + "locatedstatus.equals(status)"); + assertEquals(childStatus, locatedChildStatus, + "status.equals(locatedstatus)"); } private void assertIsDirectoryReference(FileStatus status) { - assertTrue("Not a directory: " + status, status.isDirectory()); - assertFalse("Not a directory: " + status, status.isFile()); + assertTrue(status.isDirectory(), "Not a directory: " + status); + assertFalse(status.isFile(), "Not a directory: " + status); assertEquals(0, status.getLen()); } private void assertIsFileReference(FileStatus status) { - assertFalse("Not a file: " + status, status.isDirectory()); - assertTrue("Not a file: " + status, status.isFile()); + assertFalse(status.isDirectory(), "Not a file: " + status); + assertTrue(status.isFile(), "Not a file: " + status); } @Test @@ -328,8 +335,8 @@ public void testMkdirTrailingPeriodDirName() throws IOException { catch(IllegalArgumentException e) { exceptionThrown = true; } - assertTrue("Attempt to create file that ended with a dot should" - + " throw IllegalArgumentException", exceptionThrown); + assertTrue(exceptionThrown, "Attempt to create file that ended with a dot should" + + " throw IllegalArgumentException"); } @Test @@ -350,8 +357,8 @@ public void testCreateTrailingPeriodFileName() throws IOException { catch(IllegalArgumentException e) { exceptionThrown = true; } - assertTrue("Attempt to create file that ended with a dot should" - + " throw IllegalArgumentException", exceptionThrown); + assertTrue(exceptionThrown, "Attempt to create file that ended with a dot should" + + " throw IllegalArgumentException"); } @Test @@ -369,8 +376,8 @@ public void testRenameTrailingPeriodFile() throws IOException { catch(IllegalArgumentException e) { exceptionThrown = true; } - assertTrue("Attempt to create file that ended with a dot should" - + " throw IllegalArgumentException", exceptionThrown); + assertTrue(exceptionThrown, "Attempt to create file that ended with a dot should" + + " throw IllegalArgumentException"); } @@ -393,7 +400,7 @@ public void testListStatusWithImplicitExplicitChildren() throws Exception { // Assert that implicit directory is returned FileStatus[] fileStatuses = fs.listStatus(root); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List size is not expected").isEqualTo(1); assertImplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(dir)); @@ -402,7 +409,7 @@ public void testListStatusWithImplicitExplicitChildren() throws Exception { // Assert that only one entry of explicit directory is returned fileStatuses = fs.listStatus(root); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List size is not expected").isEqualTo(1); assertExplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(dir)); @@ -412,7 +419,7 @@ public void testListStatusWithImplicitExplicitChildren() throws Exception { // Assert that two entries are returned in alphabetic order. fileStatuses = fs.listStatus(root); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List size is not expected").isEqualTo(2); assertExplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(dir)); assertFilePathFileStatus(fileStatuses[1], fs.makeQualified(file1)); @@ -423,7 +430,7 @@ public void testListStatusWithImplicitExplicitChildren() throws Exception { // Assert that three entries are returned in alphabetic order. fileStatuses = fs.listStatus(root); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List size is not expected").isEqualTo(3); assertExplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(dir)); assertFilePathFileStatus(fileStatuses[1], fs.makeQualified(file1)); @@ -441,12 +448,12 @@ public void testListStatusOnImplicitDirectoryPath() throws Exception { createAzCopyFolder(implicitPath); FileStatus[] statuses = fs.listStatus(implicitPath); - Assertions.assertThat(statuses.length) + assertThat(statuses.length) .describedAs("List size is not expected").isGreaterThanOrEqualTo(1); assertImplicitDirectoryFileStatus(statuses[0], fs.makeQualified(statuses[0].getPath())); FileStatus[] statuses1 = fs.listStatus(new Path(statuses[0].getPath().toString())); - Assertions.assertThat(statuses1.length) + assertThat(statuses1.length) .describedAs("List size is not expected").isGreaterThanOrEqualTo(1); assertFilePathFileStatus(statuses1[0], fs.makeQualified(statuses1[0].getPath())); } @@ -458,7 +465,7 @@ public void testListStatusOnEmptyDirectory() throws Exception { fs.mkdirs(emptyDir); FileStatus[] statuses = fs.listStatus(emptyDir); - Assertions.assertThat(statuses.length) + assertThat(statuses.length) .describedAs("List size is not expected").isEqualTo(0); } @@ -469,7 +476,7 @@ public void testListStatusOnRenamePendingJsonFile() throws Exception { fs.create(renamePendingJsonPath); FileStatus[] statuses = fs.listStatus(renamePendingJsonPath); - Assertions.assertThat(statuses.length) + assertThat(statuses.length) .describedAs("List size is not expected").isEqualTo(1); assertFilePathFileStatus(statuses[0], fs.makeQualified(statuses[0].getPath())); } @@ -488,18 +495,18 @@ public void testContinuationTokenAcrossListStatus() throws Exception { "/testContinuationToken", false, 1, null, getTestTracingContext(fs, true), fs.getAbfsStore().getUri()); - Assertions.assertThat(listResponseData.getContinuationToken()) + assertThat(listResponseData.getContinuationToken()) .describedAs("Continuation Token Should not be null").isNotNull(); - Assertions.assertThat(listResponseData.getFileStatusList()) + assertThat(listResponseData.getFileStatusList()) .describedAs("Listing Size Not as expected").hasSize(1); ListResponseData listResponseData1 = fs.getAbfsStore().getClient().listPath( "/testContinuationToken", false, 1, listResponseData.getContinuationToken(), getTestTracingContext(fs, true), fs.getAbfsStore().getUri()); - Assertions.assertThat(listResponseData1.getContinuationToken()) + assertThat(listResponseData1.getContinuationToken()) .describedAs("Continuation Token Should be null").isNull(); - Assertions.assertThat(listResponseData1.getFileStatusList()) + assertThat(listResponseData1.getFileStatusList()) .describedAs("Listing Size Not as expected").hasSize(1); } @@ -530,9 +537,9 @@ public void testEmptyContinuationToken() throws Exception { "/testInvalidContinuationToken", false, 1, "", getTestTracingContext(fs, true), fs.getAbfsStore().getUri()); - Assertions.assertThat(listResponseData.getContinuationToken()) + assertThat(listResponseData.getContinuationToken()) .describedAs("Continuation Token Should Not be null").isNotNull(); - Assertions.assertThat(listResponseData.getFileStatusList()) + assertThat(listResponseData.getFileStatusList()) .describedAs("Listing Size Not as expected").hasSize(1); } @@ -545,13 +552,13 @@ public void testEmptyContinuationToken() throws Exception { */ @Test public void testDuplicateEntriesAcrossListBlobIterations() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AzureBlobFileSystem fs = spy(getFileSystem()); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); store.getAbfsConfiguration().setListMaxResults(1); - AbfsClient client = Mockito.spy(store.getClient()); + AbfsClient client = spy(store.getClient()); - Mockito.doReturn(store).when(fs).getAbfsStore(); - Mockito.doReturn(client).when(store).getClient(); + doReturn(store).when(fs).getAbfsStore(); + doReturn(client).when(store).getClient(); /* * Following entries will be created inside the root path. @@ -592,11 +599,11 @@ public void testDuplicateEntriesAcrossListBlobIterations() throws Exception { // Assert that client.listPath was called 11 times. // This will assert server returned 11 entries in total. - Mockito.verify(client, Mockito.times(TOTAL_NUMBER_OF_PATHS)) + verify(client, times(TOTAL_NUMBER_OF_PATHS)) .listPath(eq(ROOT_PATH), eq(false), eq(1), any(), any(), any()); // Assert that after duplicate removal, only 7 unique entries are returned. - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List size is not expected").isEqualTo(NUMBER_OF_UNIQUE_PATHS); // Assert that for duplicates, entry corresponding to marker blob is returned. @@ -611,7 +618,7 @@ public void testDuplicateEntriesAcrossListBlobIterations() throws Exception { // Assert that there are no duplicates in the returned file statuses. Set uniquePaths = new HashSet<>(); for (FileStatus fileStatus : fileStatuses) { - Assertions.assertThat(uniquePaths.add(fileStatus.getPath())) + assertThat(uniquePaths.add(fileStatus.getPath())) .describedAs("Duplicate Entries found") .isTrue(); } @@ -619,13 +626,13 @@ public void testDuplicateEntriesAcrossListBlobIterations() throws Exception { private void assertFilePathFileStatus(final FileStatus fileStatus, final Path qualifiedPath) { - Assertions.assertThat(fileStatus.getPath()) + assertThat(fileStatus.getPath()) .describedAs("Path Not as expected").isEqualTo(qualifiedPath); - Assertions.assertThat(fileStatus.isFile()) + assertThat(fileStatus.isFile()) .describedAs("Expecting a File Path").isEqualTo(true); - Assertions.assertThat(fileStatus.isDirectory()) + assertThat(fileStatus.isDirectory()) .describedAs("Expecting a File Path").isEqualTo(false); - Assertions.assertThat(fileStatus.getModificationTime()).isNotEqualTo(0); + assertThat(fileStatus.getModificationTime()).isNotEqualTo(0); } private void assertImplicitDirectoryFileStatus(final FileStatus fileStatus, @@ -633,7 +640,7 @@ private void assertImplicitDirectoryFileStatus(final FileStatus fileStatus, assertDirectoryFileStatus(fileStatus, qualifiedPath); DirectoryStateHelper.isImplicitDirectory(qualifiedPath, getFileSystem(), getTestTracingContext(getFileSystem(), true)); - Assertions.assertThat(fileStatus.getModificationTime()) + assertThat(fileStatus.getModificationTime()) .describedAs("Last Modified Time Not as Expected").isEqualTo(0); } @@ -642,19 +649,19 @@ private void assertExplicitDirectoryFileStatus(final FileStatus fileStatus, assertDirectoryFileStatus(fileStatus, qualifiedPath); DirectoryStateHelper.isExplicitDirectory(qualifiedPath, getFileSystem(), getTestTracingContext(getFileSystem(), true)); - Assertions.assertThat(fileStatus.getModificationTime()) + assertThat(fileStatus.getModificationTime()) .describedAs("Last Modified Time Not as Expected").isNotEqualTo(0); } private void assertDirectoryFileStatus(final FileStatus fileStatus, final Path qualifiedPath) { - Assertions.assertThat(fileStatus.getPath()) + assertThat(fileStatus.getPath()) .describedAs("Path Not as Expected").isEqualTo(qualifiedPath); - Assertions.assertThat(fileStatus.isDirectory()) + assertThat(fileStatus.isDirectory()) .describedAs("Expecting a Directory Path").isEqualTo(true); - Assertions.assertThat(fileStatus.isFile()) + assertThat(fileStatus.isFile()) .describedAs("Expecting a Directory Path").isEqualTo(false); - Assertions.assertThat(fileStatus.getLen()) + assertThat(fileStatus.getLen()) .describedAs("Content Length Not as Expected").isEqualTo(0); } @@ -665,7 +672,7 @@ private void assertDirectoryFileStatus(final FileStatus fileStatus, * @param newHeader the header to add in place of the old one */ public static void mockAbfsRestOperation(AbfsBlobClient abfsBlobClient, String... newHeader) { - Mockito.doAnswer(invocation -> { + doAnswer(invocation -> { List requestHeaders = invocation.getArgument(3); // Remove the actual HDI config header and add the new one @@ -688,13 +695,13 @@ public static void mockAbfsRestOperation(AbfsBlobClient abfsBlobClient, String.. * @return the mocked AbfsBlobClient */ public static AbfsBlobClient mockIngressClientHandler(AzureBlobFileSystem fs) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler()); - AbfsBlobClient abfsBlobClient = (AbfsBlobClient) Mockito.spy( + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + AbfsClientHandler clientHandler = spy(store.getClientHandler()); + AbfsBlobClient abfsBlobClient = (AbfsBlobClient) spy( clientHandler.getClient()); fs.getAbfsStore().setClient(abfsBlobClient); fs.getAbfsStore().setClientHandler(clientHandler); - Mockito.doReturn(abfsBlobClient).when(clientHandler).getIngressClient(); + doReturn(abfsBlobClient).when(clientHandler).getIngressClient(); return abfsBlobClient; } @@ -703,7 +710,7 @@ public static AbfsBlobClient mockIngressClientHandler(AzureBlobFileSystem fs) { * verifying the correct header and directory state. */ private void testIsDirectory(boolean expected, String... configName) throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(getFileSystem())) { + try (AzureBlobFileSystem fs = spy(getFileSystem())) { assumeBlobServiceType(); AbfsBlobClient abfsBlobClient = mockIngressClientHandler(fs); // Mock the operation to modify the headers @@ -721,17 +728,17 @@ private void testIsDirectory(boolean expected, String... configName) throws Exce true, getTestTracingContext(fs, true), null).getResult(); - Assertions.assertThat(abfsBlobClient.checkIsDir(op)) + assertThat(abfsBlobClient.checkIsDir(op)) .describedAs("Directory should be marked as " + expected) .isEqualTo(expected); // Verify the header and directory state - Assertions.assertThat(fileStatus.length) + assertThat(fileStatus.length) .describedAs("Expected directory state: " + expected) .isEqualTo(1); // Verify the header and directory state - Assertions.assertThat(fileStatus[0].isDirectory()) + assertThat(fileStatus[0].isDirectory()) .describedAs("Expected directory state: " + expected) .isEqualTo(expected); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java index e54b98e0b7a6e..53323c16ca542 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java @@ -21,7 +21,7 @@ import java.util.UUID; import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.apache.hadoop.conf.Configuration; @@ -71,8 +71,9 @@ public void testMkdirExistingDirOverwriteFalse() throws Exception { assertMkdirs(fs, path); //checks that mkdirs returns true long timeCreated = fs.getFileStatus(path).getModificationTime(); assertMkdirs(fs, path); //call to existing dir should return success - assertEquals("LMT should not be updated for existing dir", timeCreated, - fs.getFileStatus(path).getModificationTime()); + assertEquals(timeCreated, + fs.getFileStatus(path).getModificationTime(), + "LMT should not be updated for existing dir"); } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index f27e75839b73f..5a95574e3ea00 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -22,8 +22,7 @@ import java.io.InputStream; import java.util.Map; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +46,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; +import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Test Azure Oauth with Blob Data contributor role and Blob Data Reader role. @@ -62,7 +62,7 @@ public class ITestAzureBlobFileSystemOauth extends AbstractAbfsIntegrationTest{ LoggerFactory.getLogger(ITestAbfsStreamStatistics.class); public ITestAzureBlobFileSystemOauth() throws Exception { - Assume.assumeTrue(this.getAuthType() == AuthType.OAuth); + assumeTrue(this.getAuthType() == AuthType.OAuth); } /* * BLOB DATA CONTRIBUTOR should have full access to the container and blobs in the container. @@ -70,9 +70,9 @@ public ITestAzureBlobFileSystemOauth() throws Exception { @Test public void testBlobDataContributor() throws Exception { String clientId = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID); - Assume.assumeTrue("Contributor client id not provided", clientId != null); + assumeTrue(clientId != null, "Contributor client id not provided"); String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET); - Assume.assumeTrue("Contributor client secret not provided", secret != null); + assumeTrue(secret != null, "Contributor client secret not provided"); Path existedFilePath = path(EXISTED_FILE_PATH); Path existedFolderPath = path(EXISTED_FOLDER_PATH); @@ -125,9 +125,9 @@ public void testBlobDataContributor() throws Exception { @Test public void testBlobDataReader() throws Exception { String clientId = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_ID); - Assume.assumeTrue("Reader client id not provided", clientId != null); + assumeTrue(clientId != null, "Reader client id not provided"); String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET); - Assume.assumeTrue("Reader client secret not provided", secret != null); + assumeTrue(secret != null, "Reader client secret not provided"); Path existedFilePath = path(EXISTED_FILE_PATH); Path existedFolderPath = path(EXISTED_FOLDER_PATH); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java index 00f5af02622a6..374a0aab57c99 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java @@ -18,28 +18,26 @@ package org.apache.hadoop.fs.azurebfs; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + import java.util.ArrayList; import java.util.Collection; import java.util.UUID; import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.fs.azurebfs.utils.Parallelized; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; /** * Test permission operations. */ -@RunWith(Parallelized.class) public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationTest{ private static Path testRoot = new Path("/test"); @@ -50,12 +48,15 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT private Path path; - public ITestAzureBlobFileSystemPermission(FsPermission testPermission) throws Exception { + protected ITestAzureBlobFileSystemPermission() throws Exception { super(); - permission = testPermission; } - @Parameterized.Parameters(name = "{0}") + public void initITestAzureBlobFileSystemPermission( + FsPermission pTestPermission) throws Exception { + permission = pTestPermission; + } + public static Collection abfsCreateNonRecursiveTestData() throws Exception { /* @@ -73,12 +74,13 @@ public static Collection abfsCreateNonRecursiveTestData() return datas; } - @Test - public void testFilePermission() throws Exception { - + @ParameterizedTest(name = "Permission : {0}") + @MethodSource("abfsCreateNonRecursiveTestData") + public void testFilePermission(FsPermission pTestPermission) throws Exception { + initITestAzureBlobFileSystemPermission(pTestPermission); final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(getIsNamespaceEnabled(fs)); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); + assumeTrue(getIsNamespaceEnabled(fs)); + assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE); path = new Path(testRoot, UUID.randomUUID().toString()); @@ -89,14 +91,14 @@ public void testFilePermission() throws Exception { fs.create(path, permission, true, KILOBYTE, (short) 1, KILOBYTE - 1, null).close(); FileStatus status = fs.getFileStatus(path); - Assert.assertEquals(permission.applyUMask(DEFAULT_UMASK_PERMISSION), status.getPermission()); + assertEquals(permission.applyUMask(DEFAULT_UMASK_PERMISSION), status.getPermission()); } @Test public void testFolderPermission() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(getIsNamespaceEnabled(fs)); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); + assumeTrue(getIsNamespaceEnabled(fs)); + assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027"); path = new Path(testRoot, UUID.randomUUID().toString()); @@ -107,6 +109,6 @@ public void testFolderPermission() throws Exception { fs.mkdirs(path, permission); FileStatus status = fs.getFileStatus(path); - Assert.assertEquals(permission.applyUMask(DEFAULT_UMASK_PERMISSION), status.getPermission()); + assertEquals(permission.applyUMask(DEFAULT_UMASK_PERMISSION), status.getPermission()); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java index 9c762866ca299..d2730c259e6de 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java @@ -17,15 +17,16 @@ */ package org.apache.hadoop.fs.azurebfs; +import static org.junit.jupiter.api.Assumptions.assumeFalse; + import java.io.EOFException; import java.io.IOException; import java.util.Random; import java.util.concurrent.Callable; import java.util.UUID; -import org.junit.Assume; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,7 +96,7 @@ public void testBasicRead() throws Exception { // forward seek and read a kilobyte into first kilobyte of bufferV2 inputStream.seek(5 * MEGABYTE); int numBytesRead = inputStream.read(buffer, 0, KILOBYTE); - assertEquals("Wrong number of bytes read", KILOBYTE, numBytesRead); + assertEquals(KILOBYTE, numBytesRead, "Wrong number of bytes read"); int len = MEGABYTE; int offset = buffer.length - len; @@ -103,7 +104,7 @@ public void testBasicRead() throws Exception { // reverse seek and read a megabyte into last megabyte of bufferV1 inputStream.seek(3 * MEGABYTE); numBytesRead = inputStream.read(buffer, offset, len); - assertEquals("Wrong number of bytes read after seek", len, numBytesRead); + assertEquals(len, numBytesRead, "Wrong number of bytes read after seek"); } } @@ -113,9 +114,9 @@ public void testBasicRead() throws Exception { */ @Test public void testRandomRead() throws Exception { - Assume.assumeFalse("This test does not support namespace enabled account", - getIsNamespaceEnabled(getFileSystem())); - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(getIsNamespaceEnabled(getFileSystem()), + "This test does not support namespace enabled account"); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); Path testPath = path(TEST_FILE_PREFIX + "_testRandomRead"); assumeHugeFileExists(testPath); @@ -216,11 +217,9 @@ public Long call() throws Exception { } ); long elapsedTimeMs = timer.elapsedTimeMs(); - assertTrue( - String.format( - "There should not be any network I/O (elapsedTimeMs=%1$d).", - elapsedTimeMs), - elapsedTimeMs < MAX_ELAPSEDTIMEMS); + assertTrue(elapsedTimeMs < MAX_ELAPSEDTIMEMS, String.format( + "There should not be any network I/O (elapsedTimeMs=%1$d).", + elapsedTimeMs)); } } @@ -251,7 +250,7 @@ public FSDataInputStream call() throws Exception { } ); - assertTrue("Test file length only " + testFileLength, testFileLength > 0); + assertTrue(testFileLength > 0, "Test file length only " + testFileLength); inputStream.seek(testFileLength); assertEquals(testFileLength, inputStream.getPos()); @@ -267,11 +266,9 @@ public FSDataInputStream call() throws Exception { ); long elapsedTimeMs = timer.elapsedTimeMs(); - assertTrue( - String.format( - "There should not be any network I/O (elapsedTimeMs=%1$d).", - elapsedTimeMs), - elapsedTimeMs < MAX_ELAPSEDTIMEMS); + assertTrue(elapsedTimeMs < MAX_ELAPSEDTIMEMS, String.format( + "There should not be any network I/O (elapsedTimeMs=%1$d).", + elapsedTimeMs)); } } @@ -433,21 +430,20 @@ public void testSequentialReadAfterReverseSeekPerformance() (long) afterSeekElapsedMs, ratio))); } - assertTrue(String.format( - "Performance of ABFS stream after reverse seek is not acceptable:" - + " beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d," - + " ratio=%3$.2f", - (long) beforeSeekElapsedMs, - (long) afterSeekElapsedMs, - ratio), - ratio < maxAcceptableRatio); + assertTrue(ratio < maxAcceptableRatio, String.format( + "Performance of ABFS stream after reverse seek is not acceptable:" + + " beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d," + + " ratio=%3$.2f", + (long) beforeSeekElapsedMs, + (long) afterSeekElapsedMs, + ratio)); } @Test - @Ignore("HADOOP-16915") + @Disabled("HADOOP-16915") public void testRandomReadPerformance() throws Exception { - Assume.assumeFalse("This test does not support namespace enabled account", - getIsNamespaceEnabled(getFileSystem())); + assumeFalse(getIsNamespaceEnabled(getFileSystem()), + "This test does not support namespace enabled account"); Path testPath = path(TEST_FILE_PREFIX + "_testRandomReadPerformance"); assumeHugeFileExists(testPath); @@ -470,13 +466,12 @@ public void testRandomReadPerformance() throws Exception { (long) v2ElapsedMs, ratio)); } - assertTrue(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); + 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)); } /** @@ -716,7 +711,8 @@ private long assumeHugeFileExists(Path testPath) throws Exception{ ContractTestUtils.assertPathExists(this.getFileSystem(), "huge file not created", testPath); FileStatus status = fs.getFileStatus(testPath); ContractTestUtils.assertIsFile(testPath, status); - assertTrue("File " + testPath + " is not of expected size " + fileSize + ":actual=" + status.getLen(), status.getLen() == fileSize); + assertTrue(status.getLen() == fileSize, + "File " + testPath + " is not of expected size " + fileSize + ":actual=" + status.getLen()); return fileSize; } @@ -726,12 +722,12 @@ private void verifyConsistentReads(FSDataInputStream inputStreamV1, byte[] bufferV2) throws IOException { int size = bufferV1.length; final int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, size); - assertEquals("Bytes read from wasb stream", size, numBytesReadV1); + assertEquals(size, numBytesReadV1, "Bytes read from wasb stream"); final int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, size); - assertEquals("Bytes read from abfs stream", size, numBytesReadV2); + assertEquals(size, numBytesReadV2, "Bytes read from abfs stream"); - assertArrayEquals("Mismatch in read data", bufferV1, bufferV2); + assertArrayEquals(bufferV1, bufferV2, "Mismatch in read data"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index 408f94d78d88c..cb6ec97d4a6ce 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -32,10 +32,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.invocation.InvocationOnMock; -import org.mockito.Mockito; import org.mockito.stubbing.Answer; import org.apache.hadoop.conf.Configuration; @@ -108,6 +106,19 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; /** * Test rename operation. @@ -160,10 +171,10 @@ public void testRenameFileUnderDir() throws Exception { Path destDir = path("/testDst"); assertRenameOutcome(fs, sourceDir, destDir, true); FileStatus[] fileStatus = fs.listStatus(destDir); - assertNotNull("Null file status", fileStatus); + assertNotNull(fileStatus, "Null file status"); FileStatus status = fileStatus[0]; - assertEquals("Wrong filename in " + status, - filename, status.getPath().getName()); + assertEquals(filename, status.getPath().getName(), + "Wrong filename in " + status); } @Test @@ -210,7 +221,7 @@ public Void call() throws Exception { assertRenameOutcome(fs, source, dest, true); FileStatus[] files = fs.listStatus(dest); - assertEquals("Wrong number of files in listing", 1000, files.length); + assertEquals(1000, files.length, "Wrong number of files in listing"); assertPathDoesNotExist(fs, "rename source dir", source); } @@ -260,8 +271,8 @@ public void testRenameWithNoDestinationParentDir() throws Exception { // Verify that renaming on a destination with no parent dir wasn't // successful. - assertFalse("Rename result expected to be false with no Parent dir", - fs.rename(sourcePath, destPath)); + assertFalse(fs.rename(sourcePath, destPath), + "Rename result expected to be false with no Parent dir"); // Verify that metadata was in an incomplete state after the rename // failure, and we retired the rename once more. @@ -321,7 +332,7 @@ public void testRenameNotFoundBlobToEmptyRoot() throws Exception { public void testRenameBlobToDstWithColonInSourcePath() throws Exception { AzureBlobFileSystem fs = getFileSystem(); fs.create(new Path("/src:/file")); - Assertions.assertThat( + assertThat( fs.rename(new Path("/src:"), new Path("/dst"))) .describedAs("Rename should succeed") .isTrue(); @@ -340,7 +351,7 @@ public void testRenameBlobToDstWithColonInSourcePath() throws Exception { public void testRenameWithColonInDestinationPath() throws Exception { AzureBlobFileSystem fs = getFileSystem(); fs.create(new Path("/src")); - Assertions.assertThat( + assertThat( fs.rename(new Path("/src"), new Path("/dst:"))) .describedAs("Rename should succeed") .isTrue(); @@ -355,28 +366,28 @@ public void testRenameWithColonInSourcePath() throws Exception { fs.create(new Path(sourceDirectory, fileName)); fs.create(new Path(sourceDirectory + "/Test:", fileName)); // Rename from source to destination - Assertions.assertThat( + assertThat( fs.rename(new Path(sourceDirectory), new Path(destinationDirectory))) .describedAs("Rename should succeed") .isTrue(); - Assertions.assertThat( + assertThat( fs.exists(new Path(sourceDirectory, fileName))) .describedAs("Source directory should not exist after rename") .isFalse(); - Assertions.assertThat( + assertThat( fs.exists(new Path(destinationDirectory, fileName))) .describedAs("Destination directory should exist after rename") .isTrue(); // Rename from destination to source - Assertions.assertThat( + assertThat( fs.rename(new Path(destinationDirectory), new Path(sourceDirectory))) .describedAs("Rename should succeed").isTrue(); - Assertions.assertThat( + assertThat( fs.exists(new Path(sourceDirectory, fileName))) .describedAs("Destination directory should exist after rename") .isTrue(); - Assertions.assertThat( + assertThat( fs.exists(new Path(destinationDirectory, fileName))) .describedAs("Source directory should not exist after rename") .isFalse(); @@ -489,7 +500,7 @@ public void testPosixRenameDirectoryWherePartAlreadyThereOnDestination() @Test public void testRenamePendingJsonIsRemovedPostSuccessfulRename() throws Exception { - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.setWorkingDirectory(new Path("/")); @@ -500,7 +511,7 @@ public void testRenamePendingJsonIsRemovedPostSuccessfulRename() fs.create(new Path("hbase/test4/file1")); final Integer[] correctDeletePathCount = new Integer[1]; correctDeletePathCount[0] = 0; - Mockito.doAnswer(answer -> { + doAnswer(answer -> { final String correctDeletePath = "/hbase/test1/test2/test3" + SUFFIX; if (correctDeletePath.equals( ((Path) answer.getArgument(0)).toUri().getPath())) { @@ -509,13 +520,12 @@ public void testRenamePendingJsonIsRemovedPostSuccessfulRename() return null; }) .when(client) - .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), nullable(String.class), + any(TracingContext.class)); assertTrue(fs.rename(new Path("hbase/test1/test2/test3"), new Path("hbase/test4"))); - assertEquals("RenamePendingJson should be deleted", - 1, - (int) correctDeletePathCount[0]); + assertEquals(1, (int) correctDeletePathCount[0], + "RenamePendingJson should be deleted"); } /** @@ -527,7 +537,7 @@ public void testRenamePendingJsonIsRemovedPostSuccessfulRename() @Test public void testHBaseHandlingForFailedRenameWithListRecovery() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); String srcPath = "hbase/test1/test2"; @@ -551,7 +561,7 @@ public void testHBaseHandlingForFailedRenameWithListRecovery() @Test public void testHBaseHandlingForFailedRenameWithGetFileStatusRecovery() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); String srcPath = "hbase/test1/test2"; @@ -575,7 +585,7 @@ public void testHBaseHandlingForFailedRenameWithGetFileStatusRecovery() @Test public void testHbaseListStatusBeforeRenamePendingFileAppendedWithIngressOnBlob() throws Exception { - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); fs.setWorkingDirectory(new Path(ROOT_PATH)); testRenamePreRenameFailureResolution(fs); @@ -589,7 +599,7 @@ public void testHbaseListStatusBeforeRenamePendingFileAppendedWithIngressOnBlob( @Test public void testRenameJsonDeletedBeforeRenameAtomicityCanDelete() throws Exception { - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.setWorkingDirectory(new Path(ROOT_PATH)); @@ -600,7 +610,7 @@ public void testRenameJsonDeletedBeforeRenameAtomicityCanDelete() os.write("{}".getBytes(StandardCharsets.UTF_8)); os.close(); int[] renameJsonDeleteCounter = new int[1]; - Mockito.doAnswer(deleteAnswer -> { + doAnswer(deleteAnswer -> { Path ansPath = deleteAnswer.getArgument(0); if (renameJson.toUri() .getPath() @@ -611,8 +621,8 @@ public void testRenameJsonDeletedBeforeRenameAtomicityCanDelete() return deleteAnswer.callRealMethod(); }) .when(client) - .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), nullable(String.class), + any(TracingContext.class)); new RenameAtomicity(renameJson, 2, getTestTracingContext(fs, true), null, client); } @@ -635,7 +645,7 @@ public void testRenameJsonDeletedBeforeRenameAtomicityCanDelete() */ @Test public void testRenameCompleteBeforeRenameAtomicityRedo() throws Exception { - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.setWorkingDirectory(new Path(ROOT_PATH)); @@ -651,7 +661,7 @@ public void testRenameCompleteBeforeRenameAtomicityRedo() throws Exception { new Path("/hbase/test4"), renameJson, getTestTracingContext(fs, true), fileStatus.getEtag(), client).preRename(); - RenameAtomicity redoRenameAtomicity = Mockito.spy( + RenameAtomicity redoRenameAtomicity = spy( new RenameAtomicity(renameJson, jsonLen, getTestTracingContext(fs, true), null, client)); RenameAtomicityTestUtils.addReadPathMock(redoRenameAtomicity, @@ -662,10 +672,10 @@ public void testRenameCompleteBeforeRenameAtomicityRedo() throws Exception { }); AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, redoRenameAtomicity::redo); - Assertions.assertThat(ex.getStatusCode()) + assertThat(ex.getStatusCode()) .describedAs("RenameAtomicity redo should fail with 404") .isEqualTo(SOURCE_PATH_NOT_FOUND.getStatusCode()); - Assertions.assertThat(ex.getErrorCode()) + assertThat(ex.getErrorCode()) .describedAs("RenameAtomicity redo should fail with 404") .isEqualTo(SOURCE_PATH_NOT_FOUND); } @@ -686,14 +696,14 @@ public void testRenameCompleteBeforeRenameAtomicityRedo() throws Exception { */ @Test public void testCopyBlobIdempotency() throws Exception { - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.setWorkingDirectory(new Path(ROOT_PATH)); Path src = new Path("/srcDir/src"); Path dst = new Path("/dst"); fs.create(src); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { Path srcCopy = answer.getArgument(0); Path dstCopy = answer.getArgument(1); String leaseId = answer.getArgument(2); @@ -705,17 +715,17 @@ public void testCopyBlobIdempotency() throws Exception { ((AbfsBlobClient) getFileSystem().getAbfsClient()).copyBlob(srcCopy, dstCopy, leaseId, tracingContext); return answer.callRealMethod(); - }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); - Assertions.assertThat(fs.rename(src, dst)) + }).when(client).copyBlob(any(Path.class), any(Path.class), + nullable(String.class), + any(TracingContext.class)); + assertThat(fs.rename(src, dst)) .describedAs("Rename should be successful and copyBlob should" + "be able to handle idempotency issue") .isTrue(); - Assertions.assertThat(fs.exists(src)) + assertThat(fs.exists(src)) .describedAs("Source should not exist after rename") .isFalse(); - Assertions.assertThat(fs.exists(dst)) + assertThat(fs.exists(dst)) .describedAs("Destination should exist after rename") .isTrue(); } @@ -737,25 +747,25 @@ public void testCopyBlobIdempotency() throws Exception { @Test public void testRenameBlobIdempotencyWhereDstIsCreatedFromSomeOtherProcess() throws IOException { - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.setWorkingDirectory(new Path(ROOT_PATH)); Path src = new Path("/src"); Path dst = new Path("/dst"); fs.create(src); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { Path dstCopy = answer.getArgument(1); fs.create(dstCopy); return answer.callRealMethod(); - }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); - Assertions.assertThat(fs.rename(src, dst)) + }).when(client).copyBlob(any(Path.class), any(Path.class), + nullable(String.class), + any(TracingContext.class)); + assertThat(fs.rename(src, dst)) .describedAs("Rename should be successful and copyBlob should" + "be able to handle idempotency issue") .isFalse(); - Assertions.assertThat(fs.exists(src)) + assertThat(fs.exists(src)) .describedAs("Source should exist after rename failure") .isTrue(); } @@ -766,7 +776,7 @@ public void testRenameBlobIdempotencyWhereDstIsCreatedFromSomeOtherProcess() * destination directory, and renaming the source directory to the destination. * It then verifies that the renamed directory exists at the expected destination path. * - * @throws Exception If an error occurs during the file system operations or assertions. + * @throws Exception If an error occurs during the file system operations or */ @Test public void testRenameDirWhenMarkerBlobIsAbsentOnDstDir() throws Exception { @@ -790,7 +800,7 @@ public void testRenameDirWhenMarkerBlobIsAbsentOnDstDir() throws Exception { * creates a new target directory, and renames the source directory to the target location. * It verifies that the renamed source directory exists in the target path. * - * @throws Exception If an error occurs during the file system operations or assertions. + * @throws Exception If an error occurs during the file system operations or */ @Test public void testBlobRenameSrcDirHasNoMarker() throws Exception { @@ -820,7 +830,7 @@ public void testBlobRenameSrcDirHasNoMarker() throws Exception { */ @Test public void testCopyBlobTakeTime() throws Exception { - AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fileSystem = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient( fileSystem); @@ -834,9 +844,9 @@ public void testCopyBlobTakeTime() throws Exception { }); fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); assertTrue(fileSystem.exists(new Path("/test1/file2"))); - Mockito.verify(blobRenameHandlers[0], Mockito.times(1)) - .handleCopyInProgress(Mockito.any(Path.class), - Mockito.any(TracingContext.class), Mockito.any(String.class)); + verify(blobRenameHandlers[0], times(1)) + .handleCopyInProgress(any(Path.class), + any(TracingContext.class), any(String.class)); } /** @@ -853,7 +863,7 @@ public void testCopyBlobTakeTime() throws Exception { */ @Test public void testCopyBlobTakeTimeAndEventuallyFail() throws Exception { - AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fileSystem = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient( fileSystem); @@ -864,10 +874,10 @@ public void testCopyBlobTakeTimeAndEventuallyFail() throws Exception { () -> { fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); }); - Assertions.assertThat(ex.getStatusCode()) + assertThat(ex.getStatusCode()) .describedAs("Expecting COPY_FAILED status code") .isEqualTo(COPY_BLOB_FAILED.getStatusCode()); - Assertions.assertThat(ex.getErrorCode()) + assertThat(ex.getErrorCode()) .describedAs("Expecting COPY_FAILED error code") .isEqualTo(COPY_BLOB_FAILED); } @@ -886,7 +896,7 @@ public void testCopyBlobTakeTimeAndEventuallyFail() throws Exception { */ @Test public void testCopyBlobTakeTimeAndEventuallyAborted() throws Exception { - AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fileSystem = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient( fileSystem); @@ -897,10 +907,10 @@ public void testCopyBlobTakeTimeAndEventuallyAborted() throws Exception { () -> { fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); }); - Assertions.assertThat(ex.getStatusCode()) + assertThat(ex.getStatusCode()) .describedAs("Expecting COPY_ABORTED status code") .isEqualTo(COPY_BLOB_ABORTED.getStatusCode()); - Assertions.assertThat(ex.getErrorCode()) + assertThat(ex.getErrorCode()) .describedAs("Expecting COPY_ABORTED error code") .isEqualTo(COPY_BLOB_ABORTED); } @@ -919,25 +929,25 @@ public void testCopyBlobTakeTimeAndEventuallyAborted() throws Exception { */ @Test public void testCopyBlobTakeTimeAndBlobIsDeleted() throws Exception { - AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fileSystem = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient( fileSystem); String srcFile = "/test1/file"; String dstFile = "/test1/file2"; - Mockito.doAnswer(answer -> { - AbfsRestOperation op = Mockito.spy( + doAnswer(answer -> { + AbfsRestOperation op = spy( (AbfsRestOperation) answer.callRealMethod()); fileSystem.delete(new Path(dstFile), false); - AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); - Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + AbfsHttpOperation httpOp = spy(op.getResult()); + doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( HttpHeaderConfigurations.X_MS_COPY_STATUS); - Mockito.doReturn(httpOp).when(op).getResult(); + doReturn(httpOp).when(op).getResult(); return op; }) .when(spiedClient) - .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + .copyBlob(any(Path.class), any(Path.class), + nullable(String.class), any(TracingContext.class)); fileSystem.create(new Path(srcFile)); assertFalse(fileSystem.rename(new Path(srcFile), new Path(dstFile))); assertFalse(fileSystem.exists(new Path(dstFile))); @@ -972,7 +982,7 @@ public void testCopyAfterSourceHasBeenDeleted() throws Exception { client.copyBlob(new Path("/src"), new Path("/dst"), null, getTestTracingContext(fs, true)); }); - Assertions.assertThat(ex.getStatusCode()) + assertThat(ex.getStatusCode()) .describedAs("Source has to be not found at copy") .isEqualTo(HTTP_NOT_FOUND); } @@ -993,7 +1003,7 @@ public void testCopyAfterSourceHasBeenDeleted() throws Exception { public void testParallelRenameForAtomicRenameShouldFail() throws Exception { Configuration config = getRawConfiguration(); config.set(FS_AZURE_LEASE_THREADS, "2"); - AzureBlobFileSystem fs = Mockito.spy( + AzureBlobFileSystem fs = spy( (AzureBlobFileSystem) FileSystem.newInstance(config)); assumeBlobServiceType(); fs.setWorkingDirectory(new Path(ROOT_PATH)); @@ -1004,16 +1014,16 @@ public void testParallelRenameForAtomicRenameShouldFail() throws Exception { AtomicBoolean leaseAcquired = new AtomicBoolean(false); AtomicBoolean exceptionOnParallelRename = new AtomicBoolean(false); AtomicBoolean parallelThreadDone = new AtomicBoolean(false); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); leaseAcquired.set(true); while (!parallelThreadDone.get()) {} return op; }) .when(client) - .acquireLease(Mockito.anyString(), Mockito.anyInt(), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .acquireLease(anyString(), anyInt(), + nullable(String.class), + any(TracingContext.class)); new Thread(() -> { while (!leaseAcquired.get()) {} try { @@ -1031,7 +1041,7 @@ public void testParallelRenameForAtomicRenameShouldFail() throws Exception { }).start(); fs.rename(src, dst); while (!parallelThreadDone.get()) {} - Assertions.assertThat(exceptionOnParallelRename.get()) + assertThat(exceptionOnParallelRename.get()) .describedAs("Parallel rename should fail") .isTrue(); } @@ -1049,7 +1059,7 @@ public void testParallelRenameForAtomicRenameShouldFail() throws Exception { */ @Test public void testAppendAtomicBlobDuringRename() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); Path src = new Path("/hbase/src"); @@ -1058,12 +1068,12 @@ public void testAppendAtomicBlobDuringRename() throws Exception { AtomicBoolean copyInProgress = new AtomicBoolean(false); AtomicBoolean outputStreamClosed = new AtomicBoolean(false); AtomicBoolean appendFailed = new AtomicBoolean(false); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { copyInProgress.set(true); while (!outputStreamClosed.get()) {} return answer.callRealMethod(); - }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + }).when(client).copyBlob(any(Path.class), any(Path.class), + nullable(String.class), any(TracingContext.class)); new Thread(() -> { while (!copyInProgress.get()) {} try { @@ -1076,7 +1086,7 @@ public void testAppendAtomicBlobDuringRename() throws Exception { } }).start(); fs.rename(src, dst); - Assertions.assertThat(appendFailed.get()) + assertThat(appendFailed.get()) .describedAs("Append should fail") .isTrue(); } @@ -1105,13 +1115,13 @@ public void testBlobRenameOfDirectoryHavingNeighborWithSamePrefix() fs.create(new Path("/testDir/dirSamePrefix/file1")); fs.create(new Path("/testDir/dirSamePrefix/file2")); fs.rename(new Path("/testDir/dir"), new Path("/testDir/dir2")); - Assertions.assertThat(fs.exists(new Path("/testDir/dirSamePrefix/file1"))) + assertThat(fs.exists(new Path("/testDir/dirSamePrefix/file1"))) .isTrue(); - Assertions.assertThat(fs.exists(new Path("/testDir/dir/file1"))) + assertThat(fs.exists(new Path("/testDir/dir/file1"))) .isFalse(); - Assertions.assertThat(fs.exists(new Path("/testDir/dir/file2"))) + assertThat(fs.exists(new Path("/testDir/dir/file2"))) .isFalse(); - Assertions.assertThat(fs.exists(new Path("/testDir/dir/"))) + assertThat(fs.exists(new Path("/testDir/dir/"))) .isFalse(); } @@ -1131,14 +1141,14 @@ public void testBlobRenameOfDirectoryHavingNeighborWithSamePrefix() @Test public void testBlobRenameWithListGivingPaginatedResultWithOneObjectPerList() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.mkdirs(new Path("/testDir/dir1")); for (int i = 0; i < 10; i++) { fs.create(new Path("/testDir/dir1/file" + i)); } - Mockito.doAnswer(answer -> { + doAnswer(answer -> { String path = answer.getArgument(0); boolean recursive = answer.getArgument(1); String continuation = answer.getArgument(3); @@ -1147,12 +1157,12 @@ public void testBlobRenameWithListGivingPaginatedResultWithOneObjectPerList() .listPath(path, recursive, 1, continuation, context, null); }) .when(spiedClient) - .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class), Mockito.nullable(URI.class)); + .listPath(anyString(), anyBoolean(), anyInt(), + nullable(String.class), + any(TracingContext.class), nullable(URI.class)); fs.rename(new Path("/testDir/dir1"), new Path("/testDir/dir2")); for (int i = 0; i < 10; i++) { - Assertions.assertThat(fs.exists(new Path("/testDir/dir2/file" + i))) + assertThat(fs.exists(new Path("/testDir/dir2/file" + i))) .describedAs("File " + i + " should exist in /testDir/dir2") .isTrue(); } @@ -1173,7 +1183,7 @@ public void testBlobRenameWithListGivingPaginatedResultWithOneObjectPerList() */ @Test public void testProducerStopOnRenameFailure() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); fs.mkdirs(new Path("/src")); ExecutorService executorService = Executors.newFixedThreadPool(10); @@ -1191,20 +1201,20 @@ public void testProducerStopOnRenameFailure() throws Exception { future.get(); } AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); - AbfsBlobClient spiedClient = Mockito.spy(client); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsBlobClient spiedClient = spy(client); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); store.setClient(spiedClient); - Mockito.doReturn(store).when(fs).getAbfsStore(); + doReturn(store).when(fs).getAbfsStore(); final int[] copyCallInvocation = new int[1]; - Mockito.doAnswer(answer -> { + doAnswer(answer -> { throw new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", new Exception()); }).when(spiedClient) - .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + .copyBlob(any(Path.class), any(Path.class), + nullable(String.class), any(TracingContext.class)); AbfsClientTestUtil.mockGetRenameBlobHandler(spiedClient, (blobRenameHandler) -> { - Mockito.doAnswer(answer -> { + doAnswer(answer -> { try { answer.callRealMethod(); } catch (AbfsRestOperationException ex) { @@ -1219,7 +1229,7 @@ public void testProducerStopOnRenameFailure() throws Exception { return null; }); final int[] listCallInvocation = new int[1]; - Mockito.doAnswer(answer -> { + doAnswer(answer -> { if (answer.getArgument(0).equals("/src")) { if (listCallInvocation[0] == 1) { while (copyCallInvocation[0] == 0) {} @@ -1232,13 +1242,13 @@ public void testProducerStopOnRenameFailure() throws Exception { return answer.callRealMethod(); }) .when(spiedClient) - .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), - Mockito.nullable(String.class), Mockito.any(TracingContext.class), Mockito.nullable(URI.class)); + .listPath(anyString(), anyBoolean(), anyInt(), + nullable(String.class), any(TracingContext.class), nullable(URI.class)); intercept(AccessDeniedException.class, () -> { fs.rename(new Path("/src"), new Path("/dst")); }); - Assertions.assertThat(listCallInvocation[0]) + assertThat(listCallInvocation[0]) .describedAs("List on src should have been invoked at-most twice." + "One before consumption and the other after consumption has starting." + "Once consumption fails, listing would be stopped.") @@ -1259,7 +1269,7 @@ public void testProducerStopOnRenameFailure() throws Exception { @Test public void testRenameResumeThroughListStatusWithSrcDirDeletedJustBeforeResume() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.setWorkingDirectory(new Path(ROOT_PATH)); @@ -1270,13 +1280,13 @@ public void testRenameResumeThroughListStatusWithSrcDirDeletedJustBeforeResume() crashRename(fs, client, srcPath.toUri().getPath()); fs.delete(srcPath, true); AtomicInteger copiedBlobs = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { copiedBlobs.incrementAndGet(); return answer.callRealMethod(); - }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + }).when(client).copyBlob(any(Path.class), any(Path.class), + nullable(String.class), any(TracingContext.class)); fs.listStatus(new Path("hbase")); - Assertions.assertThat(copiedBlobs.get()) + assertThat(copiedBlobs.get()) .describedAs("No Copy on resume") .isEqualTo(0); } @@ -1296,7 +1306,7 @@ public void testRenameResumeThroughListStatusWithSrcDirDeletedJustBeforeResume() @Test public void testRenameResumeThroughListStatusWithSrcDirETagChangedJustBeforeResume() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.setWorkingDirectory(new Path(ROOT_PATH)); @@ -1309,13 +1319,13 @@ public void testRenameResumeThroughListStatusWithSrcDirETagChangedJustBeforeResu fs.delete(srcPath, true); fs.mkdirs(srcPath); AtomicInteger copiedBlobs = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { copiedBlobs.incrementAndGet(); return answer.callRealMethod(); - }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + }).when(client).copyBlob(any(Path.class), any(Path.class), + nullable(String.class), any(TracingContext.class)); AtomicInteger pendingJsonDeleted = new AtomicInteger(0); - Mockito.doAnswer(listAnswer -> { + doAnswer(listAnswer -> { Path path = listAnswer.getArgument(0); if (path.toUri().getPath().endsWith(SUFFIX)) { pendingJsonDeleted.incrementAndGet(); @@ -1323,13 +1333,13 @@ public void testRenameResumeThroughListStatusWithSrcDirETagChangedJustBeforeResu return listAnswer.callRealMethod(); }) .when(client) - .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), nullable(String.class), + any(TracingContext.class)); fs.listStatus(new Path("/hbase")); - Assertions.assertThat(copiedBlobs.get()) + assertThat(copiedBlobs.get()) .describedAs("No Copy on resume") .isEqualTo(0); - Assertions.assertThat(pendingJsonDeleted.get()) + assertThat(pendingJsonDeleted.get()) .describedAs("RenamePendingJson should be deleted") .isEqualTo(1); } @@ -1344,7 +1354,7 @@ public void testRenameResumeThroughListStatusWithSrcDirETagChangedJustBeforeResu @Test public void testRenameResumeThroughGetStatusWithSrcDirETagChangedJustBeforeResume() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.setWorkingDirectory(new Path(ROOT_PATH)); @@ -1357,13 +1367,13 @@ public void testRenameResumeThroughGetStatusWithSrcDirETagChangedJustBeforeResum fs.delete(srcPath, true); fs.mkdirs(srcPath); AtomicInteger copiedBlobs = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { copiedBlobs.incrementAndGet(); return answer.callRealMethod(); - }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + }).when(client).copyBlob(any(Path.class), any(Path.class), + nullable(String.class), any(TracingContext.class)); AtomicInteger pendingJsonDeleted = new AtomicInteger(0); - Mockito.doAnswer(listAnswer -> { + doAnswer(listAnswer -> { Path path = listAnswer.getArgument(0); if (path.toUri().getPath().endsWith(SUFFIX)) { pendingJsonDeleted.incrementAndGet(); @@ -1371,15 +1381,15 @@ public void testRenameResumeThroughGetStatusWithSrcDirETagChangedJustBeforeResum return listAnswer.callRealMethod(); }) .when(client) - .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); - Assertions.assertThat(fs.exists(srcPath)) + .deleteBlobPath(any(Path.class), nullable(String.class), + any(TracingContext.class)); + assertThat(fs.exists(srcPath)) .describedAs("Source should exist") .isTrue(); - Assertions.assertThat(copiedBlobs.get()) + assertThat(copiedBlobs.get()) .describedAs("No Copy on resume") .isEqualTo(0); - Assertions.assertThat(pendingJsonDeleted.get()) + assertThat(pendingJsonDeleted.get()) .describedAs("RenamePendingJson should be deleted") .isEqualTo(1); } @@ -1393,7 +1403,7 @@ public void testRenameResumeThroughGetStatusWithSrcDirETagChangedJustBeforeResum @Test public void testRenameSrcDirDeleteEmitDeletionCountInClientRequestId() throws Exception { - AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs = spy(getFileSystem()); assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); String dirPathStr = "/testDir/dir1"; @@ -1415,7 +1425,7 @@ public void testRenameSrcDirDeleteEmitDeletionCountInClientRequestId() fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.RENAME, true, 0); fs.registerListener(tracingHeaderValidator); - Mockito.doAnswer(copyAnswer -> { + doAnswer(copyAnswer -> { if (dirPathStr.equalsIgnoreCase( ((Path) copyAnswer.getArgument(0)).toUri().getPath())) { tracingHeaderValidator.setOperatedBlobCount(BLOB_COUNT); @@ -1424,10 +1434,10 @@ public void testRenameSrcDirDeleteEmitDeletionCountInClientRequestId() return copyAnswer.callRealMethod(); }) .when(client) - .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); - Mockito.doAnswer(deleteAnswer -> { + .copyBlob(any(Path.class), any(Path.class), + nullable(String.class), + any(TracingContext.class)); + doAnswer(deleteAnswer -> { if (dirPathStr.equalsIgnoreCase( ((Path) deleteAnswer.getArgument(0)).toUri().getPath())) { Object result = deleteAnswer.callRealMethod(); @@ -1437,9 +1447,9 @@ public void testRenameSrcDirDeleteEmitDeletionCountInClientRequestId() return deleteAnswer.callRealMethod(); }) .when(client) - .deleteBlobPath(Mockito.any(Path.class), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), + nullable(String.class), + any(TracingContext.class)); fs.rename(new Path(dirPathStr), new Path("/dst/")); } @@ -1465,7 +1475,7 @@ public void testRenameDirWithDifferentParallelismConfig() throws Exception { .getMessage(); // Validate error message for invalid configuration - Assertions.assertThat(errorMessage) + assertThat(errorMessage) .describedAs("maxConsumptionLag should be lesser than maxSize") .contains( "Invalid configuration value detected for \"fs.azure.blob.dir.list.consumer.max.lag\". " @@ -1510,7 +1520,7 @@ public void testGetAtomicRenameKeyForRootFolder() throws Exception { AzureBlobFileSystem fs = getFileSystem(); assumeBlobServiceType(); AbfsBlobClient abfsBlobClient = (AbfsBlobClient) fs.getAbfsClient(); - Assertions.assertThat(abfsBlobClient.isAtomicRenameKey("/hbase")) + assertThat(abfsBlobClient.isAtomicRenameKey("/hbase")) .describedAs("Atomic rename key should return false for Root folder") .isFalse(); } @@ -1551,7 +1561,7 @@ public void testGetAtomicRenameKeyForNonRootFolder() throws Exception { */ @Test public void testGetPathStatusWithoutPendingJsonFile() throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem())) { + try (AzureBlobFileSystem fs = spy(this.getFileSystem())) { assumeBlobServiceType(); Path path = new Path("/hbase/A1/A2"); @@ -1563,12 +1573,12 @@ public void testGetPathStatusWithoutPendingJsonFile() throws Exception { AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); AtomicInteger redoRenameCall = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { redoRenameCall.incrementAndGet(); return answer.callRealMethod(); }).when(client).getRedoRenameAtomicity( - Mockito.any(Path.class), Mockito.anyInt(), - Mockito.any(TracingContext.class)); + any(Path.class), anyInt(), + any(TracingContext.class)); TracingContext tracingContext = new TracingContext( conf.getClientCorrelationId(), fs.getFileSystemId(), @@ -1579,15 +1589,15 @@ public void testGetPathStatusWithoutPendingJsonFile() throws Exception { path.toUri().getPath(), true, tracingContext, null).getResult(); - Assertions.assertThat(abfsHttpOperation.getStatusCode()) + assertThat(abfsHttpOperation.getStatusCode()) .describedAs("Path should be found.") .isEqualTo(HTTP_OK); - Assertions.assertThat(extractEtagHeader(abfsHttpOperation)) + assertThat(extractEtagHeader(abfsHttpOperation)) .describedAs("Etag should be present.") .isNotNull(); - Assertions.assertThat(redoRenameCall.get()) + assertThat(redoRenameCall.get()) .describedAs("There should be no redo rename call.") .isEqualTo(0); } @@ -1605,7 +1615,7 @@ public void testGetPathStatusWithoutPendingJsonFile() throws Exception { */ @Test public void testGetPathStatusWithPendingJsonDir() throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem())) { + try (AzureBlobFileSystem fs = spy(this.getFileSystem())) { assumeBlobServiceType(); Path path = new Path("/hbase/A1/A2"); @@ -1619,11 +1629,11 @@ public void testGetPathStatusWithPendingJsonDir() throws Exception { AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); AtomicInteger redoRenameCall = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { redoRenameCall.incrementAndGet(); return answer.callRealMethod(); - }).when(client).getRedoRenameAtomicity(Mockito.any(Path.class), - Mockito.anyInt(), Mockito.any(TracingContext.class)); + }).when(client).getRedoRenameAtomicity(any(Path.class), + anyInt(), any(TracingContext.class)); TracingContext tracingContext = new TracingContext( conf.getClientCorrelationId(), fs.getFileSystemId(), @@ -1633,19 +1643,19 @@ public void testGetPathStatusWithPendingJsonDir() throws Exception { = client.getPathStatus(path.toUri().getPath(), true, tracingContext, null).getResult(); - Assertions.assertThat(abfsHttpOperation.getStatusCode()) + assertThat(abfsHttpOperation.getStatusCode()) .describedAs("Path should be found.") .isEqualTo(HTTP_OK); - Assertions.assertThat(extractEtagHeader(abfsHttpOperation)) + assertThat(extractEtagHeader(abfsHttpOperation)) .describedAs("Etag should be present.") .isNotNull(); - Assertions.assertThat(redoRenameCall.get()) + assertThat(redoRenameCall.get()) .describedAs("There should be no redo rename call.") .isEqualTo(0); - Assertions.assertThat(fs.exists(new Path(path.getParent(), path.getName() + SUFFIX))) + assertThat(fs.exists(new Path(path.getParent(), path.getName() + SUFFIX))) .describedAs("Directory with suffix -RenamePending.json should exist.") .isTrue(); } @@ -1665,7 +1675,7 @@ public void testRenamePathRetryIdempotency() throws Exception { configuration.set(FS_AZURE_ENABLE_CLIENT_TRANSACTION_ID, "true"); try (AzureBlobFileSystem fs = getFileSystem()) { assumeRecoveryThroughClientTransactionID(false); - AbfsDfsClient abfsClient = (AbfsDfsClient) Mockito.spy(fs.getAbfsClient()); + AbfsDfsClient abfsClient = (AbfsDfsClient) spy(fs.getAbfsClient()); fs.getAbfsStore().setClient(abfsClient); Path sourceDir = path("/testSrc"); assertMkdirs(fs, sourceDir); @@ -1677,9 +1687,9 @@ public void testRenamePathRetryIdempotency() throws Exception { final List headers = new ArrayList<>(); mockRetriedRequest(abfsClient, headers, 0); - AbfsRestOperation getPathRestOp = Mockito.mock(AbfsRestOperation.class); - AbfsHttpOperation op = Mockito.mock(AbfsHttpOperation.class); - Mockito.doAnswer(answer -> { + AbfsRestOperation getPathRestOp = mock(AbfsRestOperation.class); + AbfsHttpOperation op = mock(AbfsHttpOperation.class); + doAnswer(answer -> { String requiredHeader = null; for (AbfsHttpHeader httpHeader : headers) { if (X_MS_CLIENT_TRANSACTION_ID.equalsIgnoreCase( @@ -1690,16 +1700,16 @@ public void testRenamePathRetryIdempotency() throws Exception { } return requiredHeader; }).when(op).getResponseHeader(X_MS_CLIENT_TRANSACTION_ID); - Mockito.doReturn(true).when(getPathRestOp).hasResult(); - Mockito.doReturn(op).when(getPathRestOp).getResult(); - Mockito.doReturn(DIRECTORY) + doReturn(true).when(getPathRestOp).hasResult(); + doReturn(op).when(getPathRestOp).getResult(); + doReturn(DIRECTORY) .when(op) .getResponseHeader(X_MS_RESOURCE_TYPE); - Mockito.doReturn(getPathRestOp).when(abfsClient).getPathStatus( - Mockito.nullable(String.class), Mockito.nullable(Boolean.class), - Mockito.nullable(TracingContext.class), - Mockito.nullable(ContextEncryptionAdapter.class)); - Assertions.assertThat(fs.rename(sourceFilePath, destFilePath)) + doReturn(getPathRestOp).when(abfsClient).getPathStatus( + nullable(String.class), nullable(Boolean.class), + nullable(TracingContext.class), + nullable(ContextEncryptionAdapter.class)); + assertThat(fs.rename(sourceFilePath, destFilePath)) .describedAs("Rename should succeed.") .isTrue(); } @@ -1720,7 +1730,7 @@ public void testRenamePathRetryIdempotency() throws Exception { public void testGetClientTransactionIdAfterRename() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { assumeRecoveryThroughClientTransactionID(false); - AbfsDfsClient abfsDfsClient = (AbfsDfsClient) Mockito.spy(fs.getAbfsClient()); + AbfsDfsClient abfsDfsClient = (AbfsDfsClient) spy(fs.getAbfsClient()); fs.getAbfsStore().setClient(abfsDfsClient); final String[] clientTransactionId = new String[1]; mockAddClientTransactionIdToHeader(abfsDfsClient, clientTransactionId); @@ -1735,11 +1745,11 @@ public void testGetClientTransactionIdAfterRename() throws Exception { final AbfsHttpOperation getPathStatusOp = abfsDfsClient.getPathStatus(destFilePath.toUri().getPath(), false, getTestTracingContext(fs, true), null).getResult(); - Assertions.assertThat( + assertThat( getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID)) .describedAs("Client transaction id should be present in dest file") .isNotNull(); - Assertions.assertThat( + assertThat( getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID)) .describedAs("Client transaction ID should be equal to the one set in the header") .isEqualTo(clientTransactionId[0]); @@ -1757,13 +1767,13 @@ public void testGetClientTransactionIdAfterRename() throws Exception { public void testFailureInGetPathStatusDuringRenameRecovery() throws Exception { try (AzureBlobFileSystem fs = getFileSystem()) { assumeRecoveryThroughClientTransactionID(false); - AbfsDfsClient abfsDfsClient = (AbfsDfsClient) Mockito.spy(fs.getAbfsClient()); + AbfsDfsClient abfsDfsClient = (AbfsDfsClient) spy(fs.getAbfsClient()); fs.getAbfsStore().setClient(abfsDfsClient); final String[] clientTransactionId = new String[1]; mockAddClientTransactionIdToHeader(abfsDfsClient, clientTransactionId); mockRetriedRequest(abfsDfsClient, new ArrayList<>(), 1); int[] flag = new int[1]; - Mockito.doAnswer(getPathStatus -> { + doAnswer(getPathStatus -> { if (flag[0] == 1) { flag[0] += 1; throw new AbfsRestOperationException(HTTP_CLIENT_TIMEOUT, "", "", new Exception()); @@ -1771,9 +1781,9 @@ public void testFailureInGetPathStatusDuringRenameRecovery() throws Exception { flag[0] += 1; return getPathStatus.callRealMethod(); }).when(abfsDfsClient).getPathStatus( - Mockito.nullable(String.class), Mockito.nullable(Boolean.class), - Mockito.nullable(TracingContext.class), - Mockito.nullable(ContextEncryptionAdapter.class)); + nullable(String.class), nullable(Boolean.class), + nullable(TracingContext.class), + nullable(ContextEncryptionAdapter.class)); Path sourceDir = path("/testSrc"); assertMkdirs(fs, sourceDir); @@ -1785,7 +1795,7 @@ public void testFailureInGetPathStatusDuringRenameRecovery() throws Exception { String errorMessage = intercept(AbfsDriverException.class, () -> fs.rename(sourceFilePath, destFilePath)).getErrorMessage(); - Assertions.assertThat(errorMessage) + assertThat(errorMessage) .describedAs("getPathStatus should fail while recovering") .contains(ERR_RENAME_RECOVERY); } @@ -1804,7 +1814,7 @@ public void testRenameWithDestParentNotExist() throws Exception { Path dst = new Path("/A3/A4"); fs.create(new Path(src, "file.txt")); - Assertions.assertThat(fs.rename(src, dst)) + assertThat(fs.rename(src, dst)) .describedAs("Rename should fail as destination parent not exist.") .isFalse(); } @@ -1823,10 +1833,10 @@ public void testRenameWithDestParentAsRoot() throws Exception { Path dst = new Path("/A3"); fs.create(new Path(src, "file.txt")); - Assertions.assertThat(fs.rename(src, dst)) + assertThat(fs.rename(src, dst)) .describedAs("Rename should succeed.") .isTrue(); - Assertions.assertThat(fs.exists(new Path(dst, "file.txt"))) + assertThat(fs.exists(new Path(dst, "file.txt"))) .describedAs("File should exist in destination directory.") .isTrue(); } @@ -1845,10 +1855,10 @@ public void testFileRenameWithDestAsRoot() throws Exception { Path dst = new Path("/"); fs.create(src); - Assertions.assertThat(fs.rename(src, dst)) + assertThat(fs.rename(src, dst)) .describedAs("Rename should succeed.") .isTrue(); - Assertions.assertThat(fs.exists(new Path(dst, "file.txt"))) + assertThat(fs.exists(new Path(dst, "file.txt"))) .describedAs("File should exist in root.") .isTrue(); } @@ -1867,10 +1877,10 @@ public void testDirRenameWithDestAsRoot() throws Exception { Path dst = new Path("/"); fs.create(new Path(src, "file.txt")); - Assertions.assertThat(fs.rename(src, dst)) + assertThat(fs.rename(src, dst)) .describedAs("Rename should succeed.") .isTrue(); - Assertions.assertThat(fs.exists(new Path(dst, src.getName()))) + assertThat(fs.exists(new Path(dst, src.getName()))) .describedAs("A2 directory should exist in root.") .isTrue(); } @@ -1895,14 +1905,14 @@ public void testRenameWithMultipleDirsInSource() throws Exception { fs.mkdirs(new Path("/testDir/dir2")); fs.create(new Path("/testDir/dir2/file2")); createAzCopyFolder(new Path("/testDir/dir3")); - Assertions.assertThat(fs.rename(new Path("/testDir"), + assertThat(fs.rename(new Path("/testDir"), new Path("/testDir2"))) .describedAs("Rename should succeed.") .isTrue(); - Assertions.assertThat(fs.exists(new Path("/testDir"))) + assertThat(fs.exists(new Path("/testDir"))) .describedAs("Old directory should not exist.") .isFalse(); - Assertions.assertThat(fs.exists(new Path("/testDir2"))) + assertThat(fs.exists(new Path("/testDir2"))) .describedAs("New directory should exist.") .isTrue(); } @@ -1927,14 +1937,14 @@ public void testRenameWithMultipleImplicitDirsInSource() throws Exception { createAzCopyFolder(new Path("/testDir/dir2")); createAzCopyFile(new Path("/testDir/dir2/file2")); createAzCopyFolder(new Path("/testDir/dir3")); - Assertions.assertThat(fs.rename(new Path("/testDir"), + assertThat(fs.rename(new Path("/testDir"), new Path("/testDir2"))) .describedAs("Rename should succeed.") .isTrue(); - Assertions.assertThat(fs.exists(new Path("/testDir"))) + assertThat(fs.exists(new Path("/testDir"))) .describedAs("Old directory should not exist.") .isFalse(); - Assertions.assertThat(fs.exists(new Path("/testDir2"))) + assertThat(fs.exists(new Path("/testDir2"))) .describedAs("New directory should exist.") .isTrue(); } @@ -1954,14 +1964,14 @@ public void testRenameWithExplicitDirInSource() throws Exception { assumeBlobServiceType(); fs.create(new Path("/testDir/dir3/file2")); fs.create(new Path("/testDir/dir3/file1")); - Assertions.assertThat(fs.rename(new Path("/testDir"), + assertThat(fs.rename(new Path("/testDir"), new Path("/testDir2"))) .describedAs("Rename should succeed.") .isTrue(); - Assertions.assertThat(fs.exists(new Path("/testDir"))) + assertThat(fs.exists(new Path("/testDir"))) .describedAs("Old directory should not exist.") .isFalse(); - Assertions.assertThat(fs.exists(new Path("/testDir2"))) + assertThat(fs.exists(new Path("/testDir2"))) .describedAs("New directory should exist.") .isTrue(); } @@ -1975,10 +1985,10 @@ public void testRenameWithExplicitDirInSource() throws Exception { * @return the spied AbfsClient for interaction verification */ private AbfsClient addSpyHooksOnClient(final AzureBlobFileSystem fs) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - Mockito.doReturn(store).when(fs).getAbfsStore(); - AbfsClient client = Mockito.spy(store.getClient()); - Mockito.doReturn(client).when(store).getClient(); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = spy(store.getClient()); + doReturn(client).when(store).getClient(); return client; } @@ -1998,11 +2008,11 @@ private void crashRenameAndRecover(final AzureBlobFileSystem fs, final FunctionRaisingIOE recoveryCallable) throws Exception { crashRename(fs, client, srcPath); - AzureBlobFileSystem fs2 = Mockito.spy(getFileSystem()); + AzureBlobFileSystem fs2 = spy(getFileSystem()); fs2.setWorkingDirectory(new Path(ROOT_PATH)); client = (AbfsBlobClient) addSpyHooksOnClient(fs2); int[] renameJsonDeleteCounter = new int[1]; - Mockito.doAnswer(answer -> { + doAnswer(answer -> { if ((ROOT_PATH + srcPath + SUFFIX) .equalsIgnoreCase(((Path) answer.getArgument(0)).toUri().getPath())) { renameJsonDeleteCounter[0] = 1; @@ -2010,10 +2020,10 @@ private void crashRenameAndRecover(final AzureBlobFileSystem fs, return answer.callRealMethod(); }) .when(client) - .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), nullable(String.class), + any(TracingContext.class)); recoveryCallable.apply(fs2); - Assertions.assertThat(renameJsonDeleteCounter[0]) + assertThat(renameJsonDeleteCounter[0]) .describedAs("RenamePendingJson should be deleted") .isEqualTo(1); //List would complete the rename orchestration. @@ -2045,12 +2055,12 @@ private void crashRename(final AzureBlobFileSystem fs, return null; }); //Fail rename orchestration on path hbase/test1/test2/test3/file1 - Mockito.doThrow(new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", + doThrow(new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", new Exception())) .when(client) - .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .copyBlob(any(Path.class), any(Path.class), + nullable(String.class), + any(TracingContext.class)); LambdaTestUtils.intercept(AccessDeniedException.class, () -> { fs.rename(new Path(srcPath), new Path("hbase/test4")); @@ -2117,24 +2127,24 @@ private void testRenamePreRenameFailureResolution(final AzureBlobFileSystem fs) */ Answer renamePendingJsonCreateAns = createAnswer -> { Path path = createAnswer.getArgument(0); - Mockito.doAnswer(clientFlushAns -> { + doAnswer(clientFlushAns -> { if (renamePendingJsonWriteCounter[0]++ == 0) { fs.delete(path, true); } return clientFlushAns.callRealMethod(); }) .when(client) - .flush(Mockito.any(byte[].class), Mockito.anyString(), - Mockito.anyBoolean(), Mockito.nullable(String.class), - Mockito.nullable(String.class), Mockito.anyString(), - Mockito.nullable(ContextEncryptionAdapter.class), - Mockito.any(TracingContext.class)); + .flush(any(byte[].class), anyString(), + anyBoolean(), nullable(String.class), + nullable(String.class), anyString(), + nullable(ContextEncryptionAdapter.class), + any(TracingContext.class)); return createAnswer.callRealMethod(); }; RenameAtomicityTestUtils.addCreatePathMock(client, renamePendingJsonCreateAns); fs.rename(src, dest); - Assertions.assertThat(renamePendingJsonWriteCounter[0]) + assertThat(renamePendingJsonWriteCounter[0]) .describedAs("Creation of RenamePendingJson should be attempted twice") .isEqualTo(2); } @@ -2158,7 +2168,7 @@ private void testAtomicityRedoInvalidFile(final AzureBlobFileSystem fs) os.write("{".getBytes(StandardCharsets.UTF_8)); os.close(); int[] renameJsonDeleteCounter = new int[1]; - Mockito.doAnswer(deleteAnswer -> { + doAnswer(deleteAnswer -> { Path ansPath = deleteAnswer.getArgument(0); if (renameJson.toUri() .getPath() @@ -2168,16 +2178,16 @@ private void testAtomicityRedoInvalidFile(final AzureBlobFileSystem fs) return deleteAnswer.callRealMethod(); }) .when(client) - .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .deleteBlobPath(any(Path.class), nullable(String.class), + any(TracingContext.class)); new RenameAtomicity(renameJson, 1, getTestTracingContext(fs, true), null, client).redo(); - Assertions.assertThat(renameJsonDeleteCounter[0]) + assertThat(renameJsonDeleteCounter[0]) .describedAs("RenamePendingJson should be deleted") .isEqualTo(1); - Mockito.verify(client, Mockito.times(0)).copyBlob(Mockito.any(Path.class), - Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + verify(client, times(0)).copyBlob(any(Path.class), + any(Path.class), nullable(String.class), + any(TracingContext.class)); } /** @@ -2191,18 +2201,18 @@ private void testAtomicityRedoInvalidFile(final AzureBlobFileSystem fs) */ private void addMockForProgressStatusOnCopyOperation(final AbfsBlobClient spiedClient) throws AzureBlobFileSystemException { - Mockito.doAnswer(answer -> { - AbfsRestOperation op = Mockito.spy( + doAnswer(answer -> { + AbfsRestOperation op = spy( (AbfsRestOperation) answer.callRealMethod()); - AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); - Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + AbfsHttpOperation httpOp = spy(op.getResult()); + doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( HttpHeaderConfigurations.X_MS_COPY_STATUS); - Mockito.doReturn(httpOp).when(op).getResult(); + doReturn(httpOp).when(op).getResult(); return op; }) .when(spiedClient) - .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + .copyBlob(any(Path.class), any(Path.class), + nullable(String.class), any(TracingContext.class)); } /** @@ -2218,19 +2228,19 @@ private void addMockForCopyOperationFinalStatus(final AbfsBlobClient spiedClient final String requiredCopyFinalStatus) { AbfsClientTestUtil.mockGetRenameBlobHandler(spiedClient, blobRenameHandler -> { - Mockito.doAnswer(onHandleCopyInProgress -> { + doAnswer(onHandleCopyInProgress -> { Path handlePath = onHandleCopyInProgress.getArgument(0); TracingContext tracingContext = onHandleCopyInProgress.getArgument( 1); - Mockito.doAnswer(onStatusCheck -> { - AbfsRestOperation op = Mockito.spy( + doAnswer(onStatusCheck -> { + AbfsRestOperation op = spy( (AbfsRestOperation) onStatusCheck.callRealMethod()); - AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); - Mockito.doReturn(requiredCopyFinalStatus) + AbfsHttpOperation httpOp = spy(op.getResult()); + doReturn(requiredCopyFinalStatus) .when(httpOp) .getResponseHeader( HttpHeaderConfigurations.X_MS_COPY_STATUS); - Mockito.doReturn(httpOp).when(op).getResult(); + doReturn(httpOp).when(op).getResult(); return op; }) .when(spiedClient) @@ -2239,8 +2249,8 @@ private void addMockForCopyOperationFinalStatus(final AbfsBlobClient spiedClient return onHandleCopyInProgress.callRealMethod(); }) .when(blobRenameHandler) - .handleCopyInProgress(Mockito.any(Path.class), - Mockito.any(TracingContext.class), Mockito.any(String.class)); + .handleCopyInProgress(any(Path.class), + any(TracingContext.class), any(String.class)); return null; }); } @@ -2264,7 +2274,7 @@ private void renameDir(AzureBlobFileSystem currentFs, String producerQueueSize, config.set(FS_AZURE_CONSUMER_MAX_LAG, consumerMaxLag); config.set(FS_AZURE_BLOB_DIR_RENAME_MAX_THREAD, maxThread); try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(), config)) { - Assertions.assertThat(fs.rename(src, dst)) + assertThat(fs.rename(src, dst)) .describedAs("Rename should succeed.") .isTrue(); } @@ -2281,27 +2291,27 @@ private void renameDir(AzureBlobFileSystem currentFs, String producerQueueSize, private void performRenameAndValidate(AzureBlobFileSystem fs, Path src, Path dst, String fileName) throws IOException { // Assert the source directory exists - Assertions.assertThat(fs.exists(src)) + assertThat(fs.exists(src)) .describedAs("Old directory should exist before rename") .isTrue(); // Perform rename - Assertions.assertThat(fs.rename(src, dst)) + assertThat(fs.rename(src, dst)) .describedAs("Rename should succeed.") .isTrue(); // Assert the destination directory and file exist after rename - Assertions.assertThat(fs.exists(new Path(dst, fileName))) + assertThat(fs.exists(new Path(dst, fileName))) .describedAs("Rename should be successful") .isTrue(); // Assert the source directory no longer exists - Assertions.assertThat(fs.exists(src)) + assertThat(fs.exists(src)) .describedAs("Old directory should not exist") .isFalse(); // Assert the new destination directory exists - Assertions.assertThat(fs.exists(dst)) + assertThat(fs.exists(dst)) .describedAs("New directory should exist") .isTrue(); } @@ -2314,7 +2324,7 @@ private void performRenameAndValidate(AzureBlobFileSystem fs, Path src, Path dst * @param expected the expected value (true or false) */ private void validateAtomicRenameKey(AbfsBlobClient abfsBlobClient, String path, boolean expected) { - Assertions.assertThat(abfsBlobClient.isAtomicRenameKey(path)) + assertThat(abfsBlobClient.isAtomicRenameKey(path)) .describedAs("Atomic rename key check for path: " + path) .isEqualTo(expected); } @@ -2342,14 +2352,14 @@ public void answer(final AbfsRestOperation mockedObj, throws AbfsRestOperationException { if (count == 0) { count = 1; - AbfsHttpOperation op = Mockito.mock(AbfsHttpOperation.class); - Mockito.doReturn(HTTP_METHOD_PUT).when(op).getMethod(); - Mockito.doReturn(EMPTY_STRING).when(op).getStorageErrorMessage(); - Mockito.doReturn(SOURCE_PATH_NOT_FOUND.getErrorCode()).when(op) + AbfsHttpOperation op = mock(AbfsHttpOperation.class); + doReturn(HTTP_METHOD_PUT).when(op).getMethod(); + doReturn(EMPTY_STRING).when(op).getStorageErrorMessage(); + doReturn(SOURCE_PATH_NOT_FOUND.getErrorCode()).when(op) .getStorageErrorCode(); - Mockito.doReturn(true).when(mockedObj).hasResult(); - Mockito.doReturn(op).when(mockedObj).getResult(); - Mockito.doReturn(HTTP_NOT_FOUND).when(op).getStatusCode(); + doReturn(true).when(mockedObj).hasResult(); + doReturn(op).when(mockedObj).getResult(); + doReturn(HTTP_NOT_FOUND).when(op).getStatusCode(); headers.addAll(mockedObj.getRequestHeaders()); throw new AbfsRestOperationException(HTTP_NOT_FOUND, SOURCE_PATH_NOT_FOUND.getErrorCode(), EMPTY_STRING, null, op); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameRecovery.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameRecovery.java index e4d9f826000c4..ff76292e551d8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameRecovery.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameRecovery.java @@ -22,9 +22,7 @@ import java.util.Arrays; import java.util.concurrent.atomic.AtomicInteger; -import org.assertj.core.api.Assertions; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -52,6 +50,14 @@ import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicity.SUFFIX; import static org.apache.hadoop.fs.azurebfs.utils.AbfsTestUtils.createFiles; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; /** * Test rename recovery operation. @@ -73,7 +79,7 @@ public ITestAzureBlobFileSystemRenameRecovery() throws Exception { */ @Test public void testRenameCopyFailureInBetween() throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem(getConfig()))) { + try (AzureBlobFileSystem fs = spy(this.getFileSystem(getConfig()))) { assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.getAbfsStore().setClient(client); @@ -95,7 +101,7 @@ public void testRenameCopyFailureInBetween() throws Exception { */ @Test public void testRenameDeleteFailureInBetween() throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem(getConfig()))) { + try (AzureBlobFileSystem fs = spy(this.getFileSystem(getConfig()))) { assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.getAbfsStore().setClient(client); @@ -107,7 +113,7 @@ public void testRenameDeleteFailureInBetween() throws Exception { // Track the number of delete operations AtomicInteger deleteCall = new AtomicInteger(0); - Mockito.doAnswer(deleteRequest -> { + doAnswer(deleteRequest -> { if (deleteCall.get() == FAILED_CALL) { throw new AbfsRestOperationException( BLOB_PATH_NOT_FOUND.getStatusCode(), @@ -117,8 +123,8 @@ public void testRenameDeleteFailureInBetween() throws Exception { } deleteCall.incrementAndGet(); return deleteRequest.callRealMethod(); - }).when(client).deleteBlobPath(Mockito.any(Path.class), - Mockito.anyString(), Mockito.any(TracingContext.class)); + }).when(client).deleteBlobPath(any(Path.class), + anyString(), any(TracingContext.class)); renameOperationWithRecovery(fs, src, dst, deleteCall); } @@ -133,7 +139,7 @@ public void testRenameDeleteFailureInBetween() throws Exception { */ @Test public void testRenameRecoveryWhenDestAlreadyExist() throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem(getConfig()))) { + try (AzureBlobFileSystem fs = spy(this.getFileSystem(getConfig()))) { assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.getAbfsStore().setClient(client); @@ -162,7 +168,7 @@ public void testRenameRecoveryWhenDestAlreadyExist() throws Exception { */ @Test public void testRenameRecoveryWithMarkerPresentInDest() throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem(getConfig()))) { + try (AzureBlobFileSystem fs = spy(this.getFileSystem(getConfig()))) { assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.getAbfsStore().setClient(client); @@ -185,7 +191,7 @@ public void testRenameRecoveryWithMarkerPresentInDest() throws Exception { */ @Test public void testRenameWhenAlreadyRenamePendingJsonFilePresent() throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem(getConfig()))) { + try (AzureBlobFileSystem fs = spy(this.getFileSystem(getConfig()))) { assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); fs.getAbfsStore().setClient(client); @@ -218,7 +224,7 @@ public void testListCrashRecoveryWithSingleChildFolder() throws Exception { FileStatus[] fileStatuses = fs.listStatus(new Path("/hbase/A1")); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List should return 0 file") .isEqualTo(0); assertPendingJsonFile(fs, renameJson, fileStatuses, path, false); @@ -244,7 +250,7 @@ public void testListCrashRecoveryWithMultipleChildFolder() throws Exception { FileStatus[] fileStatuses = fs.listStatus(new Path("/hbase/A1")); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List should return 2 files") .isEqualTo(2); assertPendingJsonFile(fs, renameJson, fileStatuses, path, false); @@ -271,7 +277,7 @@ public void testListCrashRecoveryWithPendingJsonFile() throws Exception { FileStatus[] fileStatuses = fs.listStatus(path.getParent()); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List should return 2 files") .isEqualTo(2); assertPendingJsonFile(fs, renameJson, fileStatuses, path, false); @@ -297,7 +303,7 @@ public void testListCrashRecoveryWithoutAnyPendingJsonFile() throws Exception { FileStatus[] fileStatuses = fs.listStatus(path.getParent()); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List should return 3 files") .isEqualTo(3); // Pending json file not present, no recovery take place, so source directory should exist. @@ -315,7 +321,7 @@ public void testListCrashRecoveryWithoutAnyPendingJsonFile() throws Exception { */ @Test public void testListCrashRecoveryWithPendingJsonDir() throws Exception { - try (AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem())) { + try (AzureBlobFileSystem fs = spy(this.getFileSystem())) { assumeBlobServiceType(); AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); @@ -327,23 +333,23 @@ public void testListCrashRecoveryWithPendingJsonDir() throws Exception { fs.create(new Path(path, "file2.txt")); AtomicInteger redoRenameCall = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { redoRenameCall.incrementAndGet(); return answer.callRealMethod(); - }).when(client).getRedoRenameAtomicity(Mockito.any(Path.class), - Mockito.anyInt(), Mockito.any(TracingContext.class)); + }).when(client).getRedoRenameAtomicity(any(Path.class), + anyInt(), any(TracingContext.class)); FileStatus[] fileStatuses = fs.listStatus(path.getParent()); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List should return 3 files") .isEqualTo(3); - Assertions.assertThat(redoRenameCall.get()) + assertThat(redoRenameCall.get()) .describedAs("No redo rename call should be made") .isEqualTo(0); - Assertions.assertThat( + assertThat( Arrays.stream(fileStatuses) .anyMatch(status -> renameJson.toUri().getPath().equals(status.getPath().toUri().getPath()))) .describedAs("Directory with suffix -RenamePending.json should exist.") @@ -384,19 +390,19 @@ renameJson2, getTestTracingContext(fs, true), fs.create(new Path(path, "file2.txt")); AtomicInteger redoRenameCall = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { redoRenameCall.incrementAndGet(); return answer.callRealMethod(); - }).when(client).getRedoRenameAtomicity(Mockito.any(Path.class), - Mockito.anyInt(), Mockito.any(TracingContext.class)); + }).when(client).getRedoRenameAtomicity(any(Path.class), + anyInt(), any(TracingContext.class)); FileStatus[] fileStatuses = fs.listStatus(path.getParent()); - Assertions.assertThat(fileStatuses.length) + assertThat(fileStatuses.length) .describedAs("List should return 0 paths") .isEqualTo(0); - Assertions.assertThat(redoRenameCall.get()) + assertThat(redoRenameCall.get()) .describedAs("2 redo rename calls should be made") .isEqualTo(2); assertPathStatus(fs, path, false, @@ -436,11 +442,11 @@ public void testGetPathStatusWithPendingJsonFile() throws Exception { AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); AtomicInteger redoRenameCall = new AtomicInteger(0); - Mockito.doAnswer(answer -> { + doAnswer(answer -> { redoRenameCall.incrementAndGet(); return answer.callRealMethod(); - }).when(client).getRedoRenameAtomicity(Mockito.any(Path.class), - Mockito.anyInt(), Mockito.any(TracingContext.class)); + }).when(client).getRedoRenameAtomicity(any(Path.class), + anyInt(), any(TracingContext.class)); TracingContext tracingContext = new TracingContext( conf.getClientCorrelationId(), fs.getFileSystemId(), @@ -451,15 +457,15 @@ public void testGetPathStatusWithPendingJsonFile() throws Exception { path.toUri().getPath(), true, tracingContext, null)).getErrorCode(); - Assertions.assertThat(azureServiceErrorCode.getErrorCode()) + assertThat(azureServiceErrorCode.getErrorCode()) .describedAs("Path had to be recovered from atomic rename operation.") .isEqualTo(PATH_NOT_FOUND.getErrorCode()); - Assertions.assertThat(redoRenameCall.get()) + assertThat(redoRenameCall.get()) .describedAs("There should be one redo rename call") .isEqualTo(1); - Assertions.assertThat(fs.exists(renameJson)) + assertThat(fs.exists(renameJson)) .describedAs("Rename Pending Json file should not exist.") .isFalse(); } @@ -495,32 +501,32 @@ public void testETagChangedDuringRename() throws Exception { fs.create(new Path(path, "file1.txt")); fs.create(new Path(path, "file2.txt")); AtomicInteger numberOfCopyBlobCalls = new AtomicInteger(0); - Mockito.doAnswer(copyBlob -> { + doAnswer(copyBlob -> { numberOfCopyBlobCalls.incrementAndGet(); return copyBlob.callRealMethod(); }) .when(abfsBlobClient) - .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), - Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + .copyBlob(any(Path.class), any(Path.class), + nullable(String.class), + any(TracingContext.class)); AtomicInteger numberOfRedoRenameAtomicityCalls = new AtomicInteger(0); - Mockito.doAnswer(redoRenameAtomicity -> { + doAnswer(redoRenameAtomicity -> { numberOfRedoRenameAtomicityCalls.incrementAndGet(); return redoRenameAtomicity.callRealMethod(); }) .when(abfsBlobClient) - .getRedoRenameAtomicity(Mockito.any(Path.class), Mockito.anyInt(), - Mockito.any(TracingContext.class)); + .getRedoRenameAtomicity(any(Path.class), anyInt(), + any(TracingContext.class)); // Call list status to trigger rename redo fs.listStatus(path.getParent()); - Assertions.assertThat(numberOfRedoRenameAtomicityCalls.get()) + assertThat(numberOfRedoRenameAtomicityCalls.get()) .describedAs("There should be one call to getRedoRenameAtomicity") .isEqualTo(1); - Assertions.assertThat(numberOfCopyBlobCalls.get()) + assertThat(numberOfCopyBlobCalls.get()) .describedAs("There should be no copy blob call") .isEqualTo(0); - Assertions.assertThat(fs.exists(renameJson)) + assertThat(fs.exists(renameJson)) .describedAs("Rename Pending Json file should not exist.") .isFalse(); } @@ -544,7 +550,7 @@ private void triggerRenameRecovery(AzureBlobFileSystem fs, Path src) throws Exce fs.getAbfsStore().getClient().getPathStatus(src.toUri().getPath(), true, tracingContext, null); }).getErrorCode(); - Assertions.assertThat(errorCode) + assertThat(errorCode) .describedAs("Path had to be recovered from atomic rename operation.") .isEqualTo(PATH_NOT_FOUND); } @@ -564,7 +570,7 @@ private void triggerRenameRecovery(AzureBlobFileSystem fs, Path src) throws Exce private void renameCrashInBetween(AzureBlobFileSystem fs, Path src, Path dst, AbfsBlobClient client, AtomicInteger copyCall) throws Exception { - Mockito.doAnswer(copyRequest -> { + doAnswer(copyRequest -> { if (copyCall.get() == FAILED_CALL) { throw new AbfsRestOperationException( BLOB_ALREADY_EXISTS.getStatusCode(), @@ -574,9 +580,9 @@ private void renameCrashInBetween(AzureBlobFileSystem fs, Path src, Path dst, } copyCall.incrementAndGet(); return copyRequest.callRealMethod(); - }).when(client).copyBlob(Mockito.any(Path.class), - Mockito.any(Path.class), Mockito.nullable(String.class), - Mockito.any(TracingContext.class)); + }).when(client).copyBlob(any(Path.class), + any(Path.class), nullable(String.class), + any(TracingContext.class)); renameOperationWithRecovery(fs, src, dst, copyCall); } @@ -601,10 +607,10 @@ private Configuration getConfig() { * @return the spied AbfsClient for interaction verification */ private AbfsClient addSpyHooksOnClient(final AzureBlobFileSystem fs) { - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - Mockito.doReturn(store).when(fs).getAbfsStore(); - AbfsClient client = Mockito.spy(store.getClient()); - Mockito.doReturn(client).when(store).getClient(); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = spy(store.getClient()); + doReturn(client).when(store).getClient(); return client; } @@ -648,7 +654,7 @@ private void assertPathStatus(AzureBlobFileSystem fs, Path path, path.toUri().getPath(), tracingContext, null, true) .getResult().getStatusCode(); - Assertions.assertThat(actualStatus) + assertThat(actualStatus) .describedAs("%s should exists", description) .isEqualTo(HTTP_OK); } else { @@ -657,7 +663,7 @@ private void assertPathStatus(AzureBlobFileSystem fs, Path path, client.getPathStatus(path.toUri().getPath(), true, tracingContext, null); }).getErrorCode(); - Assertions.assertThat(errorCode) + assertThat(errorCode) .describedAs("%s should not exists", description) .isEqualTo(BLOB_PATH_NOT_FOUND); } @@ -672,12 +678,12 @@ private void assertPathStatus(AzureBlobFileSystem fs, Path path, */ private AzureBlobFileSystem createJsonFile(Path path, Path renameJson) throws IOException { - final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + final AzureBlobFileSystem fs = spy(this.getFileSystem()); assumeBlobServiceType(); - AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); - Mockito.doReturn(store).when(fs).getAbfsStore(); - AbfsClient client = Mockito.spy(store.getClient()); - Mockito.doReturn(client).when(store).getClient(); + AzureBlobFileSystemStore store = spy(fs.getAbfsStore()); + doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = spy(store.getClient()); + doReturn(client).when(store).getClient(); fs.setWorkingDirectory(new Path(ROOT_PATH)); fs.create(new Path(path, "file.txt")); @@ -690,7 +696,7 @@ renameJson, getTestTracingContext(fs, true), fileStatus.getEtag(), client) .preRename(); - Assertions.assertThat(fs.exists(renameJson)) + assertThat(fs.exists(renameJson)) .describedAs("Rename Pending Json file should exist.") .isTrue(); @@ -708,12 +714,12 @@ renameJson, getTestTracingContext(fs, true), */ private void renameOperationWithRecovery(AzureBlobFileSystem fs, Path src, Path dst, AtomicInteger countCall) throws Exception { - Assertions.assertThat(fs.rename(src, dst)) + assertThat(fs.rename(src, dst)) .describedAs("Rename should crash in between.") .isFalse(); // Validate copy operation count - Assertions.assertThat(countCall.get()) + assertThat(countCall.get()) .describedAs("Operation count should be less than 10.") .isLessThan(TOTAL_FILES); @@ -724,7 +730,7 @@ private void renameOperationWithRecovery(AzureBlobFileSystem fs, Path src, countCall.set(0); triggerRenameRecovery(fs, src); - Assertions.assertThat(countCall.get()) + assertThat(countCall.get()) .describedAs("Operation count should be greater than 0.") .isGreaterThan(0); @@ -745,11 +751,11 @@ private void renameOperationWithRecovery(AzureBlobFileSystem fs, Path src, private void assertPendingJsonFile(AzureBlobFileSystem fs, Path renameJson, FileStatus[] fileStatuses, Path srcPath, boolean isSrcPathExist) throws Exception { - Assertions.assertThat(fs.exists(renameJson)) + assertThat(fs.exists(renameJson)) .describedAs("Rename Pending Json file should not exist.") .isFalse(); - Assertions.assertThat( + assertThat( Arrays.stream(fileStatuses) .anyMatch(status -> renameJson.toUri().getPath() @@ -758,7 +764,7 @@ private void assertPendingJsonFile(AzureBlobFileSystem fs, "List status should not contains any file with suffix -RenamePending.json.") .isFalse(); - Assertions.assertThat( + assertThat( Arrays.stream(fileStatuses) .anyMatch(status -> srcPath.toUri().getPath() diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java index 589ca5415fc80..e8c0960cee8d6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java @@ -20,12 +20,10 @@ import java.util.Arrays; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; @@ -37,20 +35,15 @@ /** * Parameterized test of rename operations of unicode paths. */ -@RunWith(Parameterized.class) public class ITestAzureBlobFileSystemRenameUnicode extends AbstractAbfsIntegrationTest { - @Parameterized.Parameter public String srcDir; - @Parameterized.Parameter(1) public String destDir; - @Parameterized.Parameter(2) public String filename; - @Parameterized.Parameters public static Iterable params() { return Arrays.asList( new Object[][]{ @@ -69,12 +62,22 @@ public static Iterable params() { public ITestAzureBlobFileSystemRenameUnicode() throws Exception { } + public void initITestAzureBlobFileSystemRenameUnicode(String pSrcDir, + String pDestDir, String pFilename) { + this.srcDir = pSrcDir; + this.destDir = pDestDir; + this.filename = pFilename; + } + /** * Known issue: ListStatus operation to folders/files whose name contains '?' will fail. * This is because Auto rest client didn't encode '?' in the uri query parameters */ - @Test - public void testRenameFileUsingUnicode() throws Exception { + @ParameterizedTest + @MethodSource("params") + public void testRenameFileUsingUnicode(String pSrcDir, + String pDestDir, String pFilename) throws Exception { + initITestAzureBlobFileSystemRenameUnicode(pSrcDir, pDestDir, pFilename); final AzureBlobFileSystem fs = getFileSystem(); Path folderPath1 = path(srcDir); assertMkdirs(fs, folderPath1); @@ -92,9 +95,8 @@ public void testRenameFileUsingUnicode() throws Exception { FileStatus[] fileStatus = fs.listStatus(folderPath2); assertNotNull(fileStatus); - assertTrue( - "Empty listing returned from listStatus(\"" + folderPath2 + "\")", - fileStatus.length > 0); + assertTrue(fileStatus.length > 0, + "Empty listing returned from listStatus(\"" + folderPath2 + "\")"); assertEquals(fileStatus[0].getPath().getName(), filename); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java index ef7f1565df73f..9844a2edea82d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java @@ -20,19 +20,15 @@ import java.io.IOException; import java.util.Arrays; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; /** * Test AzureBlobFileSystemStore listStatus with startFrom. * */ -@RunWith(Parameterized.class) public class ITestAzureBlobFileSystemStoreListStatusWithRange extends AbstractAbfsIntegrationTest { private static final boolean SUCCEED = true; @@ -42,22 +38,22 @@ public class ITestAzureBlobFileSystemStoreListStatusWithRange extends private AzureBlobFileSystemStore store; private AzureBlobFileSystem fs; - @Parameterized.Parameter public String path; /** * A valid startFrom for listFileStatus with range is a non-fully qualified dir/file name * */ - @Parameterized.Parameter(1) public String startFrom; - @Parameterized.Parameter(2) public int expectedStartIndexInArray; - @Parameterized.Parameter(3) public boolean expectedResult; - @Parameterized.Parameters(name = "Testing path \"{0}\", startFrom: \"{1}\", Expecting result : {3}") // Test path + protected ITestAzureBlobFileSystemStoreListStatusWithRange() throws Exception { + super(); + } + + // Test path public static Iterable params() { return Arrays.asList( new Object[][]{ @@ -92,8 +88,8 @@ public static Iterable params() { }); } - public ITestAzureBlobFileSystemStoreListStatusWithRange() throws Exception { - super(); + public void initITestAzureBlobFileSystemStoreListStatusWithRange(String pPath, String pStartFrom, + int pExpectedStartIndexInArray, boolean pExpectedResult) throws Exception { if (this.getFileSystem() == null) { super.createFileSystem(); } @@ -102,37 +98,47 @@ public ITestAzureBlobFileSystemStoreListStatusWithRange() throws Exception { prepareTestFiles(); // Sort the names for verification, ABFS service should return the results in order. Arrays.sort(SORTED_ENTRY_NAMES); + this.path = pPath; + this.startFrom = pStartFrom; + this.expectedStartIndexInArray = pExpectedStartIndexInArray; + this.expectedResult = pExpectedResult; } - @Test - public void testListWithRange() throws IOException { + @ParameterizedTest(name = "Testing path {0}, startFrom: {1}, Expecting result: {3}") + @MethodSource("params") + public void testListWithRange(String pPath, String pStartFrom, + int pExpectedStartIndexInArray, boolean pExpectedResult) throws Exception { + initITestAzureBlobFileSystemStoreListStatusWithRange(pPath, pStartFrom, + pExpectedStartIndexInArray, pExpectedResult); try { FileStatus[] listResult = store.listStatus(new Path(path), startFrom, getTestTracingContext(fs, true)); if (!expectedResult) { - Assert.fail("Excepting failure with IllegalArgumentException"); + fail("Excepting failure with IllegalArgumentException"); } verifyFileStatus(listResult, new Path(path), expectedStartIndexInArray); } catch (IllegalArgumentException ex) { if (expectedResult) { - Assert.fail("Excepting success"); + fail("Excepting success"); } } } // compare the file status - private void verifyFileStatus(FileStatus[] listResult, Path parentPath, int startIndexInSortedName) throws IOException { + private void verifyFileStatus(FileStatus[] listResult, Path parentPath, + int startIndexInSortedName) throws IOException { if (startIndexInSortedName == -1) { - Assert.assertEquals("Expected empty FileStatus array", 0, listResult.length); + assertEquals(0, listResult.length, "Expected empty FileStatus array"); return; } FileStatus[] allFileStatuses = fs.listStatus(parentPath); - Assert.assertEquals("number of dir/file doesn't match", - SORTED_ENTRY_NAMES.length, allFileStatuses.length); + assertEquals(SORTED_ENTRY_NAMES.length, allFileStatuses.length, + "number of dir/file doesn't match"); int indexInResult = 0; for (int index = startIndexInSortedName; index < SORTED_ENTRY_NAMES.length; index++) { - Assert.assertEquals("fileStatus doesn't match", allFileStatuses[index], listResult[indexInResult++]); + assertEquals(allFileStatuses[index], listResult[indexInResult++], + "fileStatus doesn't match"); } } @@ -141,7 +147,7 @@ private void prepareTestFiles() throws IOException { // created 2 level file structures for (String levelOneFolder : SORTED_ENTRY_NAMES) { Path levelOnePath = new Path("/" + levelOneFolder); - Assert.assertTrue(fs.mkdirs(levelOnePath)); + assertTrue(fs.mkdirs(levelOnePath)); for (String fileName : SORTED_ENTRY_NAMES) { Path filePath = new Path(levelOnePath, fileName); ContractTestUtils.touch(fs, filePath); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index 5c07bbd13132c..b169958f3b739 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -25,9 +25,8 @@ import java.util.List; import java.util.UUID; -import org.junit.Assume; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -42,7 +41,6 @@ import org.apache.hadoop.fs.permission.FsPermission; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; -import static org.junit.Assume.assumeTrue; import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; @@ -51,6 +49,7 @@ import static org.apache.hadoop.fs.permission.AclEntryType.OTHER; import static org.apache.hadoop.fs.permission.AclEntryType.MASK; import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; +import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Test acl operations. @@ -256,30 +255,33 @@ public void testModifyAclEntriesStickyBit() throws Exception { assertPermission(fs, (short) 01750); } - @Test(expected=FileNotFoundException.class) + @Test public void testModifyAclEntriesPathNotFound() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - // Path has not been created. - List aclSpec = Lists.newArrayList( - aclEntry(ACCESS, USER, ALL), - aclEntry(ACCESS, USER, FOO, ALL), - aclEntry(ACCESS, GROUP, READ_EXECUTE), - aclEntry(ACCESS, OTHER, NONE)); - fs.modifyAclEntries(path, aclSpec); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, FOO, ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE)); + fs.modifyAclEntries(path, aclSpec); + }); } - @Test (expected=Exception.class) + @Test public void testModifyAclEntriesDefaultOnFile() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); - List aclSpec = Lists.newArrayList( + assertThrows(Exception.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); + List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, FOO, ALL)); - fs.modifyAclEntries(path, aclSpec); + fs.modifyAclEntries(path, aclSpec); + }); } @Test @@ -327,20 +329,20 @@ public void testModifyAclEntriesWithAccessMask() throws Exception { assertPermission(fs, (short) RW_X); } - @Test(expected=PathIOException.class) + @Test public void testModifyAclEntriesWithDuplicateEntries() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); - List aclSpec = Lists.newArrayList( - aclEntry(ACCESS, MASK, EXECUTE)); - fs.setAcl(path, aclSpec); - - List modifyAclSpec = Lists.newArrayList( - aclEntry(ACCESS, USER, READ_WRITE), - aclEntry(ACCESS, USER, READ_WRITE)); - fs.modifyAclEntries(path, modifyAclSpec); + assertThrows(PathIOException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, MASK, EXECUTE)); + fs.setAcl(path, aclSpec); + List modifyAclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, READ_WRITE), aclEntry(ACCESS, USER, READ_WRITE)); + fs.modifyAclEntries(path, modifyAclSpec); + }); } @Test @@ -502,59 +504,61 @@ public void testRemoveAclEntriesStickyBit() throws Exception { assertPermission(fs, (short) 01750); } - @Test(expected=FileNotFoundException.class) + @Test public void testRemoveAclEntriesPathNotFound() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - // Path has not been created. - List aclSpec = Lists.newArrayList( - aclEntry(ACCESS, USER, FOO)); - fs.removeAclEntries(path, aclSpec); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, FOO)); + fs.removeAclEntries(path, aclSpec); + }); } - @Test(expected=PathIOException.class) + @Test public void testRemoveAclEntriesAccessMask() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); - List aclSpec = Lists.newArrayList( - aclEntry(ACCESS, MASK, EXECUTE), - aclEntry(ACCESS, USER, FOO, ALL)); - fs.setAcl(path, aclSpec); - - fs.removeAclEntries(path, Lists.newArrayList(aclEntry(ACCESS, MASK, NONE))); + assertThrows(PathIOException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); + List aclSpec = Lists.newArrayList(aclEntry(ACCESS, MASK, EXECUTE), + aclEntry(ACCESS, USER, FOO, ALL)); + fs.setAcl(path, aclSpec); + fs.removeAclEntries(path, Lists.newArrayList(aclEntry(ACCESS, MASK, NONE))); + }); } - @Test(expected=PathIOException.class) + @Test public void testRemoveAclEntriesDefaultMask() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); - List aclSpec = Lists.newArrayList( - aclEntry(DEFAULT, MASK, EXECUTE), - aclEntry(DEFAULT, USER, FOO, ALL)); - fs.setAcl(path, aclSpec); - - fs.removeAclEntries(path, Lists.newArrayList(aclEntry(DEFAULT, MASK, NONE))); + assertThrows(PathIOException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); + List aclSpec = Lists.newArrayList(aclEntry(DEFAULT, MASK, EXECUTE), + aclEntry(DEFAULT, USER, FOO, ALL)); + fs.setAcl(path, aclSpec); + fs.removeAclEntries(path, Lists.newArrayList(aclEntry(DEFAULT, MASK, NONE))); + }); } - @Test(expected=PathIOException.class) + @Test public void testRemoveAclEntriesWithDuplicateEntries() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); - List aclSpec = Lists.newArrayList( - aclEntry(DEFAULT, MASK, EXECUTE)); - fs.setAcl(path, aclSpec); - - List removeAclSpec = Lists.newArrayList( - aclEntry(DEFAULT, USER, READ_WRITE), - aclEntry(DEFAULT, USER, READ_WRITE)); - fs.removeAclEntries(path, removeAclSpec); + assertThrows(PathIOException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, MASK, EXECUTE)); + fs.setAcl(path, aclSpec); + List removeAclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, READ_WRITE), + aclEntry(DEFAULT, USER, READ_WRITE)); + fs.removeAclEntries(path, removeAclSpec); + }); } @Test @@ -652,13 +656,15 @@ public void testRemoveDefaultAclStickyBit() throws Exception { assertPermission(fs, (short) STICKY_RWX_RWX); } - @Test(expected=FileNotFoundException.class) + @Test public void testRemoveDefaultAclPathNotFound() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - // Path has not been created. - fs.removeDefaultAcl(path); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + // Path has not been created. + fs.removeDefaultAcl(path); + }); } @Test @@ -736,13 +742,15 @@ public void testRemoveAclOnlyDefault() throws Exception { assertPermission(fs, (short) RWX_RX); } - @Test(expected=FileNotFoundException.class) + @Test public void testRemoveAclPathNotFound() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - // Path has not been created. - fs.removeAcl(path); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + // Path has not been created. + fs.removeAcl(path); + }); } @Test @@ -904,30 +912,33 @@ public void testSetAclStickyBit() throws Exception { assertPermission(fs, (short) STICKY_RWX_RWX); } - @Test(expected=FileNotFoundException.class) + @Test public void testSetAclPathNotFound() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - // Path has not been created. - List aclSpec = Lists.newArrayList( - aclEntry(ACCESS, USER, READ_WRITE), - aclEntry(ACCESS, USER, FOO, READ), - aclEntry(ACCESS, GROUP, READ), - aclEntry(ACCESS, OTHER, NONE)); - fs.setAcl(path, aclSpec); + assertThrows(FileNotFoundException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, READ_WRITE), + aclEntry(ACCESS, USER, FOO, READ), + aclEntry(ACCESS, GROUP, READ), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + }); } - @Test(expected=Exception.class) + @Test public void testSetAclDefaultOnFile() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); - List aclSpec = Lists.newArrayList( - aclEntry(DEFAULT, USER, FOO, ALL)); - fs.setAcl(path, aclSpec); + assertThrows(Exception.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, FOO, ALL)); + fs.setAcl(path, aclSpec); + }); } @Test @@ -954,16 +965,17 @@ public void testSetAclDoesNotChangeDefaultMask() throws Exception { assertPermission(fs, (short) RWX_RX_RX); } - @Test(expected=PathIOException.class) + @Test public void testSetAclWithDuplicateEntries() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(getIsNamespaceEnabled(fs)); - path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); - List aclSpec = Lists.newArrayList( - aclEntry(ACCESS, MASK, EXECUTE), - aclEntry(ACCESS, MASK, EXECUTE)); - fs.setAcl(path, aclSpec); + assertThrows(PathIOException.class, () -> { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(getIsNamespaceEnabled(fs)); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, MASK, EXECUTE), aclEntry(ACCESS, MASK, EXECUTE)); + fs.setAcl(path, aclSpec); + }); } @Test @@ -1043,7 +1055,7 @@ public void testSetPermissionOnlyDefault() throws Exception { public void testDefaultAclNewFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(getIsNamespaceEnabled(fs)); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); + assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1060,7 +1072,7 @@ public void testDefaultAclNewFile() throws Exception { } @Test - @Ignore // wait umask fix to be deployed + @Disabled // wait umask fix to be deployed public void testOnlyAccessAclNewFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(getIsNamespaceEnabled(fs)); @@ -1126,7 +1138,7 @@ public void testDefaultAclNewDir() throws Exception { public void testOnlyAccessAclNewDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(getIsNamespaceEnabled(fs)); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); + assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1166,7 +1178,7 @@ public void testDefaultMinimalAclNewDir() throws Exception { public void testDefaultAclNewFileWithMode() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(getIsNamespaceEnabled(fs)); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); + assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX)); List aclSpec = Lists.newArrayList( @@ -1189,7 +1201,7 @@ public void testDefaultAclNewFileWithMode() throws Exception { public void testDefaultAclNewDirWithMode() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(getIsNamespaceEnabled(fs)); - Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); + assumeTrue(getIngressServiceType() == AbfsServiceType.DFS); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX)); List aclSpec = Lists.newArrayList( @@ -1299,7 +1311,7 @@ public void testEnsureAclOperationWorksForRoot() throws Exception { public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); - Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); @@ -1322,7 +1334,7 @@ public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { @Test public void testSetPermissionForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); @@ -1341,7 +1353,7 @@ public void testSetPermissionForNonNamespaceEnabledAccount() throws Exception { @Test public void testModifyAclEntriesForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { @@ -1349,7 +1361,7 @@ public void testModifyAclEntriesForNonNamespaceEnabledAccount() throws Exception aclEntry(DEFAULT, GROUP, FOO, ALL), aclEntry(ACCESS, GROUP, BAR, ALL)); fs.modifyAclEntries(filePath, aclSpec); - assertFalse("UnsupportedOperationException is expected", false); + assertFalse(false, "UnsupportedOperationException is expected"); } catch (UnsupportedOperationException ex) { //no-op } @@ -1358,7 +1370,7 @@ public void testModifyAclEntriesForNonNamespaceEnabledAccount() throws Exception @Test public void testRemoveAclEntriesEntriesForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { @@ -1366,7 +1378,7 @@ public void testRemoveAclEntriesEntriesForNonNamespaceEnabledAccount() throws Ex aclEntry(DEFAULT, GROUP, FOO, ALL), aclEntry(ACCESS, GROUP, BAR, ALL)); fs.removeAclEntries(filePath, aclSpec); - assertFalse("UnsupportedOperationException is expected", false); + assertFalse(false, "UnsupportedOperationException is expected"); } catch (UnsupportedOperationException ex) { //no-op } @@ -1375,12 +1387,12 @@ public void testRemoveAclEntriesEntriesForNonNamespaceEnabledAccount() throws Ex @Test public void testRemoveDefaultAclForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { fs.removeDefaultAcl(filePath); - assertFalse("UnsupportedOperationException is expected", false); + assertFalse(false, "UnsupportedOperationException is expected"); } catch (UnsupportedOperationException ex) { //no-op } @@ -1389,12 +1401,12 @@ public void testRemoveDefaultAclForNonNamespaceEnabledAccount() throws Exception @Test public void testRemoveAclForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { fs.removeAcl(filePath); - assertFalse("UnsupportedOperationException is expected", false); + assertFalse(false, "UnsupportedOperationException is expected"); } catch (UnsupportedOperationException ex) { //no-op } @@ -1403,7 +1415,7 @@ public void testRemoveAclForNonNamespaceEnabledAccount() throws Exception { @Test public void testSetAclForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { @@ -1411,7 +1423,7 @@ public void testSetAclForNonNamespaceEnabledAccount() throws Exception { aclEntry(DEFAULT, GROUP, FOO, ALL), aclEntry(ACCESS, GROUP, BAR, ALL)); fs.setAcl(filePath, aclSpec); - assertFalse("UnsupportedOperationException is expected", false); + assertFalse(false, "UnsupportedOperationException is expected"); } catch (UnsupportedOperationException ex) { //no-op } @@ -1420,12 +1432,12 @@ public void testSetAclForNonNamespaceEnabledAccount() throws Exception { @Test public void testGetAclStatusForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { AclStatus aclSpec = fs.getAclStatus(filePath); - assertFalse("UnsupportedOperationException is expected", false); + assertFalse(false, "UnsupportedOperationException is expected"); } catch (UnsupportedOperationException ex) { //no-op } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientUrlScheme.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientUrlScheme.java index 44665f50c11fc..98e3b7b4beb3a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientUrlScheme.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientUrlScheme.java @@ -22,12 +22,6 @@ import java.net.URL; import java.util.Arrays; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; @@ -35,23 +29,21 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ALWAYS_USE_HTTPS; +import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Parameterized test of ABFS CLIENT URL scheme verification. */ - -@RunWith(Parameterized.class) public class ITestClientUrlScheme extends AbstractAbfsIntegrationTest{ - @Parameterized.Parameter public boolean useSecureScheme; - @Parameterized.Parameter(1) public boolean alwaysUseHttps; - @Parameterized.Parameters public static Iterable params() { return Arrays.asList( new Object[][]{ @@ -65,12 +57,21 @@ public static Iterable params() { public ITestClientUrlScheme() throws Exception { super(); // authentication like OAUTH must use HTTPS - Assume.assumeTrue("ITestClientUrlScheme is skipped because auth type is not SharedKey", - getAuthType() == AuthType.SharedKey); + assumeTrue(getAuthType() == AuthType.SharedKey, + "ITestClientUrlScheme is skipped because auth type is not SharedKey"); + } + + public void initITestClientUrlScheme(boolean pUseSecureScheme, + boolean pAlwaysUseHttps) { + this.useSecureScheme = pUseSecureScheme; + this.alwaysUseHttps = pAlwaysUseHttps; } - @Test - public void testClientUrlScheme() throws Exception { + @ParameterizedTest + @MethodSource("params") + public void testClientUrlScheme(boolean pUseSecureScheme, + boolean pAlwaysUseHttps) throws Exception { + initITestClientUrlScheme(pUseSecureScheme, pAlwaysUseHttps); String[] urlWithoutScheme = this.getTestUrl().split(":"); String fsUrl; // update filesystem scheme @@ -110,9 +111,9 @@ public void testClientUrlScheme() throws Exception { String url = ((URL) baseUrlField.get(client)).toString(); if (expectHttpConnection) { - Assert.assertTrue(url.startsWith(FileSystemUriSchemes.HTTP_SCHEME)); + assertTrue(url.startsWith(FileSystemUriSchemes.HTTP_SCHEME)); } else { - Assert.assertTrue(url.startsWith(FileSystemUriSchemes.HTTPS_SCHEME)); + assertTrue(url.startsWith(FileSystemUriSchemes.HTTPS_SCHEME)); } } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java index f7d4a5b7a83e7..cf3fb6b9b2c6c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java @@ -20,8 +20,7 @@ import java.net.URI; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; @@ -34,7 +33,7 @@ import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME; import static org.apache.hadoop.fs.CommonPathCapabilities.FS_ACLS; import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; -import static org.junit.Assume.assumeTrue; +import static org.assertj.core.api.Assertions.assertThat; /** * Test AzureBlobFileSystem initialization. @@ -58,7 +57,7 @@ public void ensureAzureBlobFileSystemIsInitialized() throws Exception { null, null, null)); - assertNotNull("working directory", fs.getWorkingDirectory()); + assertNotNull(fs.getWorkingDirectory(), "working directory"); } @Test @@ -79,7 +78,7 @@ public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception { null, null, null)); - assertNotNull("working directory", fs.getWorkingDirectory()); + assertNotNull(fs.getWorkingDirectory(), "working directory"); } } @@ -89,18 +88,18 @@ public void testFileSystemCapabilities() throws Throwable { final Path p = new Path("}"); // etags always present - Assertions.assertThat(fs.hasPathCapability(p, ETAGS_AVAILABLE)) + assertThat(fs.hasPathCapability(p, ETAGS_AVAILABLE)) .describedAs("path capability %s in %s", ETAGS_AVAILABLE, fs) .isTrue(); // readahead always correct - Assertions.assertThat(fs.hasPathCapability(p, CAPABILITY_SAFE_READAHEAD)) + assertThat(fs.hasPathCapability(p, CAPABILITY_SAFE_READAHEAD)) .describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs) .isTrue(); // etags-over-rename and ACLs are either both true or both false. final boolean etagsAcrossRename = fs.hasPathCapability(p, ETAGS_PRESERVED_IN_RENAME); final boolean acls = fs.hasPathCapability(p, FS_ACLS); - Assertions.assertThat(etagsAcrossRename) + assertThat(etagsAcrossRename) .describedAs("capabilities %s=%s and %s=%s in %s", ETAGS_PRESERVED_IN_RENAME, etagsAcrossRename, FS_ACLS, acls, fs) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 3f2a4fe98802c..d2dcfdf5acfcc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -20,8 +20,7 @@ import java.util.Hashtable; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -29,8 +28,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; - import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +import static org.assertj.core.api.Assertions.assertThat; /** * Test FileSystemProperties. @@ -99,32 +98,34 @@ public void testBase64PathProperties() throws Exception { assertEquals(properties, fetchedProperties); } - @Test (expected = Exception.class) + @Test public void testBase64InvalidFileSystemProperties() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Hashtable properties = new Hashtable<>(); - properties.put("key", "{ value: valueæ­² }"); - TracingContext tracingContext = getTestTracingContext(fs, true); - fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); - Hashtable fetchedProperties = fs.getAbfsStore() - .getFilesystemProperties(tracingContext); - - assertEquals(properties, fetchedProperties); + assertThrows(Exception.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Hashtable properties = new Hashtable<>(); + properties.put("key", "{ value: valueæ­² }"); + TracingContext tracingContext = getTestTracingContext(fs, true); + fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); + Hashtable fetchedProperties = fs.getAbfsStore() + .getFilesystemProperties(tracingContext); + assertEquals(properties, fetchedProperties); + }); } - @Test (expected = Exception.class) + @Test public void testBase64InvalidPathProperties() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Hashtable properties = new Hashtable<>(); - properties.put("key", "{ value: valueTestå…© }"); - Path testPath = path(TEST_PATH); - touch(testPath); - TracingContext tracingContext = getTestTracingContext(fs, true); - fs.getAbfsStore().setPathProperties(testPath, properties, tracingContext); - Hashtable fetchedProperties = fs.getAbfsStore() - .getPathStatus(testPath, tracingContext); - - assertEquals(properties, fetchedProperties); + assertThrows(Exception.class, () -> { + final AzureBlobFileSystem fs = getFileSystem(); + final Hashtable properties = new Hashtable<>(); + properties.put("key", "{ value: valueTestå…© }"); + Path testPath = path(TEST_PATH); + touch(testPath); + TracingContext tracingContext = getTestTracingContext(fs, true); + fs.getAbfsStore().setPathProperties(testPath, properties, tracingContext); + Hashtable fetchedProperties = fs.getAbfsStore() + .getPathStatus(testPath, tracingContext); + assertEquals(properties, fetchedProperties); + }); } @Test @@ -157,10 +158,10 @@ public void testBufferSizeSet() throws Exception { = (AbfsInputStream) inputStream.getWrappedStream(); int actualBufferSize = abfsInputStream.getBufferSize(); - Assertions.assertThat(actualBufferSize) + assertThat(actualBufferSize) .describedAs("Buffer size should be set to the value in the configuration") .isEqualTo(bufferSizeConfig); - Assertions.assertThat(actualBufferSize) + assertThat(actualBufferSize) .describedAs("Buffer size should not be set to the value passed as argument") .isNotEqualTo(bufferSizeArg); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java index 4393bd82b1161..8e8b16362c8fd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java @@ -20,7 +20,7 @@ import java.net.URI; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; @@ -43,8 +43,8 @@ public ITestFileSystemRegistration() throws Exception { private void assertConfigMatches(Configuration conf, String key, String expected) { String v = conf.get(key); - assertNotNull("No value for key " + key, v); - assertEquals("Wrong value for key " + key, expected, v); + assertNotNull(v, "No value for key " + key); + assertEquals(expected, v, "Wrong value for key " + key); } @Test @@ -79,14 +79,14 @@ public void testSecureAbfsFileContextRegistered() throws Throwable { public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { Configuration rawConfig = getRawConfiguration(); AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(rawConfig); - assertNotNull("filesystem", fs); + assertNotNull(fs, "filesystem"); if (this.getAuthType() == AuthType.OAuth) { Abfss afs = (Abfss) FileContext.getFileContext(rawConfig).getDefaultFileSystem(); - assertNotNull("filecontext", afs); + assertNotNull(afs, "filecontext"); } else { Abfs afs = (Abfs) FileContext.getFileContext(rawConfig).getDefaultFileSystem(); - assertNotNull("filecontext", afs); + assertNotNull(afs, "filecontext"); } } @@ -106,8 +106,8 @@ public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exceptio defaultUri.toString()); SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.get(rawConfig); - assertNotNull("filesystem", fs); + assertNotNull(fs, "filesystem"); Abfss afs = (Abfss) FileContext.getFileContext(rawConfig).getDefaultFileSystem(); - assertNotNull("filecontext", afs); + assertNotNull(afs, "filecontext"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 0e69a09f8002b..25112348308d1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -21,10 +21,7 @@ import java.io.IOException; import java.util.UUID; -import org.junit.Assume; -import org.junit.Test; -import org.assertj.core.api.Assertions; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; @@ -45,12 +42,16 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.accountProperty; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.api.Assumptions.assumeTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -78,28 +79,27 @@ public ITestGetNameSpaceEnabled() throws Exception { @Test public void testXNSAccount() throws IOException { - Assume.assumeTrue("Skip this test because the account being used for test is a non XNS account", - isUsingXNSAccount); - assertTrue("Expecting getIsNamespaceEnabled() return true", - getIsNamespaceEnabled(getFileSystem())); + assumeTrue(isUsingXNSAccount, "Skip this test because the account being used for test is a non XNS account"); + assertTrue(getIsNamespaceEnabled(getFileSystem()), + "Expecting getIsNamespaceEnabled() return true"); } @Test public void testNonXNSAccount() throws IOException { assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT); - Assume.assumeFalse("Skip this test because the account being used for test is a XNS account", - isUsingXNSAccount); - assertFalse("Expecting getIsNamespaceEnabled() return false", - getIsNamespaceEnabled(getFileSystem())); + assumeFalse(isUsingXNSAccount, + "Skip this test because the account being used for test is a XNS account"); + assertFalse(getIsNamespaceEnabled(getFileSystem()), + "Expecting getIsNamespaceEnabled() return false"); } @Test public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT); - Assume.assumeTrue("Blob Endpoint Does not Allow FS init on HNS Account", - getAbfsServiceType() == AbfsServiceType.DFS); + assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS, + "Blob Endpoint Does not Allow FS init on HNS Account"); AzureBlobFileSystem fs = getNewFSWithHnsConf(TRUE_STR); - Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs( + assertThat(getIsNamespaceEnabled(fs)).describedAs( "getIsNamespaceEnabled should return true when the " + "config is set as true").isTrue(); fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false)); @@ -110,7 +110,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception { assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT); AzureBlobFileSystem fs = getNewFSWithHnsConf(FALSE_STR); - Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs( + assertThat(getIsNamespaceEnabled(fs)).describedAs( "getIsNamespaceEnabled should return false when the " + "config is set as false").isFalse(); fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false)); @@ -122,7 +122,7 @@ private void unsetAndAssert() throws Exception { DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED); boolean expectedValue = this.getConfiguration() .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); - Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs( + assertThat(getIsNamespaceEnabled(fs)).describedAs( "getIsNamespaceEnabled should return the value " + "configured for fs.azure.test.namespace.enabled") .isEqualTo(expectedValue); @@ -169,7 +169,7 @@ public void testFailedRequestWhenFSNotExist() throws Exception { ? FILESYSTEM_NOT_FOUND_ERROR : CONTAINER_NOT_FOUND_ERROR; - Assertions.assertThat(ex.getMessage()).describedAs( + assertThat(ex.getMessage()).describedAs( "Expecting FileNotFoundException with message: " + expectedExceptionMessage) .contains(expectedExceptionMessage); } @@ -234,8 +234,8 @@ private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException { private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() throws IOException { - final AzureBlobFileSystem abfs = Mockito.spy(this.getFileSystem()); - final AzureBlobFileSystemStore abfsStore = Mockito.spy(abfs.getAbfsStore()); + final AzureBlobFileSystem abfs = spy(this.getFileSystem()); + final AzureBlobFileSystemStore abfsStore = spy(abfs.getAbfsStore()); final AbfsDfsClient mockClient = mock(AbfsDfsClient.class); doReturn(abfsStore).when(abfs).getAbfsStore(); doReturn(mockClient).when(abfsStore).getClient(); @@ -260,7 +260,7 @@ public void ensureGetAclDetermineHnsStatusAccurately() throws Exception { private void ensureGetAclDetermineHnsStatusAccuratelyInternal(int statusCode, boolean expectedValue, boolean isExceptionExpected) throws Exception { - AzureBlobFileSystemStore store = Mockito.spy(getFileSystem().getAbfsStore()); + AzureBlobFileSystemStore store = spy(getFileSystem().getAbfsStore()); AbfsClient mockClient = mock(AbfsClient.class); store.setNamespaceEnabled(Trilean.UNKNOWN); doReturn(mockClient).when(store).getClient(AbfsServiceType.DFS); @@ -271,20 +271,20 @@ private void ensureGetAclDetermineHnsStatusAccuratelyInternal(int statusCode, if (isExceptionExpected) { try { store.getIsNamespaceEnabled(getTestTracingContext(getFileSystem(), false)); - Assertions.fail( + fail( "Exception Should have been thrown with status code: " + statusCode); } catch (AbfsRestOperationException caughtEx) { - Assertions.assertThat(caughtEx.getStatusCode()).isEqualTo(statusCode); - Assertions.assertThat(caughtEx.getErrorMessage()).isEqualTo(ex.getErrorMessage()); + assertThat(caughtEx.getStatusCode()).isEqualTo(statusCode); + assertThat(caughtEx.getErrorMessage()).isEqualTo(ex.getErrorMessage()); } } // This should not trigger extra getAcl() call in case of exceptions. boolean isHnsEnabled = store.getIsNamespaceEnabled( getTestTracingContext(getFileSystem(), false)); - Assertions.assertThat(isHnsEnabled).isEqualTo(expectedValue); + assertThat(isHnsEnabled).isEqualTo(expectedValue); // GetAcl() should be called only once to determine the HNS status. - Mockito.verify(mockClient, times(1)) + verify(mockClient, times(1)) .getAclStatus(anyString(), any(TracingContext.class)); } @@ -335,7 +335,7 @@ public void testAccountSpecificConfig() throws Exception { try { assertFileSystemInitWithExpectedHNSSettings(rawConfig, false); } catch (Exception e) { - Assertions.assertThat(e.getCause().getMessage()) + assertThat(e.getCause().getMessage()) .describedAs("getAcl() to determine HNS Nature of account should" + "fail with Unknown Host Exception").contains("UnknownHostException"); } @@ -344,7 +344,7 @@ public void testAccountSpecificConfig() throws Exception { private void assertFileSystemInitWithExpectedHNSSettings( Configuration configuration, boolean expectedIsHnsEnabledValue) throws IOException { try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration)) { - Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs( + assertThat(getIsNamespaceEnabled(fs)).describedAs( "getIsNamespaceEnabled should return true when the " + "account specific config is not set").isEqualTo(expectedIsHnsEnabledValue); } catch (Exception e) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestOauthOverAbfsScheme.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestOauthOverAbfsScheme.java index 2c80ce85f4e77..924a343730ebf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestOauthOverAbfsScheme.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestOauthOverAbfsScheme.java @@ -17,11 +17,12 @@ */ package org.apache.hadoop.fs.azurebfs; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + import java.lang.reflect.Field; import java.net.URL; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; @@ -35,8 +36,8 @@ public class ITestOauthOverAbfsScheme extends AbstractAbfsIntegrationTest { public ITestOauthOverAbfsScheme() throws Exception { - Assume.assumeTrue("ITestOauthOverAbfsScheme is skipped because auth type is not OAuth", - getAuthType() == AuthType.OAuth); + assumeTrue(getAuthType() == AuthType.OAuth, + "ITestOauthOverAbfsScheme is skipped because auth type is not OAuth"); } @Test @@ -56,8 +57,7 @@ public void testOauthOverSchemeAbfs() throws Exception { baseUrlField.setAccessible(true); String url = ((URL) baseUrlField.get(client)).toString(); - Assume.assumeTrue("OAuth authentication over scheme abfs must use HTTPS", - url.startsWith(FileSystemUriSchemes.HTTPS_SCHEME)); - + assumeTrue(url.startsWith(FileSystemUriSchemes.HTTPS_SCHEME), + "OAuth authentication over scheme abfs must use HTTPS"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java index d7c7d655a63ba..ed55f562c254d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java @@ -19,8 +19,7 @@ import java.io.IOException; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -29,6 +28,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.junit.jupiter.api.Assumptions.assumeTrue; public class ITestSharedKeyAuth extends AbstractAbfsIntegrationTest { @@ -38,7 +38,7 @@ public ITestSharedKeyAuth() throws Exception { @Test public void testWithWrongSharedKey() throws Exception { - Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + assumeTrue(this.getAuthType() == AuthType.SharedKey); Configuration config = this.getRawConfiguration(); config.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java index 4b124231d8447..e8975d16cd2da 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java @@ -24,12 +24,6 @@ import java.util.Map; import java.io.IOException; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.runners.Parameterized; -import org.junit.runner.RunWith; -import org.junit.Test; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -38,12 +32,17 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.BYTES_SENT; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.SEND_REQUESTS; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_ENABLE_SMALL_WRITE_OPTIMIZATION; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_APPENDBLOB_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Test combination for small writes with flush and close operations. @@ -66,7 +65,6 @@ * 4. Execute test iterations with asserts on number of store requests made and * validating file content. */ -@RunWith(Parameterized.class) public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest { private static final int ONE_MB = 1024 * 1024; private static final int TWO_MB = 2 * ONE_MB; @@ -75,45 +73,37 @@ public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest { private static final int QUARTER_TEST_BUFFER_SIZE = TWO_MB / 4; private static final int TEST_FLUSH_ITERATION = 2; - @Parameterized.Parameter public String testScenario; - @Parameterized.Parameter(1) public boolean enableSmallWriteOptimization; /** * If true, will initiate close after appends. (That is, no explicit hflush or * hsync calls will be made from client app.) */ - @Parameterized.Parameter(2) public boolean directCloseTest; /** * If non-zero, test file should be created as pre-requisite with this size. */ - @Parameterized.Parameter(3) public Integer startingFileSize; /** * Determines the write sizes to be issued by client app. */ - @Parameterized.Parameter(4) public Integer recurringClientWriteSize; /** * Determines the number of Client writes to make. */ - @Parameterized.Parameter(5) public Integer numOfClientWrites; /** * True, if the small write optimization is supposed to be effective in * the scenario. */ - @Parameterized.Parameter(6) public boolean flushExpectedToBeMergedWithAppend; - @Parameterized.Parameters(name = "{0}") public static Iterable params() { return Arrays.asList( // Parameter Order : @@ -301,15 +291,36 @@ public ITestSmallWriteOptimization() throws Exception { super(); } - @Test - public void testSmallWriteOptimization() - throws IOException { + public void initITestSmallWriteOptimization(String pTestScenario, + boolean pEnableSmallWriteOptimization, boolean pDirectCloseTest, + Integer pStartingFileSize, Integer pRecurringClientWriteSize, + Integer pNumOfClientWrites, boolean pFlushExpectedToBeMergedWithAppend) throws Exception { + this.testScenario = pTestScenario; + this.enableSmallWriteOptimization = pEnableSmallWriteOptimization; + this.directCloseTest = pDirectCloseTest; + this.startingFileSize = pStartingFileSize; + this.recurringClientWriteSize = pRecurringClientWriteSize; + this.numOfClientWrites = pNumOfClientWrites; + this.flushExpectedToBeMergedWithAppend = pFlushExpectedToBeMergedWithAppend; + } + + @ParameterizedTest(name = "Scenario = {0}") + @MethodSource("params") + public void testSmallWriteOptimization(String pTestScenario, + boolean pEnableSmallWriteOptimization, boolean pDirectCloseTest, + Integer pStartingFileSize, Integer pRecurringClientWriteSize, + Integer pNumOfClientWrites, boolean pFlushExpectedToBeMergedWithAppend) throws Exception { + + initITestSmallWriteOptimization(pTestScenario, pEnableSmallWriteOptimization, + pDirectCloseTest, pStartingFileSize, pRecurringClientWriteSize, + pNumOfClientWrites, pFlushExpectedToBeMergedWithAppend); + boolean serviceDefaultOptmSettings = DEFAULT_AZURE_ENABLE_SMALL_WRITE_OPTIMIZATION; // Tests with Optimization should only run if service has the feature on by // default. Default settings will be turned on when server support is // available on all store prod regions. if (enableSmallWriteOptimization) { - Assume.assumeTrue(serviceDefaultOptmSettings); + assumeTrue(serviceDefaultOptmSettings); } final AzureBlobFileSystem currentfs = this.getFileSystem(); @@ -317,7 +328,7 @@ public void testSmallWriteOptimization() boolean isAppendBlobTestSettingEnabled = (config.get(FS_AZURE_TEST_APPENDBLOB_ENABLED) == "true"); // This optimization doesnt take effect when append blob is on. - Assume.assumeFalse(isAppendBlobTestSettingEnabled); + assumeFalse(isAppendBlobTestSettingEnabled); config.set(ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE, Integer.toString(TEST_BUFFER_SIZE)); config.set(ConfigurationKeys.AZURE_ENABLE_SMALL_WRITE_OPTIMIZATION, Boolean.toString(enableSmallWriteOptimization)); @@ -420,7 +431,7 @@ private void formulateSmallWriteTestAppendPattern(final AzureBlobFileSystem fs, && (numOfBuffersWrittenToStore == 0) && (wasDataPendingToBeWrittenToServer); - Assertions.assertThat(flushWillBeMergedWithAppend) + assertThat(flushWillBeMergedWithAppend) .describedAs(flushExpectedToBeMergedWithAppend ? "Flush was to be merged with Append" : "Flush should not have been merged with Append") @@ -481,7 +492,7 @@ private int createFileWithStartingTestSize(AzureBlobFileSystem fs, byte[] writeB startingFileSize); opStream.close(); - Assertions.assertThat(fs.getFileStatus(testPath).getLen()) + assertThat(fs.getFileStatus(testPath).getLen()) .describedAs("File should be of size %d at the start of test.", startingFileSize) .isEqualTo(startingFileSize); @@ -495,7 +506,7 @@ private void validateStoreAppends(AzureBlobFileSystem fs, byte[] bufferWritten) throws IOException { // Final validation - Assertions.assertThat(fs.getFileStatus(testPath).getLen()) + assertThat(fs.getFileStatus(testPath).getLen()) .describedAs("File should be of size %d at the end of test.", totalFileSize) .isEqualTo(totalFileSize); @@ -503,8 +514,8 @@ private void validateStoreAppends(AzureBlobFileSystem fs, byte[] fileReadFromStore = new byte[totalFileSize]; fs.open(testPath).read(fileReadFromStore, 0, totalFileSize); - assertArrayEquals("Test file content incorrect", bufferWritten, - fileReadFromStore); + assertArrayEquals(bufferWritten, + fileReadFromStore, "Test file content incorrect"); } private void assertOpStats(Map metricMap, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java index 38d4d12b10f34..78a02373f70d1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -20,8 +20,7 @@ import java.io.BufferedReader; import java.io.InputStreamReader; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,6 +36,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; +import static org.junit.jupiter.api.Assumptions.assumeFalse; /** * Test compatibility between ABFS client and WASB client. @@ -50,7 +50,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { LoggerFactory.getLogger(ITestWasbAbfsCompatibility.class); public ITestWasbAbfsCompatibility() throws Exception { - Assume.assumeFalse("Emulator is not supported", isIPAddress()); + assumeFalse(isIPAddress(), "Emulator is not supported"); } @Test @@ -58,9 +58,9 @@ public void testListFileStatus() throws Exception { // crate file using abfs AzureBlobFileSystem fs = getFileSystem(); // test only valid for non-namespace enabled account - Assume.assumeFalse("Namespace enabled account does not support this test,", - getIsNamespaceEnabled(fs)); - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(getIsNamespaceEnabled(fs), + "Namespace enabled account does not support this test,"); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -95,9 +95,9 @@ public void testReadFile() throws Exception { AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account - Assume.assumeFalse("Namespace enabled account does not support this test", - getIsNamespaceEnabled(abfs)); - Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled()); + assumeFalse(getIsNamespaceEnabled(abfs), + "Namespace enabled account does not support this test"); + assumeFalse(isAppendBlobEnabled(), "Not valid for APPEND BLOB"); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -109,7 +109,7 @@ public void testReadFile() throws Exception { final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb; if (createFs == abfs && readFs == wasb) { //Since flush changes the md5Hash value, md5 returned by GetBlobProperties will not match the one returned by GetBlob. - Assume.assumeFalse(getIngressServiceType() == AbfsServiceType.BLOB); + assumeFalse(getIngressServiceType() == AbfsServiceType.BLOB); } // Write @@ -124,8 +124,7 @@ public void testReadFile() throws Exception { try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) { String line = br.readLine(); - assertEquals("Wrong text from " + readFs, - TEST_CONTEXT, line); + assertEquals(TEST_CONTEXT, line, "Wrong text from " + readFs); } // Remove file @@ -140,8 +139,8 @@ public void testDir() throws Exception { AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account - Assume.assumeFalse("Namespace enabled account does not support this test", - getIsNamespaceEnabled(abfs)); + assumeFalse(getIsNamespaceEnabled(abfs), + "Namespace enabled account does not support this test"); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -176,8 +175,8 @@ public void testSetWorkingDirectory() throws Exception { //create folders AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account - Assume.assumeFalse("Namespace enabled account does not support this test", - getIsNamespaceEnabled(abfs)); + assumeFalse(getIsNamespaceEnabled(abfs), + "Namespace enabled account does not support this test"); NativeAzureFileSystem wasb = getWasbFileSystem(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java index 0b7645bd243ba..c670b16ad92ba 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java @@ -35,11 +35,12 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Test ConfigurationServiceFieldsValidation. @@ -108,15 +109,15 @@ public void testValidateFunctionsInConfigServiceImpl() throws Exception { for (Field field : fields) { field.setAccessible(true); if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) { - Assertions.assertThat(abfsConfiguration.validateInt(field)).isEqualTo(TEST_INT); + assertThat(abfsConfiguration.validateInt(field)).isEqualTo(TEST_INT); } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) { - Assertions.assertThat(abfsConfiguration.validateLong(field)).isEqualTo(DEFAULT_LONG); + assertThat(abfsConfiguration.validateLong(field)).isEqualTo(DEFAULT_LONG); } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) { - Assertions.assertThat(abfsConfiguration.validateString(field)).isEqualTo("stringValue"); + assertThat(abfsConfiguration.validateString(field)).isEqualTo("stringValue"); } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) { - Assertions.assertThat(abfsConfiguration.validateBase64String(field)).isEqualTo(this.encodedString); + assertThat(abfsConfiguration.validateBase64String(field)).isEqualTo(this.encodedString); } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) { - Assertions.assertThat(abfsConfiguration.validateBoolean(field)).isEqualTo(true); + assertThat(abfsConfiguration.validateBoolean(field)).isEqualTo(true); } } } @@ -124,37 +125,37 @@ public void testValidateFunctionsInConfigServiceImpl() throws Exception { @Test public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception { // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor - Assertions.assertThat(abfsConfiguration.getWriteBufferSize()) + assertThat(abfsConfiguration.getWriteBufferSize()) .describedAs("Default value of write buffer size should be initialized") .isEqualTo(DEFAULT_WRITE_BUFFER_SIZE); - Assertions.assertThat(abfsConfiguration.getReadBufferSize()) + assertThat(abfsConfiguration.getReadBufferSize()) .describedAs("Default value of read buffer size should be initialized") .isEqualTo(DEFAULT_READ_BUFFER_SIZE); - Assertions.assertThat(abfsConfiguration.getMinBackoffIntervalMilliseconds()) + assertThat(abfsConfiguration.getMinBackoffIntervalMilliseconds()) .describedAs("Default value of min backoff interval should be initialized") .isEqualTo(DEFAULT_MIN_BACKOFF_INTERVAL); - Assertions.assertThat(abfsConfiguration.getMaxBackoffIntervalMilliseconds()) + assertThat(abfsConfiguration.getMaxBackoffIntervalMilliseconds()) .describedAs("Default value of max backoff interval should be initialized") .isEqualTo(DEFAULT_MAX_BACKOFF_INTERVAL); - Assertions.assertThat(abfsConfiguration.getBackoffIntervalMilliseconds()) + assertThat(abfsConfiguration.getBackoffIntervalMilliseconds()) .describedAs("Default value of backoff interval should be initialized") .isEqualTo(DEFAULT_BACKOFF_INTERVAL); - Assertions.assertThat(abfsConfiguration.getMaxIoRetries()) + assertThat(abfsConfiguration.getMaxIoRetries()) .describedAs("Default value of max number of retries should be initialized") .isEqualTo(DEFAULT_MAX_RETRY_ATTEMPTS); - Assertions.assertThat(abfsConfiguration.getAzureBlockSize()) + assertThat(abfsConfiguration.getAzureBlockSize()) .describedAs("Default value of azure block size should be initialized") .isEqualTo(MAX_AZURE_BLOCK_SIZE); - Assertions.assertThat(abfsConfiguration.getAzureBlockLocationHost()) + assertThat(abfsConfiguration.getAzureBlockLocationHost()) .describedAs("Default value of azure block location host should be initialized") .isEqualTo(AZURE_BLOCK_LOCATION_HOST_DEFAULT); - Assertions.assertThat(abfsConfiguration.getReadAheadRange()) + assertThat(abfsConfiguration.getReadAheadRange()) .describedAs("Default value of read ahead range should be initialized") .isEqualTo(DEFAULT_READ_AHEAD_RANGE); - Assertions.assertThat(abfsConfiguration.getHttpConnectionTimeout()) + assertThat(abfsConfiguration.getHttpConnectionTimeout()) .describedAs("Default value of http connection timeout should be initialized") .isEqualTo(DEFAULT_HTTP_CONNECTION_TIMEOUT); - Assertions.assertThat(abfsConfiguration.getHttpReadTimeout()) + assertThat(abfsConfiguration.getHttpReadTimeout()) .describedAs("Default value of http read timeout should be initialized") .isEqualTo(DEFAULT_HTTP_READ_TIMEOUT); } @@ -162,7 +163,7 @@ public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception { @Test public void testConfigBlockSizeInitialized() throws Exception { // test the block size annotated field has been initialized in the constructor - Assertions.assertThat(abfsConfiguration.getAzureBlockSize()) + assertThat(abfsConfiguration.getAzureBlockSize()) .describedAs("Default value of max azure block size should be initialized") .isEqualTo(MAX_AZURE_BLOCK_SIZE); } @@ -170,42 +171,44 @@ public void testConfigBlockSizeInitialized() throws Exception { @Test public void testGetAccountKey() throws Exception { String accountKey = abfsConfiguration.getStorageAccountKey(); - Assertions.assertThat(accountKey).describedAs("Account Key should be initialized in configs") + assertThat(accountKey).describedAs("Account Key should be initialized in configs") .isEqualTo(this.encodedAccountKey); } - @Test(expected = KeyProviderException.class) + @Test public void testGetAccountKeyWithNonExistingAccountName() throws Exception { - Configuration configuration = new Configuration(); - configuration.addResource(TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME); - configuration.unset(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME); - AbfsConfiguration abfsConfig = new AbfsConfiguration(configuration, "bogusAccountName"); - abfsConfig.getStorageAccountKey(); + assertThrows(KeyProviderException.class, () -> { + Configuration configuration = new Configuration(); + configuration.addResource(TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME); + configuration.unset(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME); + AbfsConfiguration abfsConfig = new AbfsConfiguration(configuration, "bogusAccountName"); + abfsConfig.getStorageAccountKey(); + }); } @Test public void testSSLSocketFactoryConfiguration() throws InvalidConfigurationValueException, IllegalAccessException, IOException { - Assertions.assertThat(abfsConfiguration.getPreferredSSLFactoryOption()) + assertThat(abfsConfiguration.getPreferredSSLFactoryOption()) .describedAs("By default SSL Channel Mode should be Default") .isEqualTo(DelegatingSSLSocketFactory.SSLChannelMode.Default); - Assertions.assertThat(abfsConfiguration.getPreferredSSLFactoryOption()) + assertThat(abfsConfiguration.getPreferredSSLFactoryOption()) .describedAs("By default SSL Channel Mode should be Default") .isNotEqualTo(DelegatingSSLSocketFactory.SSLChannelMode.Default_JSSE); - Assertions.assertThat(abfsConfiguration.getPreferredSSLFactoryOption()) + assertThat(abfsConfiguration.getPreferredSSLFactoryOption()) .describedAs("By default SSL Channel Mode should be Default") .isNotEqualTo(DelegatingSSLSocketFactory.SSLChannelMode.OpenSSL); Configuration configuration = new Configuration(); configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DelegatingSSLSocketFactory.SSLChannelMode.Default_JSSE); AbfsConfiguration localAbfsConfiguration = new AbfsConfiguration(configuration, accountName); - Assertions.assertThat(localAbfsConfiguration.getPreferredSSLFactoryOption()) + assertThat(localAbfsConfiguration.getPreferredSSLFactoryOption()) .describedAs("SSL Channel Mode should be Default_JSSE as set") .isEqualTo(DelegatingSSLSocketFactory.SSLChannelMode.Default_JSSE); configuration = new Configuration(); configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DelegatingSSLSocketFactory.SSLChannelMode.OpenSSL); localAbfsConfiguration = new AbfsConfiguration(configuration, accountName); - Assertions.assertThat(localAbfsConfiguration.getPreferredSSLFactoryOption()) + assertThat(localAbfsConfiguration.getPreferredSSLFactoryOption()) .describedAs("SSL Channel Mode should be OpenSSL as set") .isEqualTo(DelegatingSSLSocketFactory.SSLChannelMode.OpenSSL); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsCrc64.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsCrc64.java index ab39750ebf9c9..ca45a54da00a5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsCrc64.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsCrc64.java @@ -17,10 +17,12 @@ */ package org.apache.hadoop.fs.azurebfs; -import org.junit.Assert; -import org.junit.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.azurebfs.utils.CRC64; + /** * Test for Crc64 in AzureBlobFileSystem, notice that ABFS CRC64 has its own polynomial. * */ @@ -32,7 +34,7 @@ public void tesCrc64Compute() { final String[] testStr = {"#$", "dir_2_ac83abee", "dir_42_976df1f5"}; final String[] expected = {"f91f7e6a837dbfa8", "203f9fefc38ae97b", "cc0d56eafe58a855"}; for (int i = 0; i < testStr.length; i++) { - Assert.assertEquals(expected[i], Long.toHexString(crc64.compute(testStr[i].getBytes()))); + assertEquals(expected[i], Long.toHexString(crc64.compute(testStr[i].getBytes()))); } } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsErrorTranslation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsErrorTranslation.java index 2c14b7af2821a..ff7e5640a4f81 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsErrorTranslation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsErrorTranslation.java @@ -22,7 +22,7 @@ import java.net.HttpURLConnection; import java.nio.file.AccessDeniedException; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.Path; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsInputStreamStatistics.java index 22c247f98af63..dd1b59c37a1e9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsInputStreamStatistics.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStreamStatisticsImpl; @@ -48,8 +48,8 @@ public void testBytesReadFromBufferStatistic() { * Since we incremented the bytesReadFromBuffer OPERATIONS times, this * should be the expected value. */ - assertEquals("Mismatch in bytesReadFromBuffer value", OPERATIONS, - abfsInputStreamStatistics.getBytesReadFromBuffer()); + assertEquals(OPERATIONS, abfsInputStreamStatistics.getBytesReadFromBuffer(), + "Mismatch in bytesReadFromBuffer value"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsNetworkStatistics.java index 628ad30863c9a..3273758e42a1e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsNetworkStatistics.java @@ -21,8 +21,7 @@ import java.io.IOException; import java.util.Map; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,6 +35,7 @@ import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.HTTP_POST_REQUEST; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic; +import static org.assertj.core.api.Assertions.assertThat; public class TestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest { @@ -109,10 +109,10 @@ public void testAbfsNetworkDurationTrackers() IOStatistics ioStatistics = extractStatistics(abfsCounters); // Asserting that the durationTrackers have mean > 0.0. for (AbfsStatistic abfsStatistic : HTTP_DURATION_TRACKER_LIST) { - Assertions.assertThat(lookupMeanStatistic(ioStatistics, + assertThat(lookupMeanStatistic(ioStatistics, abfsStatistic.getStatName() + StoreStatisticNames.SUFFIX_MEAN).mean()) .describedAs("The DurationTracker Named " + abfsStatistic.getStatName() - + " Doesn't match the expected value") + + " Doesn't match the expected value") .isGreaterThan(0.0); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java index 5f9404302bd2c..389c02ac7f291 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java @@ -20,7 +20,7 @@ import java.util.Random; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl; @@ -50,15 +50,17 @@ public void testAbfsOutputStreamBytesFailed() { new AbfsOutputStreamStatisticsImpl(); //Test for zero bytes uploaded. - assertEquals("Mismatch in number of bytes failed to upload", 0, - abfsOutputStreamStatistics.getBytesUploadFailed()); + assertEquals(0, + abfsOutputStreamStatistics.getBytesUploadFailed(), + "Mismatch in number of bytes failed to upload"); //Populating small random value for bytesFailed. int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE); abfsOutputStreamStatistics.uploadFailed(randomBytesFailed); //Test for bytes failed to upload. - assertEquals("Mismatch in number of bytes failed to upload", - randomBytesFailed, abfsOutputStreamStatistics.getBytesUploadFailed()); + assertEquals(randomBytesFailed, + abfsOutputStreamStatistics.getBytesUploadFailed(), + "Mismatch in number of bytes failed to upload"); //Reset statistics for the next test. abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl(); @@ -74,8 +76,9 @@ public void testAbfsOutputStreamBytesFailed() { expectedBytesFailed += randomBytesFailed; } //Test for bytes failed to upload. - assertEquals("Mismatch in number of bytes failed to upload", - expectedBytesFailed, abfsOutputStreamStatistics.getBytesUploadFailed()); + assertEquals(expectedBytesFailed, + abfsOutputStreamStatistics.getBytesUploadFailed(), + "Mismatch in number of bytes failed to upload"); } /** @@ -91,14 +94,16 @@ public void testAbfsOutputStreamTimeSpentOnWaitTask() { new AbfsOutputStreamStatisticsImpl(); //Test for initial value of timeSpentWaitTask. - assertEquals("Mismatch in time spent on waiting for tasks to complete", 0, - abfsOutputStreamStatistics.getTimeSpentOnTaskWait()); + assertEquals(0, + abfsOutputStreamStatistics.getTimeSpentOnTaskWait(), + "Mismatch in time spent on waiting for tasks to complete"); abfsOutputStreamStatistics .timeSpentTaskWait(); //Test for one op call value of timeSpentWaitTask. - assertEquals("Mismatch in time spent on waiting for tasks to complete", - 1, abfsOutputStreamStatistics.getTimeSpentOnTaskWait()); + assertEquals(1, + abfsOutputStreamStatistics.getTimeSpentOnTaskWait(), + "Mismatch in time spent on waiting for tasks to complete"); //Reset statistics for the next test. abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl(); @@ -115,9 +120,9 @@ public void testAbfsOutputStreamTimeSpentOnWaitTask() { * Test to check correct value of timeSpentTaskWait after OPERATIONS * number of op calls. */ - assertEquals("Mismatch in time spent on waiting for tasks to complete", - OPERATIONS, - abfsOutputStreamStatistics.getTimeSpentOnTaskWait()); + assertEquals(OPERATIONS, + abfsOutputStreamStatistics.getTimeSpentOnTaskWait(), + "Mismatch in time spent on waiting for tasks to complete"); } /** @@ -133,14 +138,16 @@ public void testAbfsOutputStreamQueueShrink() { new AbfsOutputStreamStatisticsImpl(); //Test for shrinking queue zero time. - assertEquals("Mismatch in queue shrunk operations", 0, - abfsOutputStreamStatistics.getQueueShrunkOps()); + assertEquals(0, + abfsOutputStreamStatistics.getQueueShrunkOps(), + "Mismatch in queue shrunk operations"); abfsOutputStreamStatistics.queueShrunk(); //Test for shrinking queue 1 time. - assertEquals("Mismatch in queue shrunk operations", 1, - abfsOutputStreamStatistics.getQueueShrunkOps()); + assertEquals(1, + abfsOutputStreamStatistics.getQueueShrunkOps(), + "Mismatch in queue shrunk operations"); //Reset statistics for the next test. abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl(); @@ -156,8 +163,8 @@ public void testAbfsOutputStreamQueueShrink() { /* * Test for random times incrementing queue shrunk operations. */ - assertEquals("Mismatch in queue shrunk operations", - randomQueueValues * OPERATIONS, - abfsOutputStreamStatistics.getQueueShrunkOps()); + assertEquals(randomQueueValues * OPERATIONS, + abfsOutputStreamStatistics.getQueueShrunkOps(), + "Mismatch in queue shrunk operations"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsStatistics.java index f831d2d4cd26b..e559437a1bb3c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsStatistics.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.util.Map; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java index 483a7e3d5d58e..f98ebb32a6c2a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java @@ -38,8 +38,7 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT; @@ -51,8 +50,9 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests correct precedence of various configurations that might be returned. @@ -126,24 +126,24 @@ public void testStringPrecedence() conf.set(globalKey, globalValue); abfsConf = new AbfsConfiguration(conf, accountName1); - assertEquals("Wrong value returned when account-specific value was requested", - abfsConf.get(accountKey1), accountValue1); - assertEquals("Account-specific value was not returned when one existed", - abfsConf.get(globalKey), accountValue1); + assertEquals(abfsConf.get(accountKey1), accountValue1, + "Wrong value returned when account-specific value was requested"); + assertEquals(abfsConf.get(globalKey), accountValue1, + "Account-specific value was not returned when one existed"); abfsConf = new AbfsConfiguration(conf, accountName2); - assertEquals("Wrong value returned when a different account-specific value was requested", - abfsConf.get(accountKey1), accountValue1); - assertEquals("Wrong value returned when account-specific value was requested", - abfsConf.get(accountKey2), accountValue2); - assertEquals("Account-agnostic value return even though account-specific value was set", - abfsConf.get(globalKey), accountValue2); + assertEquals(abfsConf.get(accountKey1), accountValue1, + "Wrong value returned when a different account-specific value was requested"); + assertEquals(abfsConf.get(accountKey2), accountValue2, + "Wrong value returned when account-specific value was requested"); + assertEquals(abfsConf.get(globalKey), accountValue2, + "Account-agnostic value return even though account-specific value was set"); abfsConf = new AbfsConfiguration(conf, accountName3); - assertNull("Account-specific value returned when none was set", - abfsConf.get(accountKey3)); - assertEquals("Account-agnostic value not returned when no account-specific value was set", - abfsConf.get(globalKey), globalValue); + assertNull(abfsConf.get(accountKey3), + "Account-specific value returned when none was set"); + assertEquals(abfsConf.get(globalKey), globalValue, + "Account-agnostic value not returned when no account-specific value was set"); } @Test @@ -170,24 +170,24 @@ public void testPasswordPrecedence() conf.set(globalKey, globalValue); abfsConf = new AbfsConfiguration(conf, accountName1); - assertEquals("Wrong value returned when account-specific value was requested", - abfsConf.getPasswordString(accountKey1), accountValue1); - assertEquals("Account-specific value was not returned when one existed", - abfsConf.getPasswordString(globalKey), accountValue1); + assertEquals(abfsConf.getPasswordString(accountKey1), accountValue1, + "Wrong value returned when account-specific value was requested"); + assertEquals(abfsConf.getPasswordString(globalKey), accountValue1, + "Account-specific value was not returned when one existed"); abfsConf = new AbfsConfiguration(conf, accountName2); - assertEquals("Wrong value returned when a different account-specific value was requested", - abfsConf.getPasswordString(accountKey1), accountValue1); - assertEquals("Wrong value returned when account-specific value was requested", - abfsConf.getPasswordString(accountKey2), accountValue2); - assertEquals("Account-agnostic value return even though account-specific value was set", - abfsConf.getPasswordString(globalKey), accountValue2); + assertEquals(abfsConf.getPasswordString(accountKey1), accountValue1, + "Wrong value returned when a different account-specific value was requested"); + assertEquals(abfsConf.getPasswordString(accountKey2), accountValue2, + "Wrong value returned when account-specific value was requested"); + assertEquals(abfsConf.getPasswordString(globalKey), accountValue2, + "Account-agnostic value return even though account-specific value was set"); abfsConf = new AbfsConfiguration(conf, accountName3); - assertNull("Account-specific value returned when none was set", - abfsConf.getPasswordString(accountKey3)); - assertEquals("Account-agnostic value not returned when no account-specific value was set", - abfsConf.getPasswordString(globalKey), globalValue); + assertNull(abfsConf.getPasswordString(accountKey3), + "Account-specific value returned when none was set"); + assertEquals(abfsConf.getPasswordString(globalKey), globalValue, + "Account-agnostic value not returned when no account-specific value was set"); } @Test @@ -202,23 +202,23 @@ public void testBooleanPrecedence() final AbfsConfiguration abfsConf = new AbfsConfiguration(conf, accountName); conf.setBoolean(globalKey, false); - assertEquals("Default value returned even though account-agnostic config was set", - abfsConf.getBoolean(globalKey, true), false); + assertEquals(abfsConf.getBoolean(globalKey, true), false, + "Default value returned even though account-agnostic config was set"); conf.unset(globalKey); - assertEquals("Default value not returned even though config was unset", - abfsConf.getBoolean(globalKey, true), true); + assertEquals(abfsConf.getBoolean(globalKey, true), true, + "Default value not returned even though config was unset"); conf.setBoolean(accountKey, false); - assertEquals("Default value returned even though account-specific config was set", - abfsConf.getBoolean(globalKey, true), false); + assertEquals(abfsConf.getBoolean(globalKey, true), false, + "Default value returned even though account-specific config was set"); conf.unset(accountKey); - assertEquals("Default value not returned even though config was unset", - abfsConf.getBoolean(globalKey, true), true); + assertEquals(abfsConf.getBoolean(globalKey, true), true, + "Default value not returned even though config was unset"); conf.setBoolean(accountKey, true); conf.setBoolean(globalKey, false); - assertEquals("Account-agnostic or default value returned even though account-specific config was set", - abfsConf.getBoolean(globalKey, false), true); + assertEquals(abfsConf.getBoolean(globalKey, false), true, + "Account-agnostic or default value returned even though account-specific config was set"); } @Test @@ -233,23 +233,23 @@ public void testLongPrecedence() final AbfsConfiguration abfsConf = new AbfsConfiguration(conf, accountName); conf.setLong(globalKey, 0); - assertEquals("Default value returned even though account-agnostic config was set", - abfsConf.getLong(globalKey, 1), 0); + assertEquals(abfsConf.getLong(globalKey, 1), 0, + "Default value returned even though account-agnostic config was set"); conf.unset(globalKey); - assertEquals("Default value not returned even though config was unset", - abfsConf.getLong(globalKey, 1), 1); + assertEquals(abfsConf.getLong(globalKey, 1), 1, + "Default value not returned even though config was unset"); conf.setLong(accountKey, 0); - assertEquals("Default value returned even though account-specific config was set", - abfsConf.getLong(globalKey, 1), 0); + assertEquals(abfsConf.getLong(globalKey, 1), 0, + "Default value returned even though account-specific config was set"); conf.unset(accountKey); - assertEquals("Default value not returned even though config was unset", - abfsConf.getLong(globalKey, 1), 1); + assertEquals(abfsConf.getLong(globalKey, 1), 1, + "Default value not returned even though config was unset"); conf.setLong(accountKey, 1); conf.setLong(globalKey, 0); - assertEquals("Account-agnostic or default value returned even though account-specific config was set", - abfsConf.getLong(globalKey, 0), 1); + assertEquals(abfsConf.getLong(globalKey, 0), 1, + "Account-agnostic or default value returned even though account-specific config was set"); } /** @@ -271,23 +271,24 @@ public void testEnumPrecedence() final AbfsConfiguration abfsConf = new AbfsConfiguration(conf, accountName); conf.setEnum(globalKey, GetEnumType.FALSE); - assertEquals("Default value returned even though account-agnostic config was set", - abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.FALSE); + assertEquals(abfsConf.getEnum(globalKey, GetEnumType.TRUE), + GetEnumType.FALSE, + "Default value returned even though account-agnostic config was set"); conf.unset(globalKey); - assertEquals("Default value not returned even though config was unset", - abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.TRUE); + assertEquals(abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.TRUE, + "Default value not returned even though config was unset"); conf.setEnum(accountKey, GetEnumType.FALSE); - assertEquals("Default value returned even though account-specific config was set", - abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.FALSE); + assertEquals(abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.FALSE, + "Default value returned even though account-specific config was set"); conf.unset(accountKey); - assertEquals("Default value not returned even though config was unset", - abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.TRUE); + assertEquals(abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.TRUE, + "Default value not returned even though config was unset"); conf.setEnum(accountKey, GetEnumType.TRUE); conf.setEnum(globalKey, GetEnumType.FALSE); - assertEquals("Account-agnostic or default value returned even though account-specific config was set", - abfsConf.getEnum(globalKey, GetEnumType.FALSE), GetEnumType.TRUE); + assertEquals(abfsConf.getEnum(globalKey, GetEnumType.FALSE), GetEnumType.TRUE, + "Account-agnostic or default value returned even though account-specific config was set"); } /** @@ -324,23 +325,24 @@ public void testClass() final Class xface = GetClassInterface.class; conf.setClass(globalKey, class0, xface); - assertEquals("Default value returned even though account-agnostic config was set", - abfsConf.getAccountAgnosticClass(globalKey, class1, xface), class0); + assertEquals(abfsConf.getAccountAgnosticClass(globalKey, class1, xface), class0, + "Default value returned even though account-agnostic config was set"); conf.unset(globalKey); - assertEquals("Default value not returned even though config was unset", - abfsConf.getAccountAgnosticClass(globalKey, class1, xface), class1); + assertEquals(abfsConf.getAccountAgnosticClass(globalKey, class1, xface), class1, + "Default value not returned even though config was unset"); conf.setClass(accountKey, class0, xface); - assertEquals("Default value returned even though account-specific config was set", - abfsConf.getAccountSpecificClass(globalKey, class1, xface), class0); + assertEquals(abfsConf.getAccountSpecificClass(globalKey, class1, xface), class0, + "Default value returned even though account-specific config was set"); conf.unset(accountKey); - assertEquals("Default value not returned even though config was unset", - abfsConf.getAccountSpecificClass(globalKey, class1, xface), class1); + assertEquals(abfsConf.getAccountSpecificClass(globalKey, class1, xface), class1, + "Default value not returned even though config was unset"); conf.setClass(accountKey, class1, xface); conf.setClass(globalKey, class0, xface); - assertEquals("Account-agnostic or default value returned even though account-specific config was set", - abfsConf.getAccountSpecificClass(globalKey, class0, xface), class1); + assertEquals(abfsConf.getAccountSpecificClass(globalKey, class0, xface), + class1, + "Account-agnostic or default value returned even though account-specific config was set"); } @Test @@ -363,7 +365,7 @@ public void testSASProviderPrecedence() abfsConf.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_SAS_PROVIDER_CLASS_CONFIG_2); - Assertions.assertThat( + assertThat( abfsConf.getSASTokenProvider().getClass().getName()) .describedAs( "Account-specific SAS token provider should be in effect.") @@ -460,7 +462,7 @@ public void testClientAndTenantIdOptionalWhenUsingMsiTokenProvider() throws Thro // Test that we managed to instantiate an MsiTokenProvider without having to define the tenant and client ID. // Those 2 fields are optional as they can automatically be determined by the Azure Metadata service when // running on an Azure VM. - Assertions.assertThat(tokenProviderTypeName).describedAs("Token Provider Should be MsiTokenProvider").isInstanceOf(MsiTokenProvider.class); + assertThat(tokenProviderTypeName).describedAs("Token Provider Should be MsiTokenProvider").isInstanceOf(MsiTokenProvider.class); } public void testGlobalAndAccountOAuthPrecedence(AbfsConfiguration abfsConf, @@ -492,7 +494,7 @@ public void testGlobalAndAccountOAuthPrecedence(AbfsConfiguration abfsConf, ? ClientCredsTokenProvider.class : CustomTokenProviderAdapter.class; - Assertions.assertThat( + assertThat( abfsConf.getTokenProvider().getClass().getTypeName()) .describedAs( "Account-specific settings takes precendence to global" diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 7fb672920cd67..43aae5c80ac9a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -27,11 +27,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.assertj.core.api.Assertions; -import org.junit.Assume; import org.junit.AssumptionViolatedException; -import org.junit.Ignore; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.Path; @@ -57,6 +55,10 @@ import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.STATIC_RETRY_POLICY_ABBREVIATION; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_ABBREVIATION; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; public class TestTracingContext extends AbstractAbfsIntegrationTest { private static final String[] CLIENT_CORRELATIONID_LIST = { @@ -128,7 +130,7 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, } } - @Ignore + @Disabled @Test //call test methods from the respective test classes //can be ignored when running all tests as these get covered @@ -137,16 +139,31 @@ public void runCorrelationTestForAllMethods() throws Exception { testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); - testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true, JDK_HTTP_URL_CONNECTION), //open, + ITestAbfsReadWriteAndSeek iTestAbfsReadWriteAndSeek = new ITestAbfsReadWriteAndSeek(); + iTestAbfsReadWriteAndSeek.initITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, + true, JDK_HTTP_URL_CONNECTION); + testClasses.put(iTestAbfsReadWriteAndSeek, //open, // read, write ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); - testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true, APACHE_HTTP_CLIENT), //open, + + ITestAbfsReadWriteAndSeek iTestAbfsReadWriteAndSeek2 = new ITestAbfsReadWriteAndSeek(); + iTestAbfsReadWriteAndSeek2.initITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, + true, APACHE_HTTP_CLIENT); + testClasses.put(iTestAbfsReadWriteAndSeek2, //open, // read, write ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); - testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false, JDK_HTTP_URL_CONNECTION), //read (bypassreadahead) + + ITestAbfsReadWriteAndSeek iTestAbfsReadWriteAndSeek3 = new ITestAbfsReadWriteAndSeek(); + iTestAbfsReadWriteAndSeek3.initITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false, + JDK_HTTP_URL_CONNECTION); + testClasses.put(iTestAbfsReadWriteAndSeek3, //read (bypassreadahead) ITestAbfsReadWriteAndSeek.class .getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); - testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false, APACHE_HTTP_CLIENT), //read (bypassreadahead) + + ITestAbfsReadWriteAndSeek iTestAbfsReadWriteAndSeek4 = new ITestAbfsReadWriteAndSeek(); + iTestAbfsReadWriteAndSeek4.initITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false, + APACHE_HTTP_CLIENT); + testClasses.put(iTestAbfsReadWriteAndSeek4, //read (bypassreadahead) ITestAbfsReadWriteAndSeek.class .getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); testClasses.put(new ITestAzureBlobFileSystemAppend(), //append @@ -203,9 +220,9 @@ public void testExternalOps() throws Exception { fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); - Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); - Assume.assumeTrue(getConfiguration().isCheckAccessEnabled()); - Assume.assumeTrue(getAuthType() == AuthType.OAuth); + assumeTrue(getIsNamespaceEnabled(getFileSystem())); + assumeTrue(getConfiguration().isCheckAccessEnabled()); + assumeTrue(getAuthType() == AuthType.OAuth); fs.setListenerOperation(FSOperationType.ACCESS); fs.getAbfsStore().setNamespaceEnabled(Trilean.TRUE); @@ -223,8 +240,8 @@ fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, new Tracin fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.CREATE_FILESYSTEM, false, 0)); - AbfsHttpOperation abfsHttpOperation = Mockito.mock(AbfsHttpOperation.class); - Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito.anyString(), Mockito.anyString()); + AbfsHttpOperation abfsHttpOperation = mock(AbfsHttpOperation.class); + doNothing().when(abfsHttpOperation).setRequestProperty(anyString(), anyString()); tracingContext.constructHeader(abfsHttpOperation, null, EXPONENTIAL_RETRY_POLICY_ABBREVIATION); String header = tracingContext.getHeader(); String clientRequestIdUsed = header.split(":")[1]; @@ -260,8 +277,8 @@ fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, new Tracin fs.getFileSystemId(), FSOperationType.CREATE_FILESYSTEM, false, 0)); tracingContext.setPrimaryRequestID(); - AbfsHttpOperation abfsHttpOperation = Mockito.mock(AbfsHttpOperation.class); - Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito.anyString(), Mockito.anyString()); + AbfsHttpOperation abfsHttpOperation = mock(AbfsHttpOperation.class); + doNothing().when(abfsHttpOperation).setRequestProperty(anyString(), anyString()); tracingContext.constructHeader(abfsHttpOperation, null, EXPONENTIAL_RETRY_POLICY_ABBREVIATION); String header = tracingContext.getHeader(); String assertionPrimaryId = header.split(":")[3]; @@ -294,8 +311,8 @@ fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, new Tracin fs.getFileSystemId(), FSOperationType.CREATE_FILESYSTEM, false, 0)); tracingContext.setPrimaryRequestID(); - AbfsHttpOperation abfsHttpOperation = Mockito.mock(AbfsHttpOperation.class); - Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito.anyString(), Mockito.anyString()); + AbfsHttpOperation abfsHttpOperation = mock(AbfsHttpOperation.class); + doNothing().when(abfsHttpOperation).setRequestProperty(anyString(), anyString()); tracingContext.constructHeader(abfsHttpOperation, null, null); checkHeaderForRetryPolicyAbbreviation(tracingContext.getHeader(), null, null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TrileanTests.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TrileanTests.java index 45467d4140132..8f835cb34d912 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TrileanTests.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TrileanTests.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException; import org.apache.hadoop.fs.azurebfs.enums.Trilean;