Browse Source

HADOOP-19129: [ABFS] Test Fixes and Test Script Bug Fixes (#6676)

Contributed by Anuj Modi
Anuj Modi 1 year ago
parent
commit
01c0a1346f
15 changed files with 294 additions and 105 deletions
  1. 48 10
      hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh
  2. 2 2
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
  3. 62 20
      hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
  4. 55 25
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
  5. 51 11
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java
  6. 2 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
  7. 8 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java
  8. 16 12
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
  9. 0 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
  10. 4 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java
  11. 3 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java
  12. 24 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbstractAbfsClusterITest.java
  13. 1 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
  14. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestExponentialRetryPolicy.java
  15. 17 12
      hadoop-tools/hadoop-azure/src/test/resources/accountSettings/accountName_settings.xml.template

+ 48 - 10
hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh

@@ -21,8 +21,20 @@ combtestfile=$resourceDir
 combtestfile+=abfs-combination-test-configs.xml
 combtestfile+=abfs-combination-test-configs.xml
 logdir=dev-support/testlogs/
 logdir=dev-support/testlogs/
 
 
-testresultsregex="Results:(\n|.)*?Tests run:"
+# Regex to filter out final test stats
+testresultsregex="Tests run: [0-9]+, Failures: [0-9]+, Errors: [0-9]+, Skipped: [0-9]+$"
+
+# Regex to filter out the test that failed due to unexpected output or error.
+failedTestRegex1="<<< FAILURE!$"
+
+# Regex to filter out the test that failed due to runtime exception.
+failedTestRegex2="<<< ERROR!$"
+
+# Regex to remove the formatting used by mvn output for better regex matching.
+removeFormattingRegex="s/\x1b\[[0-9;]*m//g"
 accountConfigFileSuffix="_settings.xml"
 accountConfigFileSuffix="_settings.xml"
+separatorbar1="============================================================"
+separatorbar2="------------------------------"
 testOutputLogFolder=$logdir
 testOutputLogFolder=$logdir
 testlogfilename=combinationTestLogFile
 testlogfilename=combinationTestLogFile
 
 
@@ -59,6 +71,16 @@ ENDOFFILE
     logOutput "Exiting. Number of properties and values differ for $combination"
     logOutput "Exiting. Number of properties and values differ for $combination"
     exit 1
     exit 1
   fi
   fi
+  echo "$separatorbar1"
+  echo "$combination"
+  echo "$separatorbar1"
+
+  # First include the account specific configurations.
+  xmlstarlet ed -P -L -s /configuration -t elem -n include -v "" $combtestfile
+  xmlstarlet ed -P -L -i /configuration/include -t attr -n href -v "$accountConfigFile" $combtestfile
+  xmlstarlet ed -P -L -i /configuration/include -t attr -n xmlns -v "http://www.w3.org/2001/XInclude" $combtestfile
+
+  # Override the combination specific configurations.
   for ((i = 0; i < propertiessize; i++)); do
   for ((i = 0; i < propertiessize; i++)); do
     key=${PROPERTIES[$i]}
     key=${PROPERTIES[$i]}
     val=${VALUES[$i]}
     val=${VALUES[$i]}
@@ -66,10 +88,6 @@ ENDOFFILE
     changeconf "$key" "$val"
     changeconf "$key" "$val"
   done
   done
   formatxml "$combtestfile"
   formatxml "$combtestfile"
-  xmlstarlet ed -P -L -s /configuration -t elem -n include -v "" $combtestfile
-  xmlstarlet ed -P -L -i /configuration/include -t attr -n href -v "$accountConfigFile" $combtestfile
-  xmlstarlet ed -P -L -i /configuration/include -t attr -n xmlns -v "http://www.w3.org/2001/XInclude" $combtestfile
-  formatxml $combtestfile
   echo ' '
   echo ' '
   echo "Activated [$combtestfile] - for account: $accountName for combination $combination"
   echo "Activated [$combtestfile] - for account: $accountName for combination $combination"
   testlogfilename="$testOutputLogFolder/Test-Logs-$combination.txt"
   testlogfilename="$testOutputLogFolder/Test-Logs-$combination.txt"
@@ -81,6 +99,8 @@ ENDOFFILE
     echo "Running test for combination $combination on account $accountName [ProcessCount=$processcount]"
     echo "Running test for combination $combination on account $accountName [ProcessCount=$processcount]"
     logOutput "Test run report can be seen in $testlogfilename"
     logOutput "Test run report can be seen in $testlogfilename"
     mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount="$processcount" verify >> "$testlogfilename" || true
     mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount="$processcount" verify >> "$testlogfilename" || true
+    # Remove the formatting used by mvn output for better regex matching.
+    sed -i "$removeFormattingRegex" "$testlogfilename"
     ENDTIME=$(date +%s)
     ENDTIME=$(date +%s)
     summary
     summary
   fi
   fi
@@ -102,19 +122,37 @@ ENDOFFILE
 summary() {
 summary() {
   {
   {
     echo ""
     echo ""
+    echo "$separatorbar1"
     echo "$combination"
     echo "$combination"
-    echo "========================"
-    pcregrep -M "$testresultsregex" "$testlogfilename"
+    echo "$separatorbar1"
+    summarycontent
   } >> "$aggregatedTestResult"
   } >> "$aggregatedTestResult"
   printf "\n----- Test results -----\n"
   printf "\n----- Test results -----\n"
-  pcregrep -M "$testresultsregex" "$testlogfilename"
+  summarycontent
   secondstaken=$((ENDTIME - STARTTIME))
   secondstaken=$((ENDTIME - STARTTIME))
   mins=$((secondstaken / 60))
   mins=$((secondstaken / 60))
   secs=$((secondstaken % 60))
   secs=$((secondstaken % 60))
   printf "\nTime taken: %s mins %s secs.\n" "$mins" "$secs"
   printf "\nTime taken: %s mins %s secs.\n" "$mins" "$secs"
-  echo "Find test result for the combination ($combination) in: $testlogfilename"
+  logOutput "For Error details refer to Test run report in: $testlogfilename"
   logOutput "Consolidated test result is saved in: $aggregatedTestResult"
   logOutput "Consolidated test result is saved in: $aggregatedTestResult"
-  echo "------------------------"
+  echo "$separatorbar2"
+}
+
+summarycontent() {
+  output=$(pcregrep -M "$failedTestRegex1" "$testlogfilename" || true)
+  if [ -n "$output" ]; then
+    echo "$output"
+  fi
+  output=$(pcregrep -M "$failedTestRegex2" "$testlogfilename" || true)
+  if [ -n "$output" ]; then
+    echo ""
+    echo "$output"
+  fi
+  output=$(pcregrep -M "$testresultsregex" "$testlogfilename" || true)
+  if [ -n "$output" ]; then
+    echo ""
+    echo "$output"
+  fi
 }
 }
 
 
 checkdependencies() {
 checkdependencies() {

+ 2 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java

@@ -593,7 +593,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
         "writeCurrentBufferToService", "append")) {
         "writeCurrentBufferToService", "append")) {
       AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0,
       AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0,
           bytesLength, APPEND_MODE, true, leaseId, isExpectHeaderEnabled);
           bytesLength, APPEND_MODE, true, leaseId, isExpectHeaderEnabled);
