فهرست منبع

HADOOP-19443: [ABFS][FnsOverBlob] Updating hadoop-azure Test Scripts and Adding Tests For Metadata APIs (#7344)

Contributed by Anuj Modi
Reviewed by Anmol Asrani, Sneha Vijayarajan, Manish Bhatt
Anuj Modi 3 ماه پیش
والد
کامیت
eae63ec182
19فایلهای تغییر یافته به همراه622 افزوده شده و 133 حذف شده
  1. 102 21
      hadoop-tools/hadoop-azure/dev-support/testrun-scripts/runtests.sh
  2. 19 0
      hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh
  3. 0 5
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
  4. 64 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
  5. 1 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
  6. 4 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
  7. 130 54
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java
  8. 2 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java
  9. 1 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
  10. 122 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
  11. 1 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
  12. 0 23
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
  13. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
  14. 154 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/TestBlobListXmlParser.java
  15. 6 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/TestDfsListJsonParser.java
  16. 11 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClientHandler.java
  17. 2 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
  18. 1 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsPaginatedDelete.java
  19. 1 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java

+ 102 - 21
hadoop-tools/hadoop-azure/dev-support/testrun-scripts/runtests.sh

@@ -32,36 +32,87 @@ processCount=8
 
 ## SECTION: TEST COMBINATION METHODS
 
-runHNSOAuthTest()
+runHNSOAuthDFSTest()
 {
   accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.hnsTestAccountName"]/value' -n $azureTestXmlPath)
   PROPERTIES=("fs.azure.account.auth.type")
   VALUES=("OAuth")
-  triggerRun "HNS-OAuth" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
+  triggerRun "HNS-OAuth-DFS" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
 }
 
-runHNSSharedKeyTest()
+runHNSSharedKeyDFSTest()
 {
   accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.hnsTestAccountName"]/value' -n $azureTestXmlPath)
   PROPERTIES=("fs.azure.account.auth.type")
   VALUES=("SharedKey")
-  triggerRun "HNS-SharedKey" "$accountName"  "$runTest" $processCount "$cleanUpTestContainers"
+  triggerRun "HNS-SharedKey-DFS" "$accountName"  "$runTest" $processCount "$cleanUpTestContainers"
 }
 
-runNonHNSSharedKeyTest()
+runNonHNSSharedKeyDFSTest()
 {
   accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.nonHnsTestAccountName"]/value' -n $azureTestXmlPath)
   PROPERTIES=("fs.azure.account.auth.type")
   VALUES=("SharedKey")
-  triggerRun "NonHNS-SharedKey" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
+  triggerRun "NonHNS-SharedKey-DFS" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
 }
 
-runAppendBlobHNSOAuthTest()
+runAppendBlobHNSOAuthDFSTest()
 {
   accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.hnsTestAccountName"]/value' -n $azureTestXmlPath)
   PROPERTIES=("fs.azure.account.auth.type" "fs.azure.test.appendblob.enabled")
   VALUES=("OAuth" "true")
-  triggerRun "AppendBlob-HNS-OAuth" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
+  triggerRun "AppendBlob-HNS-OAuth-DFS" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
+}
+
+runNonHNSSharedKeyBlobTest()
+{
+  accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.nonHnsTestAccountName"]/value' -n $azureTestXmlPath)
+  fnsBlobConfigFileCheck "$accountName"
+  PROPERTIES=("fs.azure.account.auth.type")
+  VALUES=("SharedKey")
+  triggerRun "NonHNS-SharedKey-Blob" "${accountName}_blob" "$runTest" $processCount "$cleanUpTestContainers"
+}
+
+runNonHNSOAuthDFSTest()
+{
+  accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.nonHnsTestAccountName"]/value' -n $azureTestXmlPath)
+  PROPERTIES=("fs.azure.account.auth.type")
+  VALUES=("OAuth")
+  triggerRun "NonHNS-OAuth-DFS" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
+}
+
+runNonHNSOAuthBlobTest()
+{
+  accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.nonHnsTestAccountName"]/value' -n $azureTestXmlPath)
+  fnsBlobConfigFileCheck "$accountName"
+  PROPERTIES=("fs.azure.account.auth.type")
+  VALUES=("OAuth")
+  triggerRun "NonHNS-OAuth-Blob" "${accountName}_blob" "$runTest" $processCount "$cleanUpTestContainers"
+}
+
+runAppendBlobNonHNSOAuthBlobTest()
+{
+  accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.nonHnsTestAccountName"]/value' -n $azureTestXmlPath)
+  fnsBlobConfigFileCheck "$accountName"
+  PROPERTIES=("fs.azure.account.auth.type" "fs.azure.test.appendblob.enabled")
+  VALUES=("OAuth" "true")
+  triggerRun "AppendBlob-NonHNS-OAuth-Blob" "${accountName}_blob" "$runTest" $processCount "$cleanUpTestContainers"
+}
+
+runHNSOAuthDFSIngressBlobTest()
+{
+  accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.hnsTestAccountName"]/value' -n $azureTestXmlPath)
+  PROPERTIES=("fs.azure.account.auth.type" "fs.azure.ingress.service.type")
+  VALUES=("OAuth" "blob")
+  triggerRun "HNS-Oauth-DFS-IngressBlob" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
+}
+
+runNonHNSOAuthDFSIngressBlobTest()
+{
+  accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.nonHnsTestAccountName"]/value' -n $azureTestXmlPath)
+  PROPERTIES=("fs.azure.account.auth.type" "fs.azure.ingress.service.type")
+  VALUES=("OAuth" "blob")
+  triggerRun "NonHNS-OAuth-DFS-IngressBlob" "$accountName" "$runTest" $processCount "$cleanUpTestContainers"
 }
 
 runTest=false
@@ -130,23 +181,47 @@ done
 
 echo ' '
 echo 'Set the active test combination to run the action:'
-select combo in HNS-OAuth HNS-SharedKey nonHNS-SharedKey AppendBlob-HNS-OAuth AllCombinationsTestRun Quit
+select combo in HNS-OAuth-DFS HNS-SharedKey-DFS NonHNS-SharedKey-DFS AppendBlob-HNS-OAuth-DFS NonHNS-SharedKey-Blob NonHNS-OAuth-DFS NonHNS-OAuth-Blob AppendBlob-NonHNS-OAuth-Blob HNS-Oauth-DFS-IngressBlob NonHNS-Oauth-DFS-IngressBlob AllCombinationsTestRun Quit
 do
    case $combo in
-      HNS-OAuth)
-         runHNSOAuthTest
+      HNS-OAuth-DFS)
+         runHNSOAuthDFSTest
+         break
+         ;;
+      HNS-SharedKey-DFS)
+         runHNSSharedKeyDFSTest
+         break
+         ;;
+      NonHNS-SharedKey-DFS)
+         runNonHNSSharedKeyDFSTest
+         break
+         ;;
+       AppendBlob-HNS-OAuth-DFS)
+         runAppendBlobHNSOAuthDFSTest
+         break
+         ;;
+       NonHNS-SharedKey-Blob)
+         runNonHNSSharedKeyBlobTest
+         break
+         ;;
+        NonHNS-OAuth-DFS)
+         runNonHNSOAuthDFSTest
+         break
+         ;;
+        NonHNS-OAuth-Blob)
+         runNonHNSOAuthBlobTest
          break
          ;;
