Explorar o código

HADOOP-18960: [ABFS] Making Contract tests run in sequential and Other Test Fixes (#7104)

Contributed by: Anuj Modi
Anuj Modi hai 6 meses
pai
achega
487727a5d1

+ 27 - 1
hadoop-tools/hadoop-azure/pom.xml

@@ -662,7 +662,6 @@
                     <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
                   </systemPropertyVariables>
                   <includes>
-                    <include>**/azurebfs/contract/ITest*.java</include>
                     <include>**/azurebfs/ITestAzureBlobFileSystemE2EScale.java</include>
                     <include>**/azurebfs/ITestAbfsReadWriteAndSeek.java</include>
                     <include>**/azurebfs/ITestAzureBlobFileSystemListStatus.java</include>
@@ -674,6 +673,33 @@
                   </includes>
                 </configuration>
               </execution>
+              <execution>
+                <id>integration-test-abfs-sequential-contract-classes</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+                <configuration>
+                  <!--NOTICE: hadoop contract tests methods can not be run in parallel-->
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
+                  <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed sequentially -->
+                    <test.parallel.execution>false</test.parallel.execution>
+                    <!-- Propagate scale parameters -->
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                    <fs.azure.scale.test.list.performance.threads>${fs.azure.scale.test.list.performance.threads}</fs.azure.scale.test.list.performance.threads>
+                    <fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
+                    <!-- http connection pool size -->
+                    <http.maxConnections>${http.maxConnections}</http.maxConnections>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/azurebfs/contract/ITest*.java</include>
+                  </includes>
+                </configuration>
+              </execution>
             </executions>
           </plugin>
         </plugins>

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

@@ -400,9 +400,11 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
       AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
           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(
           getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT,
-              false))
+              true))
           .describedAs("Encryption tests should run only on namespace enabled account")
           .isTrue();
       return fs;

+ 27 - 26
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java

@@ -30,11 +30,14 @@ import org.apache.hadoop.fs.azurebfs.oauth2.RetryTestTokenProvider;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DOT;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CUSTOM_TOKEN_FETCH_RETRY_COUNT;
 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_IS_HNS_ENABLED;
 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.accountProperty;
 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;
 
 /**
@@ -140,19 +143,8 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest
 
   public void testWithDifferentCustomTokenFetchRetry(int numOfRetries) throws Exception {
     AzureBlobFileSystem fs = this.getFileSystem();
-
-    Configuration config = new Configuration(this.getRawConfiguration());
-    String accountName = config.get("fs.azure.abfs.account.name");
-    // Setup to configure custom token provider.
-    config.set("fs.azure.account.auth.type." + accountName, "Custom");
-    config.set("fs.azure.account.oauth.provider.type." + accountName, "org.apache.hadoop.fs"
-        + ".azurebfs.oauth2.RetryTestTokenProvider");
-    config.set("fs.azure.custom.token.fetch.retry.count", Integer.toString(numOfRetries));
-    // Stop filesystem creation as it will lead to calls to store.
-    config.set("fs.azure.createRemoteFileSystemDuringInitialization", "false");
-
-    try (final AzureBlobFileSystem fs1 =
-        (AzureBlobFileSystem) FileSystem.newInstance(fs.getUri(),
+    Configuration config = getCustomAuthConfiguration(numOfRetries);
+    try (AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(fs.getUri(),
         config)) {
       RetryTestTokenProvider retryTestTokenProvider
               = RetryTestTokenProvider.getCurrentRetryTestProviderInstance(
@@ -174,18 +166,7 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest
 
   @Test
   public void testAuthFailException() throws Exception {
-    Configuration config = new Configuration(getRawConfiguration());
-    String accountName = config
-        .get(FS_AZURE_ABFS_ACCOUNT_NAME);
-    // Setup to configure custom token provider
-    config.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + DOT
-        + accountName, "Custom");
-    config.set(
-        FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + DOT + accountName,
-        RETRY_TEST_TOKEN_PROVIDER);
-    // Stop filesystem creation as it will lead to calls to store.
-    config.set(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, "false");
-
+    Configuration config = getCustomAuthConfiguration(0);
     final AzureBlobFileSystem fs = getFileSystem(config);
     AbfsRestOperationException e = intercept(AbfsRestOperationException.class, () -> {
       fs.getFileStatus(new Path("/"));
@@ -201,4 +182,24 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest
         .describedAs("Incorrect error message: " + errorDesc)
         .contains("Auth failure: ");
   }
+
+  /**
+   * Returns a configuration with a custom token provider configured. {@link RetryTestTokenProvider}
+   * @param numOfRetries Number of retries to be configured for token fetch.
+   * @return Configuration
+   */
+  private Configuration getCustomAuthConfiguration(final int numOfRetries) {
+    Configuration config = new Configuration(this.getRawConfiguration());
+    String accountName = config.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    // Setup to configure custom token provider.
+    config.set(accountProperty(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, accountName), "Custom");
+    config.set(accountProperty(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME, accountName),
+        RETRY_TEST_TOKEN_PROVIDER);
+    config.setInt(AZURE_CUSTOM_TOKEN_FETCH_RETRY_COUNT, numOfRetries);
+    // Stop filesystem creation as it will lead to calls to store.
+    config.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, false);
+    config.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, config.getBoolean(
+        FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, true));
+    return config;
+  }
 }