-      AbfsRestOperation op = client.append(path, uploadData.toByteArray(),
+      AbfsRestOperation op = getClient().append(path, uploadData.toByteArray(),
           reqParams, cachedSasToken.get(), contextEncryptionAdapter,
           reqParams, cachedSasToken.get(), contextEncryptionAdapter,
           new TracingContext(tracingContext));
           new TracingContext(tracingContext));
       cachedSasToken.update(op.getSasToken());
       cachedSasToken.update(op.getSasToken());
@@ -606,7 +606,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
       outputStreamStatistics.uploadFailed(bytesLength);
       outputStreamStatistics.uploadFailed(bytesLength);
       failureWhileSubmit(ex);
       failureWhileSubmit(ex);
     } finally {
     } finally {
-      IOUtils.close(uploadData);
+      IOUtils.close(uploadData, activeBlock);
     }
     }
   }
   }
 
 

+ 62 - 20
hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md

@@ -581,7 +581,7 @@ This will delete the containers; the output log of the test run will
 provide the details and summary of the operation.
 provide the details and summary of the operation.
 
 
 
 
-## Testing the Azure ABFS Client
+# Testing the Azure ABFS Client
 
 
 Azure Data Lake Storage Gen 2 (ADLS Gen 2) is a set of capabilities dedicated to
 Azure Data Lake Storage Gen 2 (ADLS Gen 2) is a set of capabilities dedicated to
 big data analytics, built on top of Azure Blob Storage. The ABFS and ABFSS
 big data analytics, built on top of Azure Blob Storage. The ABFS and ABFSS
@@ -602,29 +602,26 @@ various test combinations, it will:
 2. Run tests for all combinations
 2. Run tests for all combinations
 3. Summarize results across all the test combination runs.
 3. Summarize results across all the test combination runs.
 
 
-Below are the pre-requiste steps to follow:
-1. Copy
+Below are the pre-requisite steps to follow:
+1. Copy `./src/test/resources/azure-auth-keys.xml.template` to
+`./src/test/resources/azure-auth-keys.xml`
+1. Update account names that should be used in the test run for HNS and non-HNS
+combinations in the 2 properties present in the xml (account name should be
+without domain part), namely
+   1. `fs.azure.hnsTestAccountName`: Specify the HNS Enabled Account
+   2. `fs.azure.nonHnsTestAccountName`: Specify the HNS Disabled Account
 
 
-        ./src/test/resources/azure-auth-keys.xml.template
-        TO
-        ./src/test/resources/azure-auth-keys.xml
-  Update account names that should be used in the test run for HNS and non-HNS
-  combinations in the 2 properties present in the xml (account name should be
-  without domain part), namely
+   Note: `azure-auth-keys.xml` is listed in .gitignore, so any accidental account name leak is prevented.
 
 
-    fs.azure.hnsTestAccountName
-    fs.azure.nonHnsTestAccountName
-  azure-auth-keys.xml is listed in .gitignore, so any accidental account name leak is prevented.
+    ```
+    XInclude is supported, so for extra security secrets may be
+    kept out of the source tree then referenced through an XInclude element:
 
 
-```
-XInclude is supported, so for extra security secrets may be
-kept out of the source tree then referenced through an XInclude element:
-
-      <include xmlns="http://www.w3.org/2001/XInclude"
-        href="/users/self/.secrets/auth-keys.xml" />
-```
+          <include xmlns="http://www.w3.org/2001/XInclude"
+            href="/users/self/.secrets/auth-keys.xml" />
+    ```
 
 
-2. Create account config files (one config file per account) in folder:
+1. Create account config files (one config file per account) in folder:
 
 
         ./src/test/resources/accountSettings/
         ./src/test/resources/accountSettings/
    Follow the instruction in the start of the template file
    Follow the instruction in the start of the template file