-      HNS-SharedKey)
-         runHNSSharedKeyTest
+        AppendBlob-NonHNS-OAuth-Blob)
+         runAppendBlobNonHNSOAuthBlobTest
          break
          ;;
-      nonHNS-SharedKey)
-         runNonHNSSharedKeyTest
+        HNS-Oauth-DFS-IngressBlob)
+         runHNSOAuthDFSIngressBlobTest
          break
          ;;
-       AppendBlob-HNS-OAuth)
-         runAppendBlobHNSOAuthTest
+        NonHNS-Oauth-DFS-IngressBlob)
+         runNonHNSOAuthDFSIngressBlobTest
          break
          ;;
       AllCombinationsTestRun)
@@ -155,10 +230,16 @@ do
           logOutput "ERROR: Invalid selection for SET_ACTIVE_TEST_CONFIG. This is applicable only for RUN_TEST."
           break
         fi
-         runHNSOAuthTest
-         runHNSSharedKeyTest
-         runNonHNSSharedKeyTest
-         runAppendBlobHNSOAuthTest ## Keep this as the last run scenario always
+        runHNSOAuthDFSTest
+        runHNSSharedKeyDFSTest
+        runNonHNSSharedKeyDFSTest
+        runAppendBlobHNSOAuthDFSTest
+        runNonHNSSharedKeyBlobTest
+        runNonHNSOAuthDFSTest
+        runNonHNSOAuthBlobTest
+        runAppendBlobNonHNSOAuthBlobTest
+        runHNSOAuthDFSIngressBlobTest
+        runNonHNSOAuthDFSIngressBlobTest
          break
          ;;
       Quit)

+ 19 - 0
hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh

@@ -45,6 +45,25 @@ ENDTIME=$(date +%s)
 outputFormatOn="\033[0;95m"
 outputFormatOff="\033[0m"
 