+ 24 - 16
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java

@@ -28,7 +28,6 @@ import org.junit.Test;
 import org.apache.hadoop.conf.Configuration;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException;
 import org.apache.hadoop.fs.azurebfs.extensions.MockSASTokenProvider;
@@ -36,6 +35,9 @@ import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 
+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_IS_HNS_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.MOCK_SASTOKENPROVIDER_FAIL_INIT;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN;
 import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
@@ -61,12 +63,12 @@ public class ITestAzureBlobFileSystemAuthorization extends AbstractAbfsIntegrati
 
   @Override
   public void setup() throws Exception {
-    boolean isHNSEnabled = this.getConfiguration().getBoolean(
+    boolean isHNSEnabled = getConfiguration().getBoolean(
         TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
     Assume.assumeTrue(isHNSEnabled);
     loadConfiguredFileSystem();
-    this.getConfiguration().set(ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_AUTHZ_CLASS);
-    this.getConfiguration().set(ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, "SAS");
+    getConfiguration().set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_AUTHZ_CLASS);
+    getConfiguration().set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SAS.toString());
     super.setup();
   }
 
@@ -75,13 +77,15 @@ public class ITestAzureBlobFileSystemAuthorization extends AbstractAbfsIntegrati
     final AzureBlobFileSystem fs = this.getFileSystem();
 
     final AzureBlobFileSystem testFs = new AzureBlobFileSystem();
-    Configuration testConfig = this.getConfiguration().getRawConfiguration();
-    testConfig.set(ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_ERR_AUTHZ_CLASS);
-    testConfig.set(MOCK_SASTOKENPROVIDER_FAIL_INIT, "true");
+    Configuration testConfig = new Configuration(this.getConfiguration().getRawConfiguration());
+    testConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_ERR_AUTHZ_CLASS);
+    testConfig.setBoolean(MOCK_SASTOKENPROVIDER_FAIL_INIT, true);
+    // Setting IS_HNS_ENABLED to avoid the exception thrown by the HNS check.
+    testConfig.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, this.getIsNamespaceEnabled(fs));
 
     intercept(SASTokenProviderException.class,
         ()-> {
-          testFs.initialize(fs.getUri(), this.getConfiguration().getRawConfiguration());
+          testFs.initialize(fs.getUri(), testConfig);
         });
   }
 
@@ -90,12 +94,13 @@ public class ITestAzureBlobFileSystemAuthorization extends AbstractAbfsIntegrati
     final AzureBlobFileSystem fs = this.getFileSystem();
 
     final AzureBlobFileSystem testFs = new AzureBlobFileSystem();
-    Configuration testConfig = this.getConfiguration().getRawConfiguration();
-    testConfig.set(ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_ERR_AUTHZ_CLASS);
-    testConfig.set(MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN, "true");
+    Configuration testConfig = new Configuration(this.getConfiguration().getRawConfiguration());
+    testConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_ERR_AUTHZ_CLASS);
+    testConfig.setBoolean(MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN, true);
+    // Setting IS_HNS_ENABLED to avoid the exception thrown by the HNS check.
+    testConfig.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, this.getIsNamespaceEnabled(fs));
 