@@ -634,6 +631,8 @@ kept out of the source tree then referenced through an XInclude element:
    New files created in folder accountSettings is listed in .gitignore to
    New files created in folder accountSettings is listed in .gitignore to
    prevent accidental cred leaks.
    prevent accidental cred leaks.
 
 
+You are all set to run the test script.
+
 **To run PR validation:** Running command
 **To run PR validation:** Running command
 * `dev-support/testrun-scripts/runtests.sh` will prompt as below:
 * `dev-support/testrun-scripts/runtests.sh` will prompt as below:
 ```bash
 ```bash
@@ -664,6 +663,9 @@ Set the active test combination to run the action:
 2) HNS-SharedKey           4) AppendBlob-HNS-OAuth    6) Quit
 2) HNS-SharedKey           4) AppendBlob-HNS-OAuth    6) Quit
 #? 1
 #? 1
 
 
+============================================================
+HNS-OAuth
+============================================================
 Combination specific property setting: [ key=fs.azure.account.auth.type , value=OAuth ]
 Combination specific property setting: [ key=fs.azure.account.auth.type , value=OAuth ]
 
 
 Activated [src/test/resources/abfs-combination-test-configs.xml] - for account: snvijayacontracttest for combination HNS-OAuth
 Activated [src/test/resources/abfs-combination-test-configs.xml] - for account: snvijayacontracttest for combination HNS-OAuth
@@ -682,6 +684,46 @@ consolidated results of all the combination runs will be saved into a file as
 Test-Results.log in the same folder. When run for PR validation, the
 Test-Results.log in the same folder. When run for PR validation, the
 consolidated test results needs to be pasted into the PR comment section.
 consolidated test results needs to be pasted into the PR comment section.
 
 
+**Aggregated Test Results**: `Test-Results.txt` file will show the aggregated results
+across all th combinations ran as part of script in following format
+```bash
+    ============================================================
+    HNS-OAuth
+    ============================================================
+    [ERROR] testAbfsHttpSendStatistics(org.apache.hadoop.fs.azurebfs.ITestAbfsNetworkStatistics)  Time elapsed: 3.137 s  <<< FAILURE!
+    [ERROR] testBlobDataContributor(org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemOauth)  Time elapsed: 4.154 s  <<< ERROR!
+
+    [WARNING] Tests run: 137, Failures: 0, Errors: 0, Skipped: 2
+    [ERROR] Tests run: 623, Failures: 1, Errors: 0, Skipped: 73
+    [ERROR] Tests run: 340, Failures: 0, Errors: 1, Skipped: 55
+
+    ============================================================
+    HNS-SharedKey
+    ============================================================
+    [ERROR] testAbfsHttpSendStatistics(org.apache.hadoop.fs.azurebfs.ITestAbfsNetworkStatistics)  Time elapsed: 2.175 s  <<< FAILURE!
+
+    [WARNING] Tests run: 137, Failures: 0, Errors: 0, Skipped: 3
+    [ERROR] Tests run: 623, Failures: 1, Errors: 0, Skipped: 42
+    [WARNING] Tests run: 340, Failures: 0, Errors: 0, Skipped: 41
+
+    ============================================================
+    NonHNS-SharedKey
+    ============================================================
+    [ERROR] testNonRecursiveDeleteWithPagination(org.apache.hadoop.fs.azurebfs.services.ITestAbfsPaginatedDelete)  Time elapsed: 0.85 s  <<< ERROR!
+
+    [WARNING] Tests run: 137, Failures: 0, Errors: 0, Skipped: 9
+    [ERROR] Tests run: 607, Failures: 1, Errors: 1, Skipped: 269
+    [WARNING] Tests run: 340, Failures: 0, Errors: 0, Skipped: 44
+
+    ============================================================
+    AppendBlob-HNS-OAuth
+    ============================================================
+
+    [WARNING] Tests run: 137, Failures: 0, Errors: 0, Skipped: 2
+    [ERROR] Tests run: 623, Failures: 0, Errors: 0, Skipped: 73
+    [ERROR] Tests run: 340, Failures: 0, Errors: 0, Skipped: 79
+```
+
 **To add a new test combination:** Templates for mandatory test combinations
 **To add a new test combination:** Templates for mandatory test combinations
 for PR validation are present in `dev-support/testrun-scripts/runtests.sh`.
 for PR validation are present in `dev-support/testrun-scripts/runtests.sh`.
 If a new one needs to be added, add a combination to
 If a new one needs to be added, add a combination to

+ 55 - 25
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java

@@ -26,6 +26,7 @@ import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
 
 
 import org.junit.After;
 import org.junit.After;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -106,18 +107,11 @@ public abstract class AbstractAbfsIntegrationTest extends
     abfsConfig = new AbfsConfiguration(rawConfig, accountName);
     abfsConfig = new AbfsConfiguration(rawConfig, accountName);
 
 
     authType = abfsConfig.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey);
     authType = abfsConfig.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey);
+    assumeValidAuthConfigsPresent();
+
     abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME
     abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME
             : FileSystemUriSchemes.ABFS_SECURE_SCHEME;
             : FileSystemUriSchemes.ABFS_SECURE_SCHEME;
 
 
-    if (authType == AuthType.SharedKey) {
-      assumeTrue("Not set: " + FS_AZURE_ACCOUNT_KEY,
-          abfsConfig.get(FS_AZURE_ACCOUNT_KEY) != null);
-      // Update credentials
-    } else {
-      assumeTrue("Not set: " + FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME,
-          abfsConfig.get(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME) != null);
-    }
-
     final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
     final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
     URI defaultUri = null;
     URI defaultUri = null;
 
 