+# Function to check if the blob config file exists and create one if it doesn't.
+fnsBlobConfigFileCheck() {
+  baseFileName=$1
+  targetWord=".dfs."
+  replacementWord=".blob."
+  accountSettingsDir="src/test/resources/accountSettings/"
+  accountConfigFileSuffix="_settings.xml"
+  sourceFilePath="${accountSettingsDir}${baseFileName}${accountConfigFileSuffix}"
+  targetFilePath="${accountSettingsDir}${baseFileName}_blob${accountConfigFileSuffix}"
+
+  if [ ! -f "$targetFilePath" ]; then
+    cp "$sourceFilePath" "$targetFilePath"
+    sed -i "s/$targetWord/$replacementWord/g" "$targetFilePath"
+    echo "File created and word replaced."
+  else
+    echo "File already exists."
+  fi
+}
+
 triggerRun()
 {
   echo ' '

+ 0 - 5
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java

@@ -550,11 +550,6 @@ public class AbfsConfiguration{
    */
   public void validateConfiguredServiceType(boolean isHNSEnabled)
       throws InvalidConfigurationValueException {
-    // TODO: [FnsOverBlob][HADOOP-19179] Remove this check when FNS over Blob is ready.
-    if (getFsConfiguredServiceType() == AbfsServiceType.BLOB) {
-      throw new InvalidConfigurationValueException(FS_DEFAULT_NAME_KEY,
-          "Blob Endpoint Support not yet available");
-    }
     if (isHNSEnabled && getConfiguredServiceTypeForFNSAccounts() == AbfsServiceType.BLOB) {
       throw new InvalidConfigurationValueException(
           FS_AZURE_FNS_ACCOUNT_SERVICE_TYPE, "Service Type Cannot be BLOB for HNS Account");

+ 64 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java

@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 
+import org.assertj.core.api.Assertions;
 import org.junit.After;
 import org.junit.Assume;
 import org.junit.Before;
@@ -64,6 +65,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DFS_DOMAIN_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
@@ -632,8 +634,70 @@ public abstract class AbstractAbfsIntegrationTest extends
         + accountName + FORWARD_SLASH + fileSystemName + pathFromContainerRoot;
   }
 
+  /**
+   * Utility method to assume that the test is running against a Blob service.
+   * Otherwise, the test will be skipped.
+   */
   protected void assumeBlobServiceType() {
     Assume.assumeTrue("Blob service type is required for this test",
         getAbfsServiceType() == AbfsServiceType.BLOB);
   }
+
+  /**
+   * Utility method to assume that the test is running against a DFS service.
+   * Otherwise, the test will be skipped.
+   */
+  protected void assumeDfsServiceType() {
+    Assume.assumeTrue("DFS service type is required for this test",
+        getAbfsServiceType() == AbfsServiceType.DFS);
+  }
+
+  /**
+   * Utility method to assume that the test is running against a HNS Enabled account.
+   * Otherwise, the test will be skipped.
+   * @throws IOException if an error occurs while checking the account type.
+   */
+  protected void assumeHnsEnabled() throws IOException {
+    assumeHnsEnabled("HNS-Enabled account must be used for this test");
+  }
+
+  /**
+   * Utility method to assume that the test is running against a HNS Enabled account.
+   * @param errorMessage error message to be displayed if the test is skipped.
+   * @throws IOException if an error occurs while checking the account type.
+   */
+  protected void assumeHnsEnabled(String errorMessage) throws IOException {
+    Assume.assumeTrue(errorMessage, getIsNamespaceEnabled(getFileSystem()));
+  }
+
+  /**
+   * Utility method to assume that the test is running against a HNS Disabled account.
+   * Otherwise, the test will be skipped.
+   * @throws IOException if an error occurs while checking the account type.
+   */
+  protected void assumeHnsDisabled() throws IOException {
+    assumeHnsDisabled("HNS-Enabled account must not be used for this test");
+  }
+
+  /**
+   * Utility method to assume that the test is running against a HNS Disabled account.
+   * @param message error message to be displayed if the test is skipped.
+   * @throws IOException if an error occurs while checking the account type.
+   */
+  protected void assumeHnsDisabled(String message) throws IOException {
+    Assume.assumeFalse(message, getIsNamespaceEnabled(getFileSystem()));
+  }
+
+  /**
+   * Assert that the path contains the expected DNS suffix.
+   * If service type is blob, then path should have blob domain name.
+   * @param path to be asserted.
+   */
+  protected void assertPathDns(Path path) {
+    String expectedDns = getAbfsServiceType() == AbfsServiceType.BLOB
+        ? ABFS_BLOB_DOMAIN_NAME : ABFS_DFS_DOMAIN_NAME;
+    Assertions.assertThat(path.toString())
+        .describedAs("Path does not contain expected DNS")
+        .contains(expectedDns);
+  }
 }

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

@@ -225,6 +225,7 @@ public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTe
 
   private void verifyIteratorResultContent(FileStatus fileStatus,
       List<String> fileNames) {
+    assertPathDns(fileStatus.getPath());
     String pathStr = fileStatus.getPath().toString();
     Assert.assertTrue(
         String.format("Could not remove path %s from filenames %s", pathStr,

+ 4 - 7
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java

@@ -341,9 +341,7 @@ public class ITestAzureBlobFileSystemAppend extends
   @Test
   public void testCreateAppendBlobOverDfsEndpointAppendOverBlob()
       throws IOException, NoSuchFieldException, IllegalAccessException {
-    Assume.assumeTrue(
-        "FNS does not support append blob creation for DFS endpoint",
-        getIsNamespaceEnabled(getFileSystem()));
+    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());
@@ -777,7 +775,7 @@ public class ITestAzureBlobFileSystemAppend extends
     Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
     try (AzureBlobFileSystem fs = Mockito.spy(
         (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) {
-      Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+      assumeHnsDisabled();
       AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
       assumeBlobServiceType();
 
@@ -982,8 +980,7 @@ public class ITestAzureBlobFileSystemAppend extends
     // Create a spy of AzureBlobFileSystem
     try (AzureBlobFileSystem fs = Mockito.spy(
         (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) {
-      Assume.assumeTrue(!getIsNamespaceEnabled(fs));
-
+      assumeHnsDisabled();
       // Create a spy of AzureBlobFileSystemStore
       AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
       assumeBlobServiceType();
@@ -1080,7 +1077,7 @@ public class ITestAzureBlobFileSystemAppend extends
     // Create a spy of AzureBlobFileSystem
     try (AzureBlobFileSystem fs = Mockito.spy(
         (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) {
-      Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+      assumeHnsDisabled();
 
       // Create a spy of AzureBlobFileSystemStore
       AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());

+ 130 - 54
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.azurebfs;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.EnumSet;
 
@@ -28,9 +29,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.utils.DirectoryStateHelper;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 
 import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
+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;
 
@@ -56,16 +59,15 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
   @Test
   public void testSetGetXAttrCreateReplace() throws Exception {
     AzureBlobFileSystem fs = getFileSystem();
-    byte[] attributeValue = fs.getAbfsStore().encodeAttribute("one");
+    String decodedAttributeValue =  "one";
+    byte[] attributeValue = fs.getAbfsStore().encodeAttribute(decodedAttributeValue);
     String attributeName = "user.someAttribute";
     Path testFile = path(getMethodName());
 
     // after creating a file, it must be possible to create a new xAttr
     touch(testFile);
     fs.setXAttr(testFile, attributeName, attributeValue, CREATE_FLAG);
-    Assertions.assertThat(fs.getXAttr(testFile, attributeName))
-        .describedAs("Retrieved Attribute Value is Not as Expected")
-        .containsExactly(attributeValue);
+    assertAttributeEqual(fs.getXAttr(testFile, attributeName), attributeValue, decodedAttributeValue);
 
     // however after the xAttr is created, creating it again must fail
     intercept(IOException.class, () -> fs.setXAttr(testFile, attributeName, attributeValue, CREATE_FLAG));
@@ -74,8 +76,10 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
   @Test
   public void testSetGetXAttrReplace() throws Exception {
     AzureBlobFileSystem fs = getFileSystem();
-    byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute("one");
-    byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute("two");
+    String decodedAttribute1 = "one";
+    String decodedAttribute2 = "two";
+    byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute(decodedAttribute1);
+    byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute(decodedAttribute2);
     String attributeName = "user.someAttribute";
     Path testFile = path(getMethodName());
 
@@ -88,9 +92,108 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
     // however after the xAttr is created, replacing it must succeed
     fs.setXAttr(testFile, attributeName, attributeValue1, CREATE_FLAG);
     fs.setXAttr(testFile, attributeName, attributeValue2, REPLACE_FLAG);
-    Assertions.assertThat(fs.getXAttr(testFile, attributeName))
-        .describedAs("Retrieved Attribute Value is Not as Expected")
-        .containsExactly(attributeValue2);
+    assertAttributeEqual(fs.getXAttr(testFile, attributeName), attributeValue2,
+        decodedAttribute2);
+  }
+
+  /**
+   * Test get and set xattr fails fine on root.
+   * @throws Exception if test fails
+   */
+  @Test
+  public void testGetSetXAttrOnRoot() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    String attributeName = "user.attribute1";
+    byte[] attributeValue = fs.getAbfsStore().encodeAttribute("hi");
+    final Path testPath = new Path(ROOT_PATH);
+
+    AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, () -> {
+      fs.getXAttr(testPath, attributeName);
+    });
+
+    Assertions.assertThat(ex.getStatusCode())
+        .describedAs("GetXAttr() on root should fail with Bad Request")
+        .isEqualTo(HTTP_BAD_REQUEST);
+
+    ex = intercept(AbfsRestOperationException.class, () -> {
+      fs.setXAttr(testPath, attributeName, attributeValue, CREATE_FLAG);
+    });
+
+    Assertions.assertThat(ex.getStatusCode())
+        .describedAs("SetXAttr() on root should fail with Bad Request")
+        .isEqualTo(HTTP_BAD_REQUEST);
+  }
+
+  /**
+   * Test get and set xattr works fine on marker blobs for directory.
+   * @throws Exception if test fails
+   */
+  @Test
+  public void testGetSetXAttrOnExplicitDir() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    final Path testPath = path(getMethodName());
+    fs.mkdirs(testPath);
+    testGetSetXAttrHelper(fs, testPath);
+
+    // Assert that the folder is now explicit
+    DirectoryStateHelper.isExplicitDirectory(testPath, fs, getTestTracingContext(fs, true));
+  }
+
+  /**
+   * Test get and set xattr fails fine on non-existing path.
+   * @throws Exception if test fails
+   */
+  @Test
+  public void testGetSetXAttrOnNonExistingPath() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    final Path testPath = path(getMethodName());
+    intercept(FileNotFoundException.class, String.valueOf(HTTP_NOT_FOUND),
+        "get/set XAttr() should fail for non-existing files",
+        () -> testGetSetXAttrHelper(fs, testPath));
+  }
+
+  /**
+   * Trying to set same attribute multiple times should result in no failure
+   * @throws Exception if test fails
+   */
+  @Test
+  public void testSetXAttrMultipleOperations() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    final Path path = path(getMethodName());
+    fs.create(path);
+
+    String attributeName = "user.attribute1";
+    String decodedAttributeValue = "hi";
+    byte[] attributeValue = fs.getAbfsStore().encodeAttribute(decodedAttributeValue);
+
+    // Attribute not present initially
+    assertAttributeNull(fs.getXAttr(path, attributeName));
+
+    // Set the Attributes Multiple times
+    // Filesystem internally adds create and replace flags
+    fs.setXAttr(path, attributeName, attributeValue);
+    fs.setXAttr(path, attributeName, attributeValue);
+
+    // Check if the attribute is retrievable
+    byte[] rv = fs.getXAttr(path, attributeName);
+    assertAttributeEqual(rv, attributeValue, decodedAttributeValue);
+  }
+
+  /**
+   * Test get and set xattr works fine on implicit directory and ends up creating marker blobs.
+   * @throws Exception if test fails
+   */
+  @Test
+  public void testGetSetXAttrOnImplicitDir() throws Exception {
+    assumeBlobServiceType();
+    AzureBlobFileSystem fs = getFileSystem();
+    final Path testPath = path(getMethodName());
+    createAzCopyFolder(testPath);
+    testGetSetXAttrHelper(fs, testPath);
+
+    // Assert that the folder is now explicit
+    DirectoryStateHelper.isExplicitDirectory(testPath, fs, getTestTracingContext(fs, true));
+    DirectoryStateHelper.isExplicitDirectory(testPath.getParent(), fs, getTestTracingContext(fs, true));
   }
 
   private void testGetSetXAttrHelper(final AzureBlobFileSystem fs,
@@ -104,12 +207,8 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
     byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute(decodedAttributeValue2);
 
     // Attribute not present initially
-    Assertions.assertThat(fs.getXAttr(testPath, attributeName1))
-        .describedAs("Cannot get attribute before setting it")
-        .isNull();
-    Assertions.assertThat(fs.getXAttr(testPath, attributeName2))
-        .describedAs("Cannot get attribute before setting it")
-        .isNull();
+    assertAttributeNull(fs.getXAttr(testPath, attributeName1));
+    assertAttributeNull(fs.getXAttr(testPath, attributeName2));
 
     // Set the Attributes
     fs.registerListener(
@@ -121,12 +220,7 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
     // Check if the attribute is retrievable
     fs.setListenerOperation(FSOperationType.GET_ATTR);
     byte[] rv = fs.getXAttr(testPath, attributeName1);
-    Assertions.assertThat(rv)
-        .describedAs("Retrieved Attribute Does not Matches in Encoded Form")
-        .containsExactly(attributeValue1);
-    Assertions.assertThat(fs.getAbfsStore().decodeAttribute(rv))
-        .describedAs("Retrieved Attribute Does not Matches in Decoded Form")
-        .isEqualTo(decodedAttributeValue1);
+    assertAttributeEqual(rv, attributeValue1, decodedAttributeValue1);
     fs.registerListener(null);
 
     // Set the second Attribute
@@ -134,43 +228,25 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
 
     // Check all the attributes present and previous Attribute not overridden
     rv = fs.getXAttr(testPath, attributeName1);
-    Assertions.assertThat(rv)
-        .describedAs("Retrieved Attribute Does not Matches in Encoded Form")
-        .containsExactly(attributeValue1);
-    Assertions.assertThat(fs.getAbfsStore().decodeAttribute(rv))
-        .describedAs("Retrieved Attribute Does not Matches in Decoded Form")
-        .isEqualTo(decodedAttributeValue1);
+    assertAttributeEqual(rv, attributeValue1, decodedAttributeValue1);
 
     rv = fs.getXAttr(testPath, attributeName2);
-    Assertions.assertThat(rv)
-        .describedAs("Retrieved Attribute Does not Matches in Encoded Form")
-        .containsExactly(attributeValue2);
-    Assertions.assertThat(fs.getAbfsStore().decodeAttribute(rv))
-        .describedAs("Retrieved Attribute Does not Matches in Decoded Form")
-        .isEqualTo(decodedAttributeValue2);
+    assertAttributeEqual(rv, attributeValue2, decodedAttributeValue2);
   }
 
-  @Test
-  public void testGetSetXAttrOnRoot() throws Exception {
-    AzureBlobFileSystem fs = getFileSystem();
-    String attributeName = "user.attribute1";
-    byte[] attributeValue = fs.getAbfsStore().encodeAttribute("hi");
-    final Path testPath = new Path(ROOT_PATH);
-
-    AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, () -> {
-      fs.getXAttr(testPath, attributeName);
-    });
-
-    Assertions.assertThat(ex.getStatusCode())
-        .describedAs("GetXAttr() on root should fail with Bad Request")
-        .isEqualTo(HTTP_BAD_REQUEST);
-
-    ex = intercept(AbfsRestOperationException.class, () -> {
-      fs.setXAttr(testPath, attributeName, attributeValue, CREATE_FLAG);
-    });
+  private void assertAttributeNull(byte[] rv) {
+    Assertions.assertThat(rv)
+        .describedAs("Cannot get attribute before setting it")
+        .isNull();
+  }
 
-    Assertions.assertThat(ex.getStatusCode())
-        .describedAs("SetXAttr() on root should fail with Bad Request")
-        .isEqualTo(HTTP_BAD_REQUEST);
+  private void assertAttributeEqual(byte[] rv, byte[] attributeValue,
+      String decodedAttributeValue) throws Exception {
+    Assertions.assertThat(rv)
+        .describedAs("Retrieved Attribute Does not Matches in Encoded Form")
+        .containsExactly(attributeValue);
+    Assertions.assertThat(getFileSystem().getAbfsStore().decodeAttribute(rv))
+        .describedAs("Retrieved Attribute Does not Matches in Decoded Form")
+        .isEqualTo(decodedAttributeValue);
   }
 }

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

@@ -26,7 +26,6 @@ import org.junit.Test;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException;
 import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider;
@@ -93,7 +92,8 @@ public class ITestAzureBlobFileSystemChooseSAS extends AbstractAbfsIntegrationTe
    */
   @Test
   public void testBothProviderFixedTokenConfigured() throws Exception {
-    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS && getIsNamespaceEnabled(getFileSystem()));
+    assumeDfsServiceType();
+    assumeHnsEnabled();
     AbfsConfiguration testAbfsConfig = new AbfsConfiguration(
         getRawConfiguration(), this.getAccountName());
     removeAnyPresetConfiguration(testAbfsConfig);

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

@@ -37,7 +37,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.StorageErrorResponseSchema;
@@ -240,7 +239,7 @@ public class ITestAzureBlobFileSystemDelete extends
      * endpoint, the orchestration would be done by the client. The idempotency
      * issue would not happen for blob endpoint.
      */
-    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
+    assumeDfsServiceType();
     AbfsClient client = ITestAbfsClient.createTestClientFromCurrentContext(
         fs.getAbfsStore().getClient(),
         this.getConfiguration());

+ 122 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java

@@ -18,16 +18,24 @@
 
 package org.apache.hadoop.fs.azurebfs;
 
+
+import java.io.FileNotFoundException;
 import java.io.IOException;
 
-import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
 import org.apache.hadoop.fs.permission.FsPermission;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
 
 /**
  * Test FileStatus.
@@ -58,7 +66,7 @@ public class ITestAzureBlobFileSystemFileStatus extends
   @Test
   public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE);
+    fs.getConf().set(FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE);
     Path testFile = path(TEST_FILE);
     touch(testFile);
     validateStatus(fs, testFile, false);
@@ -97,7 +105,7 @@ public class ITestAzureBlobFileSystemFileStatus extends
   @Test
   public void testFolderStatusPermissionsAndOwnerAndGroup() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
-    fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE);
+    fs.getConf().set(FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE);
     Path testFolder = path(TEST_FOLDER);
     fs.mkdirs(testFolder);
 
@@ -147,6 +155,11 @@ public class ITestAzureBlobFileSystemFileStatus extends
         createEndTime > lastModifiedTime);
   }
 
+  /**
+   * Test to verify fs.listStatus() works as expected on root path
+   * across account types and endpoints configured.
+   * @throws IOException if test fails
+   */
   @Test
   public void testFileStatusOnRoot() throws IOException {
     AzureBlobFileSystem fs = getFileSystem();
@@ -162,6 +175,112 @@ public class ITestAzureBlobFileSystemFileStatus extends
     // Assert that passing absolute root path without "/" works
     testPathStr = testPathStr.substring(0, testPathStr.length() - 1);
     validateStatus(fs, new Path(testPathStr), true);
+  }
+
+  /**
+   * Test to verify fs.getFileStatus() works as expected on explicit paths as expected.
+   * Explicit path can exist as a directory as well as a file.
+   * @throws IOException if test fails
+   */
+  @Test
+  public void testFileStatusOnExplicitPath() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    Path explicitDirPath = path("explicitDir");
+    Path filePath = new Path(explicitDirPath, "explicitFile");
+    Path nonExistingPath = new Path(explicitDirPath, "nonExistingFile");
+
+    fs.mkdirs(explicitDirPath);
+    fs.create(filePath).close();
+
+    // Test File Status on explicit dir path.
+    FileStatus fileStatus = fs.getFileStatus(explicitDirPath);
+    verifyFileStatus(fileStatus, true);
+
+    // Test File Status on file with explicit parent.
+    fileStatus = fs.getFileStatus(filePath);
+    verifyFileStatus(fileStatus, false);
+
+    // Test File Status non-existing file with explicit parent.
+    FileNotFoundException ex = intercept(FileNotFoundException.class, () -> {
+      fs.getFileStatus(nonExistingPath);
+    });
+    verifyFileNotFound(ex, nonExistingPath.getName());
+  }
+
+  /**
+   * Test to verify fs.getFileStatus() works as expected on implicit paths as expected.
+   * Implicit path can exist as a directory only in HNS-Disabled Accounts.
+   * @throws Exception
+   */
+  @Test
+  public void testFileStatusOnImplicitPath() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    Path filePath = path("implicitDir/fileWithImplicitParent");
+    Path implicitDir = filePath.getParent();
+    Path nonExistingPath = new Path(implicitDir, "nonExistingFile");
+
+    createAzCopyFile(filePath);
+
+    // Test File Status on implicit dir parent.
+    FileStatus fileStatus = fs.getFileStatus(implicitDir);
+    verifyFileStatus(fileStatus, true);
+
+    // Test File Status on file with implicit parent.
+    fileStatus = fs.getFileStatus(filePath);
+    verifyFileStatus(fileStatus, false);
+
+    // Test File Status on non-existing file with implicit parent.
+    FileNotFoundException ex = intercept(FileNotFoundException.class, () -> {
+      fs.getFileStatus(nonExistingPath);
+    });
+    verifyFileNotFound(ex, nonExistingPath.getName());
+  }
+
+  /**
+   * Test to verify fs.getFileStatus() need to internally call listStatus on path.
+   * @throws Exception if test fails
+   */
+  @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();
+
+    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(), eq(false));
+  }
+
+  /**
+   * Verifies the file status indicates a file present in the path.
+   * @param fileStatus
+   * @param isDir
+   */
+  private void verifyFileStatus(FileStatus fileStatus, boolean isDir) {
+    Assertions.assertThat(fileStatus).isNotNull();
+    if (isDir) {
+      Assertions.assertThat(fileStatus.getLen()).isEqualTo(0);
+      Assertions.assertThat(fileStatus.isDirectory()).isTrue();
+    } else {
+      Assertions.assertThat(fileStatus.isFile()).isTrue();
+    }
+    assertPathDns(fileStatus.getPath());
+  }
 
+  /**
+   * Verifies the file not found exception is thrown with the expected message.
+   * @param ex
+   * @param key
+   */
+  private void verifyFileNotFound(FileNotFoundException ex, String key) {
+    Assertions.assertThat(ex).isNotNull();
+    Assertions.assertThat(ex.getMessage()).contains(key);
   }
 }

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

@@ -33,12 +33,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
 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.Assume;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -313,7 +311,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
     AzureBlobFileSystem fs1 = getFileSystem();
     // AppendBlob not supported for FNS over DFS endpoint.
     if (!getIsNamespaceEnabled(fs1)) {
-      Assume.assumeTrue(getAbfsStore(fs1).getClient() instanceof AbfsBlobClient);
+      assumeBlobServiceType();
     }
     Configuration config = new Configuration(this.getRawConfiguration());
     config.set(FS_AZURE_APPEND_BLOB_KEY, "abfss:/");

+ 0 - 23
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.io.FileNotFoundException;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,7 +31,6 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException;
 import org.apache.hadoop.fs.azurebfs.enums.Trilean;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
@@ -40,12 +38,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 
 import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
 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_KEY_PROPERTY_NAME;
-import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.accountProperty;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DFS_DOMAIN_NAME;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.mockito.ArgumentMatchers.any;
 
@@ -115,22 +108,6 @@ public class ITestAzureBlobFileSystemInitAndCreate extends
         .getAclStatus(Mockito.anyString(), any(TracingContext.class));
   }
 
-  // TODO: [FnsOverBlob][HADOOP-19179] Remove this test case once Blob Endpoint Support is enabled.
-  @Test
-  public void testFileSystemInitFailsWithBlobEndpointUrl() throws Exception {
-    Configuration configuration = new Configuration(getRawConfiguration());
-    String defaultUri = configuration.get(FS_DEFAULT_NAME_KEY);
-    String accountKey = configuration.get(
-        accountProperty(FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, getAccountName()),
-        configuration.get(FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME));
-    configuration.set(FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME,
-        accountKey.replace(ABFS_DFS_DOMAIN_NAME, ABFS_BLOB_DOMAIN_NAME));
-    String blobUri = defaultUri.replace(ABFS_DFS_DOMAIN_NAME, ABFS_BLOB_DOMAIN_NAME);
-    intercept(InvalidConfigurationValueException.class,
-        "Blob Endpoint Support not yet available", () ->
-            FileSystem.newInstance(new Path(blobUri).toUri(), configuration));
-  }
-
   @Test
   public void testFileSystemInitFailsIfNotAbleToDetermineAccountType() throws Exception {
     AzureBlobFileSystem fs = ((AzureBlobFileSystem) FileSystem.newInstance(

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

@@ -124,7 +124,7 @@ public class ITestAzureBlobFileSystemListStatus extends
    */
   @Test
   public void testListPathTracingContext() throws Exception {
-    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
+    assumeDfsServiceType();
     final AzureBlobFileSystem fs = getFileSystem();
     final AzureBlobFileSystem spiedFs = Mockito.spy(fs);
     final AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore());

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

@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contract;
+
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import org.assertj.core.api.Assertions;
+import java.util.List;
+
+import org.junit.Test;
+import org.xml.sax.SAXException;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobListResultEntrySchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobListResultSchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobListXmlParser;
+
+public class TestBlobListXmlParser {
+  @Test
+  public void testXMLParser() throws Exception {
+    String xmlResponseWithDelimiter = ""
+        + "<?xml version=\"1.0\" encoding=\"utf-8\"?>"
+        + "<EnumerationResults ServiceEndpoint=\"https://anujtestfns.blob.core.windows.net/\" ContainerName=\"manualtest\">"
+        + "<Delimiter>/</Delimiter>"
+        + "<Blobs>"
+        + "<BlobPrefix>"
+        + "<Name>bye/</Name>"
+        + "</BlobPrefix>"
+        + "<Blob>"
+        + "<Name>explicitDir</Name>"
+        + "<Properties>"
+        + "<Creation-Time>Tue, 06 Jun 2023 08:34:28 GMT</Creation-Time>"
+        + "<Last-Modified>Tue, 06 Jun 2023 08:35:00 GMT</Last-Modified>"
+        + "<Etag>0x8DB6668EAB50E67</Etag>"
+        + "<Content-Length>0</Content-Length>"
+        + "<Content-Type>application/octet-stream</Content-Type>"
+        + "<Content-Encoding />"
+        + "<Content-Language />"
+        + "<Content-CRC64 />"
+        + "<Content-MD5>1B2M2Y8AsgTpgAmY7PhCfg==</Content-MD5>"
+        + "<Cache-Control />"
+        + "<Content-Disposition />"
+        + "<BlobType>BlockBlob</BlobType>"
+        + "<AccessTier>Hot</AccessTier>"
+        + "<AccessTierInferred>true</AccessTierInferred>"
+        + "<LeaseStatus>unlocked</LeaseStatus>"
+        + "<LeaseState>available</LeaseState>"
+        + "<ServerEncrypted>true</ServerEncrypted>"
+        + "</Properties>"
+        + "<Metadata>"
+        + "<hdi_isfolder>true</hdi_isfolder>"
+        + "</Metadata>"
+        + "<OrMetadata />"
+        + "</Blob>"
+        + "<BlobPrefix>"
+        + "<Name>hello/</Name>"
+        + "</BlobPrefix>"
+        + "<Blob>"
+        + "<Name>splits.txt</Name>"
+        + "<Properties>"
+        + "<Creation-Time>Tue, 06 Jun 2023 08:33:51 GMT</Creation-Time>"
+        + "<Last-Modified>Mon, 19 Jun 2023 04:11:51 GMT</Last-Modified>"
+        + "<Etag>0x8DB707B4F77B831</Etag>"
+        + "<Content-Length>67681</Content-Length>"
+        + "<Content-Type>text/plain</Content-Type>"
+        + "<Content-Encoding />"
+        + "<Content-Language />"
+        + "<Content-CRC64 />"
+        + "<Content-MD5>If+Z0+KfGsXjvdpT1Z66NQ==</Content-MD5>"
+        + "<Cache-Control />"
+        + "<Content-Disposition />"
+        + "<BlobType>BlockBlob</BlobType>"
+        + "<AccessTier>Hot</AccessTier>"
+        + "<AccessTierInferred>true</AccessTierInferred>"
+        + "<LeaseStatus>unlocked</LeaseStatus>"
+        + "<LeaseState>available</LeaseState>"
+        + "<ServerEncrypted>true</ServerEncrypted>"
+        + "</Properties>"
+        + "<Metadata>"
+        + "<hello>hi</hello>"
+        + "<unicodeAttribute>%D0%91%d0%BB%D1%8E%D0%B7</unicodeAttribute>"
+        + "</Metadata>"
+        + "<OrMetadata />"
+        + "</Blob>"
+        + "</Blobs>"
+        + "<NextMarker />"
+        + "</EnumerationResults>";
+    BlobListResultSchema listResultSchema = getResultSchema(xmlResponseWithDelimiter);
+    List<BlobListResultEntrySchema> paths = listResultSchema.paths();
+    Assertions.assertThat(paths.size()).isEqualTo(4);
+    Assertions.assertThat(paths.get(0).isDirectory()).isEqualTo(true);
+    Assertions.assertThat(paths.get(1).isDirectory()).isEqualTo(true);
+    Assertions.assertThat(paths.get(2).isDirectory()).isEqualTo(true);
+    Assertions.assertThat(paths.get(3).isDirectory()).isEqualTo(false);
+  }
+
+  @Test
+  public void testEmptyBlobList() throws Exception {
+    String xmlResponse = ""
+        + "<?xml version=\"1.0\" encoding=\"utf-8\"?><"
+        + "EnumerationResults ServiceEndpoint=\"https://anujtestfns.blob.core.windows.net/\" ContainerName=\"manualtest\">"
+        + "<Prefix>abc/</Prefix>"
+        + "<Delimiter>/</Delimiter>"
+        + "<Blobs /><NextMarker />"
+        + "</EnumerationResults>";
+    BlobListResultSchema listResultSchema = getResultSchema(xmlResponse);
+    List<BlobListResultEntrySchema> paths = listResultSchema.paths();
+    Assertions.assertThat(paths.size()).isEqualTo(0);
+  }
+
+  private static final ThreadLocal<SAXParser> SAX_PARSER_THREAD_LOCAL
+      = new ThreadLocal<SAXParser>() {
+    @Override
+    public SAXParser initialValue() {
+      SAXParserFactory factory = SAXParserFactory.newInstance();
+      factory.setNamespaceAware(true);
+      try {
+        return factory.newSAXParser();
+      } catch (SAXException e) {
+        throw new RuntimeException("Unable to create SAXParser", e);
+      } catch (ParserConfigurationException e) {
+        throw new RuntimeException("Check parser configuration", e);
+      }
+    }
+  };
+
+  private BlobListResultSchema getResultSchema(String xmlResponse) throws Exception {
+    byte[] bytes = xmlResponse.getBytes();
+    final InputStream stream = new ByteArrayInputStream(bytes);
+    final SAXParser saxParser = SAX_PARSER_THREAD_LOCAL.get();
+    saxParser.reset();
+    BlobListResultSchema listResultSchema = new BlobListResultSchema();
+    saxParser.parse(stream, new BlobListXmlParser(listResultSchema, "https://sample.url"));
+    return listResultSchema;
+  }
+}

+ 6 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ListResultSchemaTest.java → hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/TestDfsListJsonParser.java

@@ -23,15 +23,15 @@ import java.io.IOException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.junit.Test;
 
+import org.apache.hadoop.fs.azurebfs.contracts.services.DfsListResultSchema;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
-import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Tests the JSON parsing for the listfilestatus response to ListResultSchema
  */
-public class ListResultSchemaTest {
+public class TestDfsListJsonParser {
 
   /**
    * Test parsing a JSON which matches the properties in the ListResultSchema
@@ -62,8 +62,8 @@ public class ListResultSchemaTest {
             + "\"rw-r--r--\" } ] } ";
 
     final ObjectMapper objectMapper = new ObjectMapper();
-    final ListResultSchema listResultSchema = objectMapper
-        .readValue(matchingJson, ListResultSchema.class);
+    final DfsListResultSchema listResultSchema = objectMapper
+        .readValue(matchingJson, DfsListResultSchema.class);
 
     assertThat(listResultSchema.paths().size())
         .describedAs("Only one path is expected as present in the input JSON")
@@ -123,8 +123,8 @@ public class ListResultSchemaTest {
         + "\"rw-r--r--\" } ] } ";
 
     final ObjectMapper objectMapper = new ObjectMapper();
-    final ListResultSchema listResultSchema = objectMapper
-        .readValue(matchingJson, ListResultSchema.class);
+    final DfsListResultSchema listResultSchema = objectMapper
+        .readValue(matchingJson, DfsListResultSchema.class);
 
     assertThat(listResultSchema.paths().size())
         .describedAs("Only one path is expected as present in the input JSON")

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

@@ -42,6 +42,17 @@ public class ITestAbfsClientHandler extends AbstractAbfsIntegrationTest {
   public void testAbfsClientHandlerInitialization() throws Exception {
     AzureBlobFileSystem fs = getFileSystem();
     AbfsClientHandler clientHandler = fs.getAbfsStore().getClientHandler();
+    if (getAbfsServiceType() == AbfsServiceType.DFS) {
+      Assertions.assertThat(clientHandler.getClient()).isInstanceOf(AbfsDfsClient.class);
+    } else {
+      Assertions.assertThat(clientHandler.getClient()).isInstanceOf(AbfsBlobClient.class);
+    }
+    if (getIngressServiceType() == AbfsServiceType.DFS) {
+      Assertions.assertThat(clientHandler.getIngressClient()).isInstanceOf(AbfsDfsClient.class);
+    } else {
+      Assertions.assertThat(clientHandler.getIngressClient())
+          .isInstanceOf(AbfsBlobClient.class);
+    }
     Assertions.assertThat(clientHandler.getClient(AbfsServiceType.DFS)).isInstanceOf(AbfsDfsClient.class);
     Assertions.assertThat(clientHandler.getClient(AbfsServiceType.BLOB)).isInstanceOf(AbfsBlobClient.class);
   }

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

@@ -350,7 +350,7 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
     AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
     Assume.assumeTrue(!getIsNamespaceEnabled(fs));
     AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
-    Assume.assumeTrue(store.getClient() instanceof AbfsBlobClient);
+    assumeBlobServiceType();
 
     // Mock the clientHandler to return the blobClient when getBlobClient is called
     AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
@@ -393,7 +393,7 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
     AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
     Assume.assumeTrue(!getIsNamespaceEnabled(fs));
     AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
-    Assume.assumeTrue(store.getClient() instanceof AbfsBlobClient);
+    assumeBlobServiceType();
     Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
 
     // Step 2: Mock the clientHandler to return the blobClient when getBlobClient is called

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

@@ -34,7 +34,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider;
 import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers;
@@ -168,7 +167,7 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
    */
   @Test
   public void testNonRecursiveDeleteWithPagination() throws Exception {
-    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
+    assumeDfsServiceType();
     testNonRecursiveDeleteWithPaginationInternal(true);
     testNonRecursiveDeleteWithPaginationInternal(false);
   }

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

@@ -39,7 +39,6 @@ import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
 import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
@@ -440,7 +439,7 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
     Assume.assumeTrue(!isNamespaceEnabled);
     // In DFS endpoint, renamePath is O(1) API call and idempotency issue can happen.
     // For blob endpoint, client orchestrates the rename operation.
-    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
+    assumeDfsServiceType();
     AzureBlobFileSystem fs = getFileSystem();
     AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
     TracingContext testTracingContext = getTestTracingContext(fs, false);