-    testFs.initialize(fs.getUri(),
-        this.getConfiguration().getRawConfiguration());
+    testFs.initialize(fs.getUri(), testConfig);
     intercept(SASTokenProviderException.class,
         () -> {
           testFs.create(new org.apache.hadoop.fs.Path("/testFile")).close();
@@ -107,10 +112,13 @@ public class ITestAzureBlobFileSystemAuthorization extends AbstractAbfsIntegrati
     final AzureBlobFileSystem fs = this.getFileSystem();
 
     final AzureBlobFileSystem testFs = new AzureBlobFileSystem();
-    Configuration testConfig = this.getConfiguration().getRawConfiguration();
-    testConfig.set(ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_ERR_AUTHZ_CLASS);
+    Configuration testConfig = new Configuration(this.getConfiguration().getRawConfiguration());
+    testConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_ERR_AUTHZ_CLASS);
+    testConfig.setBoolean(MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN, true);
+    // Setting IS_HNS_ENABLED to avoid the exception thrown by the HNS check.
+    testConfig.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, this.getIsNamespaceEnabled(fs));
 
-    testFs.initialize(fs.getUri(), this.getConfiguration().getRawConfiguration());
+    testFs.initialize(fs.getUri(), testConfig);
     intercept(SASTokenProviderException.class,
         ()-> {
           testFs.create(new org.apache.hadoop.fs.Path("/testFile")).close();

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

@@ -37,6 +37,10 @@ import org.apache.hadoop.fs.azurebfs.utils.Base64;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_FIXED_TOKEN;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.accountProperty;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_APP_ID;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_APP_SECRET;
+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;
 
 /**
@@ -62,8 +66,8 @@ public class ITestAzureBlobFileSystemChooseSAS extends AbstractAbfsIntegrationTe
 
   @Override
   public void setup() throws Exception {
-    createFilesystemWithTestFileForSASTests(new Path(TEST_PATH));
     super.setup();
+    createFilesystemWithTestFileForSASTests(new Path(TEST_PATH));
     generateAccountSAS();
   }
 
@@ -95,6 +99,11 @@ public class ITestAzureBlobFileSystemChooseSAS extends AbstractAbfsIntegrationTe
     // Configuring a SASTokenProvider class which provides a user delegation SAS.
     testAbfsConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE,
         MockDelegationSASTokenProvider.class.getName());
+    // Make sure test configs required by MockDelegationSASTokenProvider are set.
+    assumeValidTestConfigPresent(this.getRawConfiguration(), FS_AZURE_TEST_APP_ID);
+    assumeValidTestConfigPresent(this.getRawConfiguration(), FS_AZURE_TEST_APP_SECRET);
+    assumeValidTestConfigPresent(this.getRawConfiguration(), FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID);
+    assumeValidTestConfigPresent(this.getRawConfiguration(), FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID);
 
     // configuring the Fixed SAS token which is an Account SAS.
     testAbfsConfig.set(FS_AZURE_SAS_FIXED_TOKEN, accountSAS);

+ 4 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java

@@ -44,6 +44,7 @@ import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
 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.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.doReturn;
@@ -286,12 +287,15 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
         this.getAccountName()));
     String testAccountName = "testAccount.dfs.core.windows.net";
     String otherAccountName = "otherAccount.dfs.core.windows.net";
+    String dummyAcountKey = "dummyKey";
     String defaultUri = this.getTestUrl().replace(this.getAccountName(), testAccountName);
     String otherUri = this.getTestUrl().replace(this.getAccountName(), otherAccountName);
 
     // Set both account specific and account agnostic config for test account
     rawConfig.set(accountProperty(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, testAccountName), FALSE_STR);
     rawConfig.set(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, TRUE_STR);
+    rawConfig.set(accountProperty(FS_AZURE_ACCOUNT_KEY, testAccountName), dummyAcountKey);
+    rawConfig.set(accountProperty(FS_AZURE_ACCOUNT_KEY, otherAccountName), dummyAcountKey);
     // Assert that account specific config takes precedence
     rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri);
     assertFileSystemInitWithExpectedHNSSettings(rawConfig, false);

+ 2 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_READ_AHEAD_RANGE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_READ_BUFFER_SIZE;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.utils.AbfsTestUtils.disableFilesystemCaching;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
@@ -68,6 +69,7 @@ public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{
   protected AbstractFSContract createContract(final Configuration conf) {
     conf.setInt(AZURE_READ_AHEAD_RANGE, MIN_BUFFER_SIZE);
     conf.setInt(AZURE_READ_BUFFER_SIZE, MIN_BUFFER_SIZE);
+    disableFilesystemCaching(conf);
     return new AbfsFileSystemContract(conf, isSecure);
   }
 

+ 11 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestExponentialRetryPolicy.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azurebfs.services;
 
 import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
 
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DOT;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_BACKOFF_INTERVAL;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES;
@@ -28,6 +29,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_AUTOTHROTTLING;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT1_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
 
@@ -65,7 +67,9 @@ public class ITestExponentialRetryPolicy extends AbstractAbfsIntegrationTest {
   private static final String TEST_PATH = "/testfile";
   private static final double MULTIPLYING_FACTOR = 1.5;
   private static final int ANALYSIS_PERIOD = 10000;
-
+  private static final String DUMMY_ACCOUNT_NAME = "dummy.dfs.core.windows.net";
+  private static final String DUMMY_ACCOUNT_NAME_1 = "dummy1.dfs.core.windows.net";
+  private static final String DUMMY_ACCOUNT_KEY = "dummyKey";
 
   public ITestExponentialRetryPolicy() throws Exception {
     super();
@@ -98,10 +102,14 @@ public class ITestExponentialRetryPolicy extends AbstractAbfsIntegrationTest {
     final Configuration configuration = new Configuration();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     configuration.setBoolean(FS_AZURE_ENABLE_AUTOTHROTTLING, false);
+    configuration.set(FS_AZURE_ACCOUNT_KEY + DOT + DUMMY_ACCOUNT_NAME,
+        DUMMY_ACCOUNT_KEY);
+    configuration.set(FS_AZURE_ACCOUNT_KEY + DOT + DUMMY_ACCOUNT_NAME_1,
+        DUMMY_ACCOUNT_KEY);
 
     // On disabling throttling AbfsNoOpThrottlingIntercept object is returned
     AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
-        "dummy.dfs.core.windows.net");
+        DUMMY_ACCOUNT_NAME);
     AbfsThrottlingIntercept intercept;
     AbfsClient abfsClient = ITestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration);
     intercept = abfsClient.getIntercept();
@@ -113,7 +121,7 @@ public class ITestExponentialRetryPolicy extends AbstractAbfsIntegrationTest {
     configuration.setBoolean(FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED, true);
     // On enabling throttling AbfsClientThrottlingIntercept object is returned
     AbfsConfiguration abfsConfiguration1 = new AbfsConfiguration(configuration,
-        "dummy1.dfs.core.windows.net");
+        DUMMY_ACCOUNT_NAME_1);
     AbfsClient abfsClient1 = ITestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration1);
     intercept = abfsClient1.getIntercept();
     Assertions.assertThat(intercept)

+ 15 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java

@@ -20,21 +20,23 @@ package org.apache.hadoop.fs.azurebfs.utils;
 import com.microsoft.azure.storage.CloudStorageAccount;
 import com.microsoft.azure.storage.blob.CloudBlobClient;
 import com.microsoft.azure.storage.blob.CloudBlobContainer;
-
 import org.junit.Assume;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_SCHEME;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_SECURE_SCHEME;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONTAINER_PREFIX;
 
 /**
  * Some Utils for ABFS tests.
  */
-public final class AbfsTestUtils extends AbstractAbfsIntegrationTest{
+public final class AbfsTestUtils extends AbstractAbfsIntegrationTest {
     private static final Logger LOG =
             LoggerFactory.getLogger(AbfsTestUtils.class);
 
@@ -82,4 +84,15 @@ public final class AbfsTestUtils extends AbstractAbfsIntegrationTest{
     }
     LOG.info("Deleted {} test containers", count);
   }
+
+    /**
+     * Turn off FS Caching: use if a filesystem with different options from
+     * the default is required.
+     * @param conf configuration to patch
+     */
+    public static void disableFilesystemCaching(Configuration conf) {
+        // Disabling cache to make sure new configs are picked up.
+        conf.setBoolean(String.format("fs.%s.impl.disable.cache", ABFS_SCHEME), true);
+        conf.setBoolean(String.format("fs.%s.impl.disable.cache", ABFS_SECURE_SCHEME), true);
+    }
 }