@@ -130,7 +124,7 @@ public abstract class AbstractAbfsIntegrationTest extends
     this.testUrl = defaultUri.toString();
     this.testUrl = defaultUri.toString();
     abfsConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
     abfsConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
     abfsConfig.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
     abfsConfig.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
-    if (abfsConfig.get(FS_AZURE_TEST_APPENDBLOB_ENABLED) == "true") {
+    if (isAppendBlobEnabled()) {
       String appendblobDirs = this.testUrl + "," + abfsConfig.get(FS_AZURE_CONTRACT_TEST_URI);
       String appendblobDirs = this.testUrl + "," + abfsConfig.get(FS_AZURE_CONTRACT_TEST_URI);
       rawConfig.set(FS_AZURE_APPEND_BLOB_KEY, appendblobDirs);
       rawConfig.set(FS_AZURE_APPEND_BLOB_KEY, appendblobDirs);
     }
     }
@@ -215,6 +209,7 @@ public abstract class AbstractAbfsIntegrationTest extends
       wasb = new NativeAzureFileSystem(azureNativeFileSystemStore);
       wasb = new NativeAzureFileSystem(azureNativeFileSystemStore);
       wasb.initialize(wasbUri, rawConfig);
       wasb.initialize(wasbUri, rawConfig);
     }
     }
+    // Todo: To be fixed in HADOOP-19137
     AbfsClientUtils.setIsNamespaceEnabled(abfs.getAbfsClient(), true);
     AbfsClientUtils.setIsNamespaceEnabled(abfs.getAbfsClient(), true);
   }
   }
 
 
@@ -263,26 +258,28 @@ public abstract class AbstractAbfsIntegrationTest extends
   }
   }
 
 
   public void loadConfiguredFileSystem() throws Exception {
   public void loadConfiguredFileSystem() throws Exception {
-      // disable auto-creation of filesystem
-      abfsConfig.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
+    // disable auto-creation of filesystem
+    abfsConfig.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
           false);
           false);
 
 
-      // AbstractAbfsIntegrationTest always uses a new instance of FileSystem,
-      // need to disable that and force filesystem provided in test configs.
-      String[] authorityParts =
-          (new URI(rawConfig.get(FS_AZURE_CONTRACT_TEST_URI))).getRawAuthority().split(
-        AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
-      this.fileSystemName = authorityParts[0];
+    // AbstractAbfsIntegrationTest always uses a new instance of FileSystem,
+    // need to disable that and force filesystem provided in test configs.
+    assumeValidTestConfigPresent(this.getRawConfiguration(), FS_AZURE_CONTRACT_TEST_URI);
 
 
-      // Reset URL with configured filesystem
-      final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
-      URI defaultUri = null;
+    String[] authorityParts =
+        (new URI(rawConfig.get(FS_AZURE_CONTRACT_TEST_URI))).getRawAuthority().split(
+      AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
+    this.fileSystemName = authorityParts[0];
 
 
-      defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
+    // Reset URL with configured filesystem
+    final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
+    URI defaultUri = null;
 
 
-      this.testUrl = defaultUri.toString();
-      abfsConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
-          defaultUri.toString());
+    defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
+
+    this.testUrl = defaultUri.toString();
+    abfsConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+        defaultUri.toString());
 
 
     useConfiguredFileSystem = true;
     useConfiguredFileSystem = true;
   }
   }
@@ -532,4 +529,37 @@ public abstract class AbstractAbfsIntegrationTest extends
         (long) metricMap.get(statistic.getStatName()));
         (long) metricMap.get(statistic.getStatName()));
     return expectedValue;
     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());
+  }
+
+  protected void assumeValidAuthConfigsPresent() {
+    final AuthType currentAuthType = getAuthType();
+    Assume.assumeFalse(
+        "SAS Based Authentication Not Allowed For Integration Tests",
+        currentAuthType == AuthType.SAS);
+    if (currentAuthType == AuthType.SharedKey) {
+      assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_ACCOUNT_KEY);
+    } else if (currentAuthType == AuthType.OAuth) {
+      assumeValidTestConfigPresent(getRawConfiguration(),
+          FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME);
+      assumeValidTestConfigPresent(getRawConfiguration(),
+          FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID);
+      assumeValidTestConfigPresent(getRawConfiguration(),
+          FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET);
+      assumeValidTestConfigPresent(getRawConfiguration(),
+          FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT);
+    } else if (currentAuthType == AuthType.Custom) {
+      assumeValidTestConfigPresent(getRawConfiguration(),
+          FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME);
+    }
+  }
+
+  protected boolean isAppendBlobEnabled() {
+    return getRawConfiguration().getBoolean(FS_AZURE_TEST_APPENDBLOB_ENABLED, false);
+  }
 }
 }

+ 51 - 11
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java

@@ -34,6 +34,7 @@ import org.junit.Test;
 
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
@@ -102,11 +103,31 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
       setListMaxResults(listMaxResults);
       setListMaxResults(listMaxResults);
       int expectedListResultsSize =
       int expectedListResultsSize =
           listMaxResults > fileCount ? fileCount : listMaxResults;
           listMaxResults > fileCount ? fileCount : listMaxResults;
-      Assertions.assertThat(listPath(directory.toString())).describedAs(
-          "AbfsClient.listPath result should contain %d items when "
-              + "listMaxResults is %d and directory contains %d items",
-          expectedListResultsSize, listMaxResults, fileCount)
-          .hasSize(expectedListResultsSize);
+
+      AbfsRestOperation op = getFileSystem().getAbfsClient().listPath(
+          directory.toString(), false, getListMaxResults(), null,
+          getTestTracingContext(getFileSystem(), true));
+
+      List<ListResultEntrySchema> list = op.getResult().getListResultSchema().paths();
+      String continuationToken = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+
+      if (continuationToken == null) {
+        // Listing is complete and number of objects should be same as expected
+        Assertions.assertThat(list)
+            .describedAs("AbfsClient.listPath() should return %d items"
+                + " when listMaxResults is %d, directory contains %d items and "
+                + "listing is complete",
+                expectedListResultsSize, listMaxResults, fileCount)
+            .hasSize(expectedListResultsSize);
+      } else {
+        // Listing is incomplete and number of objects can be less than expected
+        Assertions.assertThat(list)
+            .describedAs("AbfsClient.listPath() should return %d items"
+                + " or less when listMaxResults is %d,  directory contains"
+                + " %d items and listing is incomplete",
+                expectedListResultsSize, listMaxResults, fileCount)
+            .hasSizeLessThanOrEqualTo(expectedListResultsSize);
+      }
     }
     }
   }
   }
 
 
@@ -117,12 +138,31 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
     final Path directory = getUniquePath(
     final Path directory = getUniquePath(
         "testWithValueGreaterThanServerMaximum");
         "testWithValueGreaterThanServerMaximum");
     createDirectoryWithNFiles(directory, LIST_MAX_RESULTS_SERVER + 200);
     createDirectoryWithNFiles(directory, LIST_MAX_RESULTS_SERVER + 200);
-    Assertions.assertThat(listPath(directory.toString())).describedAs(
-        "AbfsClient.listPath result will contain a maximum of %d items "
-            + "even if listMaxResults >= %d or directory "
-            + "contains more than %d items", LIST_MAX_RESULTS_SERVER,
-        LIST_MAX_RESULTS_SERVER, LIST_MAX_RESULTS_SERVER)
-        .hasSize(LIST_MAX_RESULTS_SERVER);
+
+    AbfsRestOperation op = getFileSystem().getAbfsClient().listPath(
+        directory.toString(), false, getListMaxResults(), null,
+        getTestTracingContext(getFileSystem(), true));
+
+    List<ListResultEntrySchema> list = op.getResult().getListResultSchema().paths();
+    String continuationToken = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+
+    if (continuationToken == null) {
+      // Listing is complete and number of objects should be same as expected
+      Assertions.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,
+              LIST_MAX_RESULTS_SERVER, LIST_MAX_RESULTS_SERVER)
+          .hasSize(LIST_MAX_RESULTS_SERVER);
+    } else {
+      // Listing is incomplete and number of objects can be less than expected
+      Assertions.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,
+              LIST_MAX_RESULTS_SERVER, LIST_MAX_RESULTS_SERVER)
+          .hasSizeLessThanOrEqualTo(LIST_MAX_RESULTS_SERVER);
+    }
   }
   }
 
 
   @Test
   @Test

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

@@ -136,10 +136,10 @@ public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
       }
       }
 
 
       if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(largeOperationsFile).toString())) {
       if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(largeOperationsFile).toString())) {
-        // for appendblob data is already flushed, so there is more data to read.
+        // 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 "
         assertTrue(String.format("The actual value of %d was not equal to the "
               + "expected value", statistics.getReadOps()),
               + "expected value", statistics.getReadOps()),
-          statistics.getReadOps() == (largeValue + 3) || statistics.getReadOps() == (largeValue + 4));
+          statistics.getReadOps() >= largeValue  || statistics.getReadOps() <= (largeValue + 4));
       } else {
       } else {
         //Test for 1000000 read operations
         //Test for 1000000 read operations
         assertReadWriteOps("read", largeValue, statistics.getReadOps());
         assertReadWriteOps("read", largeValue, statistics.getReadOps());

+ 8 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.security.AccessControlException;
 
 
 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_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
 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_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_OAUTH_CLIENT_ENDPOINT;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT;
 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_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_CHECK_ACCESS;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_CHECK_ACCESS;
@@ -86,25 +87,26 @@ public class ITestAzureBlobFileSystemCheckAccess
     }
     }
     checkIfConfigIsSet(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT
     checkIfConfigIsSet(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT
         + "." + getAccountName());
         + "." + getAccountName());
-    Configuration conf = getRawConfiguration();
+    Configuration conf = new Configuration(getRawConfiguration());
     setTestFsConf(FS_AZURE_BLOB_FS_CLIENT_ID,
     setTestFsConf(FS_AZURE_BLOB_FS_CLIENT_ID,
-        FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID);
+        FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID, conf);
     setTestFsConf(FS_AZURE_BLOB_FS_CLIENT_SECRET,
     setTestFsConf(FS_AZURE_BLOB_FS_CLIENT_SECRET,
-        FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET);
+        FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET, conf);
     conf.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.OAuth.name());
     conf.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.OAuth.name());
     conf.set(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + "."
     conf.set(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + "."
         + getAccountName(), ClientCredsTokenProvider.class.getName());
         + getAccountName(), ClientCredsTokenProvider.class.getName());
     conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
     conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
         false);
         false);
-    this.testUserFs = FileSystem.newInstance(getRawConfiguration());
+    conf.unset(FS_AZURE_ACCOUNT_IS_HNS_ENABLED);
+    this.testUserFs = FileSystem.newInstance(conf);
   }
   }
 
 
   private void setTestFsConf(final String fsConfKey,
   private void setTestFsConf(final String fsConfKey,
-      final String testFsConfKey) {
+      final String testFsConfKey, Configuration conf) {
     final String confKeyWithAccountName = fsConfKey + "." + getAccountName();
     final String confKeyWithAccountName = fsConfKey + "." + getAccountName();
     final String confValue = getConfiguration()
     final String confValue = getConfiguration()
         .getString(testFsConfKey, "");
         .getString(testFsConfKey, "");
-    getRawConfiguration().set(confKeyWithAccountName, confValue);
+    conf.set(confKeyWithAccountName, confValue);
   }
   }
 
 
   @Test(expected = IllegalArgumentException.class)
   @Test(expected = IllegalArgumentException.class)

+ 16 - 12
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java

@@ -71,14 +71,15 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
     AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true);
     AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true);
     AbfsClient client = fs.getAbfsStore().getClient();
     AbfsClient client = fs.getAbfsStore().getClient();
     Path path = path("testPath" + getMethodName());
     Path path = path("testPath" + getMethodName());
-    try (FSDataOutputStream out = fs.create(path)) {
-      byte[] data = generateRandomBytes(MB_4);
-
-      appendWithOffsetHelper(client, path, data, fs, 0);
-      appendWithOffsetHelper(client, path, data, fs, ONE_MB);
-      appendWithOffsetHelper(client, path, data, fs, MB_2);
-      appendWithOffsetHelper(client, path, data, fs, MB_4 - 1);
-    }
+    fs.create(path);
+    byte[] data = generateRandomBytes(MB_4);
+    int pos = 0;
+
+    pos += appendWithOffsetHelper(client, path, data, fs, pos, 0);
+    pos += appendWithOffsetHelper(client, path, data, fs, pos, ONE_MB);
+    pos += appendWithOffsetHelper(client, path, data, fs, pos, MB_2);
+    appendWithOffsetHelper(client, path, data, fs, pos, MB_4 - 1);
+    fs.close();
   }
   }
 
 
   @Test
   @Test
@@ -94,6 +95,7 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
     readWithOffsetAndPositionHelper(client, path, data, fs, MB_4, ONE_MB);
     readWithOffsetAndPositionHelper(client, path, data, fs, MB_4, ONE_MB);
     readWithOffsetAndPositionHelper(client, path, data, fs, MB_8, MB_2);
     readWithOffsetAndPositionHelper(client, path, data, fs, MB_8, MB_2);
     readWithOffsetAndPositionHelper(client, path, data, fs, MB_15, MB_4 - 1);
     readWithOffsetAndPositionHelper(client, path, data, fs, MB_15, MB_4 - 1);
+    fs.close();
   }
   }
 
 
   @Test
   @Test
@@ -114,12 +116,13 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
     Mockito.doReturn(invalidMD5Hash).when(spiedClient).computeMD5Hash(any(),
     Mockito.doReturn(invalidMD5Hash).when(spiedClient).computeMD5Hash(any(),
         any(Integer.class), any(Integer.class));
         any(Integer.class), any(Integer.class));
     AbfsRestOperationException ex = intercept(AbfsInvalidChecksumException.class, () -> {
     AbfsRestOperationException ex = intercept(AbfsInvalidChecksumException.class, () -> {
-      appendWithOffsetHelper(spiedClient, path, data, fs, 0);
+      appendWithOffsetHelper(spiedClient, path, data, fs, 0, 0);
     });
     });
 
 
     Assertions.assertThat(ex.getErrorCode())
     Assertions.assertThat(ex.getErrorCode())
         .describedAs("Exception Message should contain MD5Mismatch")
         .describedAs("Exception Message should contain MD5Mismatch")
         .isEqualTo(AzureServiceErrorCode.MD5_MISMATCH);
         .isEqualTo(AzureServiceErrorCode.MD5_MISMATCH);
+    fs.close();
   }
   }
 
 
   @Test
   @Test
@@ -170,12 +173,13 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
    * @param offset
    * @param offset
    * @throws Exception
    * @throws Exception
    */
    */
-  private void appendWithOffsetHelper(AbfsClient client, Path path,
-      byte[] data, AzureBlobFileSystem fs, final int offset) throws Exception {
+  private int appendWithOffsetHelper(AbfsClient client, Path path,
+      byte[] data, AzureBlobFileSystem fs, final int pos, final int offset) throws Exception {
     AppendRequestParameters reqParams = new AppendRequestParameters(
     AppendRequestParameters reqParams = new AppendRequestParameters(
-        0, offset, data.length - offset, APPEND_MODE, false, null, true);
+        pos, offset, data.length - offset, APPEND_MODE, isAppendBlobEnabled(), null, true);
     client.append(path.toUri().getPath(), data, reqParams, null, null,
     client.append(path.toUri().getPath(), data, reqParams, null, null,
         getTestTracingContext(fs, false));
         getTestTracingContext(fs, false));
+    return reqParams.getLength();
   }
   }
 
 
   /**
   /**

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

@@ -18,9 +18,7 @@
 
 
 package org.apache.hadoop.fs.azurebfs;
 package org.apache.hadoop.fs.azurebfs;
 
 
-import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
-import java.io.OutputStream;
 
 
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.junit.Test;
 import org.junit.Test;

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

@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.RejectedExecutionException;
 
 
 import org.junit.Assert;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -63,7 +64,6 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
 
 
   public ITestAzureBlobFileSystemLease() throws Exception {
   public ITestAzureBlobFileSystemLease() throws Exception {
     super();
     super();
-
     this.isHNSEnabled = getConfiguration()
     this.isHNSEnabled = getConfiguration()
         .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
         .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
   }
   }
@@ -136,6 +136,7 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   public void testTwoCreate() throws Exception {
   public void testTwoCreate() throws Exception {
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1);
     final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1);
+    assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT);
     fs.mkdirs(testFilePath.getParent());
     fs.mkdirs(testFilePath.getParent());
 
 
     try (FSDataOutputStream out = fs.create(testFilePath)) {
     try (FSDataOutputStream out = fs.create(testFilePath)) {
@@ -172,6 +173,7 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   public void testTwoWritersCreateAppendNoInfiniteLease() throws Exception {
   public void testTwoWritersCreateAppendNoInfiniteLease() throws Exception {
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final AzureBlobFileSystem fs = getFileSystem();
     final AzureBlobFileSystem fs = getFileSystem();
+    Assume.assumeFalse("Parallel Writes Not Allowed on Append Blobs", isAppendBlobEnabled());
     fs.mkdirs(testFilePath.getParent());
     fs.mkdirs(testFilePath.getParent());
 
 
     twoWriters(fs, testFilePath, false);
     twoWriters(fs, testFilePath, false);
@@ -181,6 +183,7 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   public void testTwoWritersCreateAppendWithInfiniteLeaseEnabled() throws Exception {
   public void testTwoWritersCreateAppendWithInfiniteLeaseEnabled() throws Exception {
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1);
     final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1);
+    Assume.assumeFalse("Parallel Writes Not Allowed on Append Blobs", isAppendBlobEnabled());
     fs.mkdirs(testFilePath.getParent());
     fs.mkdirs(testFilePath.getParent());
 
 
     twoWriters(fs, testFilePath, true);
     twoWriters(fs, testFilePath, true);

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

@@ -72,6 +72,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
 
 
   @Test
   @Test
   public void testNonXNSAccount() throws IOException {
   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",
     Assume.assumeFalse("Skip this test because the account being used for test is a XNS account",
             isUsingXNSAccount);
             isUsingXNSAccount);
     assertFalse("Expecting getIsNamespaceEnabled() return false",
     assertFalse("Expecting getIsNamespaceEnabled() return false",
@@ -80,6 +81,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
 
 
   @Test
   @Test
   public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception {
   public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception {
+    assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT);
     AzureBlobFileSystem fs = getNewFSWithHnsConf(TRUE_STR);
     AzureBlobFileSystem fs = getNewFSWithHnsConf(TRUE_STR);
     Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs(
     Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs(
         "getIsNamespaceEnabled should return true when the "
         "getIsNamespaceEnabled should return true when the "
@@ -90,6 +92,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
 
 
   @Test
   @Test
   public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception {
   public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception {
+    assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT);
     AzureBlobFileSystem fs = getNewFSWithHnsConf(FALSE_STR);
     AzureBlobFileSystem fs = getNewFSWithHnsConf(FALSE_STR);
     Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs(
     Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs(
         "getIsNamespaceEnabled should return false when the "
         "getIsNamespaceEnabled should return false when the "

+ 24 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbstractAbfsClusterITest.java

@@ -23,17 +23,19 @@ import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
 import java.time.format.DateTimeFormatter;
 
 
 import org.junit.AfterClass;
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.Rule;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TemporaryFolder;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
 import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding;
 import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding;
 import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract;
 import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.output.committer.manifest.AbstractManifestCommitterTest;
 import org.apache.hadoop.mapreduce.lib.output.committer.manifest.AbstractManifestCommitterTest;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
@@ -41,6 +43,10 @@ import org.apache.hadoop.util.DurationInfo;
 
 
 import static java.util.Objects.requireNonNull;
 import static java.util.Objects.requireNonNull;
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled;
 import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled;
+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_OVERRIDE_OWNER_SP;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_OVERRIDE_OWNER_SP_LIST;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID;
 import static org.apache.hadoop.io.IOUtils.closeStream;
 import static org.apache.hadoop.io.IOUtils.closeStream;
 
 
 /**
 /**
@@ -226,6 +232,18 @@ public abstract class AbstractAbfsClusterITest extends
   protected JobConf newJobConf() throws IOException {
   protected JobConf newJobConf() throws IOException {
     JobConf jobConf = new JobConf(getYarn().getConfig());
     JobConf jobConf = new JobConf(getYarn().getConfig());
     jobConf.addResource(getConfiguration());
     jobConf.addResource(getConfiguration());
+
+    if (getConfiguration().getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME,
+        AuthType.SharedKey) == AuthType.OAuth) {
+      assumeValidTestConfigPresent(
+          FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID);
+      String pid = jobConf.get(FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID);
+      jobConf.set(FS_AZURE_OVERRIDE_OWNER_SP, pid);
+      jobConf.set(FS_AZURE_OVERRIDE_OWNER_SP_LIST, "*");
+      jobConf.setBoolean(String.format("fs.%s.impl.disable.cache",
+          FileSystemUriSchemes.ABFS_SECURE_SCHEME), true);
+    }
+
     applyCustomConfigOptions(jobConf);
     applyCustomConfigOptions(jobConf);
     return jobConf;
     return jobConf;
   }
   }
@@ -257,4 +275,9 @@ public abstract class AbstractAbfsClusterITest extends
     assumeScaleTestsEnabled(getConfiguration());
     assumeScaleTestsEnabled(getConfiguration());
   }
   }
 
 
+  protected void assumeValidTestConfigPresent(final String key) {
+    String configuredValue = getConfiguration().get(key);
+    Assume.assumeTrue(configuredValue != null && !configuredValue.isEmpty());
+  }
+
 }
 }

+ 1 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java

@@ -41,6 +41,7 @@ public final class TestConfigurationKeys {
 
 
   public static final String FS_AZURE_BLOB_FS_CLIENT_ID = "fs.azure.account.oauth2.client.id";
   public static final String FS_AZURE_BLOB_FS_CLIENT_ID = "fs.azure.account.oauth2.client.id";
   public static final String FS_AZURE_BLOB_FS_CLIENT_SECRET = "fs.azure.account.oauth2.client.secret";
   public static final String FS_AZURE_BLOB_FS_CLIENT_SECRET = "fs.azure.account.oauth2.client.secret";
+  public static final String FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID = "fs.azure.account.oauth2.client.service.principal.object.id";
 
 
   public static final String FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID = "fs.azure.account.test.oauth2.client.id";
   public static final String FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID = "fs.azure.account.test.oauth2.client.id";
   public static final String FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET = "fs.azure.account.test.oauth2.client.secret";
   public static final String FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET = "fs.azure.account.test.oauth2.client.secret";

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

@@ -111,7 +111,7 @@ public class ITestExponentialRetryPolicy extends AbstractAbfsIntegrationTest {
 
 
     configuration.setBoolean(FS_AZURE_ENABLE_AUTOTHROTTLING, true);
     configuration.setBoolean(FS_AZURE_ENABLE_AUTOTHROTTLING, true);
     configuration.setBoolean(FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED, true);
     configuration.setBoolean(FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED, true);
-    // On disabling throttling AbfsClientThrottlingIntercept object is returned
+    // On enabling throttling AbfsClientThrottlingIntercept object is returned
     AbfsConfiguration abfsConfiguration1 = new AbfsConfiguration(configuration,
     AbfsConfiguration abfsConfiguration1 = new AbfsConfiguration(configuration,
         "dummy1.dfs.core.windows.net");
         "dummy1.dfs.core.windows.net");
     AbfsClient abfsClient1 = ITestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration1);
     AbfsClient abfsClient1 = ITestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration1);

+ 17 - 12
hadoop-tools/hadoop-azure/src/test/resources/accountSettings/accountName_settings.xml.template

@@ -39,16 +39,17 @@
     5. SUPERUSER_TENANT_ID -> AAD tenant ID of Superuser Service principal
     5. SUPERUSER_TENANT_ID -> AAD tenant ID of Superuser Service principal
     6. SUPERUSER_CLIENT_ID -> SuperUser Service principal's client ID
     6. SUPERUSER_CLIENT_ID -> SuperUser Service principal's client ID
     7. SUPERUSER_CLIENT_SECRET -> SuperUser Service principal's client secret
     7. SUPERUSER_CLIENT_SECRET -> SuperUser Service principal's client secret
+    8. SUPERUSER_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID -> SuperUser Service principal's object ID
 
 
-    8. NO_RBAC_USER_CLIENT_ID -> noRBACUser Service principal's client ID
-    9. NO_RBAC_USER_CLIENT_SECRET -> noRBACUser Service principal's client secret
-    10. NO_RBAC_USER_OID -> noRBACUser Service principal's OID
+    9. NO_RBAC_USER_CLIENT_ID -> noRBACUser Service principal's client ID
+    10. NO_RBAC_USER_CLIENT_SECRET -> noRBACUser Service principal's client secret
+    11. NO_RBAC_USER_OID -> noRBACUser Service principal's OID
 
 
-    11. CONTRIBUTOR_RBAC_USER_CLIENT_ID -> contributorRBACUser Service principal's client ID
-    12. CONTRIBUTOR_RBAC_USER_CLIENT_SECRET -> contributorRBACUser Service principal's client secret
+    12. CONTRIBUTOR_RBAC_USER_CLIENT_ID -> contributorRBACUser Service principal's client ID
+    13. CONTRIBUTOR_RBAC_USER_CLIENT_SECRET -> contributorRBACUser Service principal's client secret
 
 
-    13. READER_RBAC_USER_CLIENT_ID -> readerRBACUser Service principal's client ID
-    14. READER_RBAC_USER_CLIENT_SECRET -> readerRBACUser Service principal's client secret
+    14. READER_RBAC_USER_CLIENT_ID -> readerRBACUser Service principal's client ID
+    15. READER_RBAC_USER_CLIENT_SECRET -> readerRBACUser Service principal's client secret
 -->
 -->
 
 
 <configuration>
 <configuration>
@@ -95,11 +96,7 @@
     <value>IS_NAMESPACE_ENABLED</value>
     <value>IS_NAMESPACE_ENABLED</value>
   </property>
   </property>
   <property>
   <property>
-    <name>fs.azure.account.key.ACCOUNTNAME.dfs.core.windows.net</name>
-    <value>ACCOUNT_KEY</value>
-  </property>
-  <property>
-    <name>fs.azure.account.key.ACCOUNTNAME.blob.core.windows.net</name>
+    <name>fs.azure.account.key</name>
     <value>ACCOUNT_KEY</value>
     <value>ACCOUNT_KEY</value>
   </property>
   </property>
   <property>
   <property>
@@ -144,6 +141,14 @@
     <name>fs.azure.account.oauth2.client.secret</name>
     <name>fs.azure.account.oauth2.client.secret</name>
     <value>SUPERUSER_CLIENT_SECRET</value>
     <value>SUPERUSER_CLIENT_SECRET</value>
   </property>
   </property>
+  <property>
+    <name>fs.azure.account.oauth2.client.service.principal.object.id.ACCOUNTNAME.dfs.core.windows.net</name>
+    <value>SUPERUSER_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID</value>
+    </property>
+  <property>
+    <name>fs.azure.account.oauth2.client.service.principal.object.id</name>
+    <value>SUPERUSER_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID</value>
+  </property>
 
 
   <!-- NO RBAC USER SETTINGS -->
   <!-- NO RBAC USER SETTINGS -->
   <property>
   <property>