瀏覽代碼

HADOOP-19232: [ABFS][FNSOverBlob] Implementing Ingress Support with various Fallback Handling (#7272)

Contributed by Anmol Asrani
Anmol Asrani 5 月之前
父節點
當前提交
c7e1b66087
共有 65 個文件被更改,包括 6429 次插入376 次删除
  1. 14 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
  2. 28 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
  3. 56 24
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
  4. 31 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
  5. 5 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
  6. 6 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
  7. 51 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidIngressServiceException.java
  8. 46 8
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
  9. 2 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
  10. 8 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/BlobAppendRequestParameters.java
  11. 2 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java
  12. 72 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
  13. 271 16
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
  14. 160 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlock.java
  15. 47 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlockStatus.java
  16. 19 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
  17. 29 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java
  18. 16 3
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java
  19. 1 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
  20. 493 121
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
  21. 59 8
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java
  22. 2 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java
  23. 194 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
  24. 348 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
  25. 170 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java
  26. 89 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
  27. 280 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
  28. 265 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDfsToBlobIngressFallbackHandler.java
  29. 209 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
  30. 86 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlockEntry.java
  31. 2 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java
  32. 51 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java
  33. 6 1
      hadoop-tools/hadoop-azure/src/site/markdown/blobEndpoint.md
  34. 65 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
  35. 33 5
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java
  36. 48 11
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java
  37. 91 42
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java
  38. 1056 30
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
  39. 37 11
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
  40. 2 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java
  41. 915 16
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
  42. 14 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
  43. 7 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
  44. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
  45. 34 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java
  46. 27 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java
  47. 3 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java
  48. 1 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
  49. 5 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java
  50. 16 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java
  51. 18 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java
  52. 9 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
  53. 1 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
  54. 87 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
  55. 1 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java
  56. 41 18
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpClientRequestExecutor.java
  57. 180 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
  58. 7 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsPaginatedDelete.java
  59. 2 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java
  60. 37 19
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
  61. 38 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AzcopyExecutionException.java
  62. 319 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AzcopyToolHelper.java
  63. 118 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DirectoryStateHelper.java
  64. 78 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestAzcopyToolHelper.java
  65. 20 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java

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

@@ -1462,6 +1462,20 @@ public class AbfsConfiguration{
     this.isNamespaceEnabledAccount = isNamespaceEnabledAccount;
     this.isNamespaceEnabledAccount = isNamespaceEnabledAccount;
   }
   }
 
 
+  /**
+   * Checks if the FixedSASTokenProvider is configured for the current account.
+   *
+   * @return true if the FixedSASTokenProvider is configured, false otherwise.
+   */
+  public boolean isFixedSASTokenProviderConfigured() {
+    try {
+      return getSASTokenProvider() instanceof FixedSASTokenProvider;
+    } catch (AzureBlobFileSystemException e) {
+      LOG.debug("Failed to get SAS token provider", e);
+      return false;
+    }
+  }
+
   private String getTrimmedPasswordString(String key, String defaultValue) throws IOException {
   private String getTrimmedPasswordString(String key, String defaultValue) throws IOException {
     String value = getPasswordString(key);
     String value = getPasswordString(key);
     if (StringUtils.isBlank(value)) {
     if (StringUtils.isBlank(value)) {

+ 28 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

@@ -46,6 +46,7 @@ import javax.annotation.Nullable;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.impl.BackReference;
 import org.apache.hadoop.fs.impl.BackReference;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.Preconditions;
@@ -122,11 +123,13 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLO
 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_IS_HNS_ENABLED;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_ACTIVE_BLOCKS;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_ACTIVE_BLOCKS;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_BUFFER_DIR;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_BUFFER_DIR;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_FIXED_TOKEN;
 import static org.apache.hadoop.fs.azurebfs.constants.FSOperationType.CREATE_FILESYSTEM;
 import static org.apache.hadoop.fs.azurebfs.constants.FSOperationType.CREATE_FILESYSTEM;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DATA_BLOCKS_BUFFER_DEFAULT;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DATA_BLOCKS_BUFFER_DEFAULT;
 import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD;
 import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD;
 import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_CREATE_ON_ROOT;
 import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_CREATE_ON_ROOT;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.UNAUTHORIZED_SAS;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
 import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel;
 import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel;
 import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
 import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
@@ -234,6 +237,29 @@ public class AzureBlobFileSystem extends FileSystem
       throw new InvalidConfigurationValueException(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, ex);
       throw new InvalidConfigurationValueException(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, ex);
     }
     }
 
 
+    /*
+     * Validates if the correct SAS Token provider is configured for non-HNS accounts.
+     * For non-HNS accounts, if the authentication type is set to SAS, only a fixed SAS Token is supported as of now.
+     * A custom SAS Token Provider should not be configured in such cases, as it will override the FixedSASTokenProvider and render it unused.
+     * If the namespace is not enabled and the FixedSASTokenProvider is not configured,
+     * an InvalidConfigurationValueException will be thrown.
+     *
+     * @throws InvalidConfigurationValueException if account is not namespace enabled and FixedSASTokenProvider is not configured.
+     */
+    try {
+      if (abfsConfiguration.getAuthType(abfsConfiguration.getAccountName()) == AuthType.SAS && // Auth type is SAS
+          !tryGetIsNamespaceEnabled(new TracingContext(initFSTracingContext)) && // Account is FNS
+          !abfsConfiguration.isFixedSASTokenProviderConfigured()) { // Fixed SAS Token Provider is not configured
+        throw new InvalidConfigurationValueException(FS_AZURE_SAS_FIXED_TOKEN, UNAUTHORIZED_SAS);
+      }
+    } catch (InvalidConfigurationValueException ex) {
+      LOG.error("File system configured with Invalid SAS Token Provider for FNS Accounts", ex);
+      throw ex;
+    } catch (AzureBlobFileSystemException ex) {
+      LOG.error("Failed to determine account type for auth type validation", ex);
+      throw new InvalidConfigurationValueException(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, ex);
+    }
+
     /*
     /*
      * Non-hierarchical-namespace account can not have a customer-provided-key(CPK).
      * Non-hierarchical-namespace account can not have a customer-provided-key(CPK).
      * Fail initialization of filesystem if the configs are provided. CPK is of
      * Fail initialization of filesystem if the configs are provided. CPK is of
@@ -266,6 +292,7 @@ public class AzureBlobFileSystem extends FileSystem
         }
         }
       }
       }
     }
     }
+    getAbfsStore().updateClientWithNamespaceInfo(new TracingContext(initFSTracingContext));
 
 
     LOG.trace("Initiate check for delegation token manager");
     LOG.trace("Initiate check for delegation token manager");
     if (UserGroupInformation.isSecurityEnabled()) {
     if (UserGroupInformation.isSecurityEnabled()) {
@@ -797,7 +824,7 @@ public class AzureBlobFileSystem extends FileSystem
     Path qualifiedPath = makeQualified(path);
     Path qualifiedPath = makeQualified(path);
 
 
     try {
     try {
-      return abfsStore.getFileStatus(qualifiedPath, tracingContext);
+      return getAbfsStore().getFileStatus(qualifiedPath, tracingContext);
     } catch (AzureBlobFileSystemException ex) {
     } catch (AzureBlobFileSystemException ex) {
       checkException(path, ex);
       checkException(path, ex);
       return null;
       return null;

+ 56 - 24
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java

@@ -293,6 +293,18 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
         "abfs-bounded");
         "abfs-bounded");
   }
   }
 
 
+  /**
+   * Updates the client with the namespace information.
+   *
+   * @param tracingContext the tracing context to be used for the operation
+   * @throws AzureBlobFileSystemException if an error occurs while updating the client
+   */
+  public void updateClientWithNamespaceInfo(TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
+    AbfsClient.setIsNamespaceEnabled(isNamespaceEnabled);
+  }
+
   /**
   /**
    * Checks if the given key in Azure Storage should be stored as a page
    * Checks if the given key in Azure Storage should be stored as a page
    * blob instead of block blob.
    * blob instead of block blob.
@@ -635,14 +647,15 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       final FsPermission permission, final FsPermission umask,
       final FsPermission permission, final FsPermission umask,
       TracingContext tracingContext) throws IOException {
       TracingContext tracingContext) throws IOException {
     try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) {
     try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) {
+      AbfsClient createClient = getClientHandler().getIngressClient();
       boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
       boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
       LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}",
       LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}",
-              getClient().getFileSystem(),
-              path,
-              overwrite,
-              permission,
-              umask,
-              isNamespaceEnabled);
+          createClient.getFileSystem(),
+          path,
+          overwrite,
+          permission,
+          umask,
+          isNamespaceEnabled);
 
 
       String relativePath = getRelativePath(path);
       String relativePath = getRelativePath(path);
       boolean isAppendBlob = false;
       boolean isAppendBlob = false;
@@ -660,9 +673,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       }
       }
 
 
       final ContextEncryptionAdapter contextEncryptionAdapter;
       final ContextEncryptionAdapter contextEncryptionAdapter;
-      if (getClient().getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) {
+      if (createClient.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) {
         contextEncryptionAdapter = new ContextProviderEncryptionAdapter(
         contextEncryptionAdapter = new ContextProviderEncryptionAdapter(
-            getClient().getEncryptionContextProvider(), getRelativePath(path));
+            createClient.getEncryptionContextProvider(), getRelativePath(path));
       } else {
       } else {
         contextEncryptionAdapter = NoContextEncryptionAdapter.getInstance();
         contextEncryptionAdapter = NoContextEncryptionAdapter.getInstance();
       }
       }
@@ -677,7 +690,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
         );
         );
 
 
       } else {
       } else {
-        op = getClient().createPath(relativePath, true,
+        op = createClient.createPath(relativePath, true,
             overwrite,
             overwrite,
             new Permissions(isNamespaceEnabled, permission, umask),
             new Permissions(isNamespaceEnabled, permission, umask),
             isAppendBlob,
             isAppendBlob,
@@ -689,15 +702,16 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       perfInfo.registerResult(op.getResult()).registerSuccess(true);
       perfInfo.registerResult(op.getResult()).registerSuccess(true);
 
 
       AbfsLease lease = maybeCreateLease(relativePath, tracingContext);
       AbfsLease lease = maybeCreateLease(relativePath, tracingContext);
-
+      String eTag = extractEtagHeader(op.getResult());
       return new AbfsOutputStream(
       return new AbfsOutputStream(
           populateAbfsOutputStreamContext(
           populateAbfsOutputStreamContext(
               isAppendBlob,
               isAppendBlob,
               lease,
               lease,
-              getClient(),
+              getClientHandler(),
               statistics,
               statistics,
               relativePath,
               relativePath,
               0,
               0,
+              eTag,
               contextEncryptionAdapter,
               contextEncryptionAdapter,
               tracingContext));
               tracingContext));
     }
     }
@@ -720,12 +734,12 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       final ContextEncryptionAdapter contextEncryptionAdapter,
       final ContextEncryptionAdapter contextEncryptionAdapter,
       final TracingContext tracingContext) throws IOException {
       final TracingContext tracingContext) throws IOException {
     AbfsRestOperation op;
     AbfsRestOperation op;
-
+    AbfsClient createClient = getClientHandler().getIngressClient();
     try {
     try {
       // Trigger a create with overwrite=false first so that eTag fetch can be
       // Trigger a create with overwrite=false first so that eTag fetch can be
       // avoided for cases when no pre-existing file is present (major portion
       // avoided for cases when no pre-existing file is present (major portion
       // of create file traffic falls into the case of no pre-existing file).
       // of create file traffic falls into the case of no pre-existing file).
-      op = getClient().createPath(relativePath, true, false, permissions,
+      op = createClient.createPath(relativePath, true, false, permissions,
           isAppendBlob, null, contextEncryptionAdapter, tracingContext);
           isAppendBlob, null, contextEncryptionAdapter, tracingContext);
 
 
     } catch (AbfsRestOperationException e) {
     } catch (AbfsRestOperationException e) {
@@ -745,12 +759,11 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
           }
           }
         }
         }
 
 
-        String eTag = op.getResult()
-            .getResponseHeader(HttpHeaderConfigurations.ETAG);
+        String eTag = extractEtagHeader(op.getResult());
 
 
         try {
         try {
           // overwrite only if eTag matches with the file properties fetched befpre
           // overwrite only if eTag matches with the file properties fetched befpre
-          op = getClient().createPath(relativePath, true, true, permissions,
+          op = createClient.createPath(relativePath, true, true, permissions,
               isAppendBlob, eTag, contextEncryptionAdapter, tracingContext);
               isAppendBlob, eTag, contextEncryptionAdapter, tracingContext);
         } catch (AbfsRestOperationException ex) {
         } catch (AbfsRestOperationException ex) {
           if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) {
           if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) {
@@ -778,22 +791,24 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
    *
    *
    * @param isAppendBlob   is Append blob support enabled?
    * @param isAppendBlob   is Append blob support enabled?
    * @param lease          instance of AbfsLease for this AbfsOutputStream.
    * @param lease          instance of AbfsLease for this AbfsOutputStream.
-   * @param client         AbfsClient.
+   * @param clientHandler  AbfsClientHandler.
    * @param statistics     FileSystem statistics.
    * @param statistics     FileSystem statistics.
    * @param path           Path for AbfsOutputStream.
    * @param path           Path for AbfsOutputStream.
    * @param position       Position or offset of the file being opened, set to 0
    * @param position       Position or offset of the file being opened, set to 0
    *                       when creating a new file, but needs to be set for APPEND
    *                       when creating a new file, but needs to be set for APPEND
    *                       calls on the same file.
    *                       calls on the same file.
+   * @param eTag           eTag of the file.
    * @param tracingContext instance of TracingContext for this AbfsOutputStream.
    * @param tracingContext instance of TracingContext for this AbfsOutputStream.
    * @return AbfsOutputStreamContext instance with the desired parameters.
    * @return AbfsOutputStreamContext instance with the desired parameters.
    */
    */
   private AbfsOutputStreamContext populateAbfsOutputStreamContext(
   private AbfsOutputStreamContext populateAbfsOutputStreamContext(
       boolean isAppendBlob,
       boolean isAppendBlob,
       AbfsLease lease,
       AbfsLease lease,
-      AbfsClient client,
+      AbfsClientHandler clientHandler,
       FileSystem.Statistics statistics,
       FileSystem.Statistics statistics,
       String path,
       String path,
       long position,
       long position,
+      String eTag,
       ContextEncryptionAdapter contextEncryptionAdapter,
       ContextEncryptionAdapter contextEncryptionAdapter,
       TracingContext tracingContext) {
       TracingContext tracingContext) {
     int bufferSize = abfsConfiguration.getWriteBufferSize();
     int bufferSize = abfsConfiguration.getWriteBufferSize();
@@ -814,7 +829,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
             .withEncryptionAdapter(contextEncryptionAdapter)
             .withEncryptionAdapter(contextEncryptionAdapter)
             .withBlockFactory(getBlockFactory())
             .withBlockFactory(getBlockFactory())
             .withBlockOutputActiveBlocks(blockOutputActiveBlocks)
             .withBlockOutputActiveBlocks(blockOutputActiveBlocks)
-            .withClient(client)
+            .withClientHandler(clientHandler)
             .withPosition(position)
             .withPosition(position)
             .withFsStatistics(statistics)
             .withFsStatistics(statistics)
             .withPath(path)
             .withPath(path)
@@ -822,16 +837,30 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
                 blockOutputActiveBlocks, true))
                 blockOutputActiveBlocks, true))
             .withTracingContext(tracingContext)
             .withTracingContext(tracingContext)
             .withAbfsBackRef(fsBackRef)
             .withAbfsBackRef(fsBackRef)
+            .withIngressServiceType(abfsConfiguration.getIngressServiceType())
+            .withDFSToBlobFallbackEnabled(abfsConfiguration.isDfsToBlobFallbackEnabled())
+            .withETag(eTag)
             .build();
             .build();
   }
   }
 
 
+  /**
+   * Creates a directory.
+   *
+   * @param path Path of the directory to create.
+   * @param permission Permission of the directory.
+   * @param umask Umask of the directory.
+   * @param tracingContext tracing context
+   *
+   * @throws AzureBlobFileSystemException server error.
+   */
   public void createDirectory(final Path path, final FsPermission permission,
   public void createDirectory(final Path path, final FsPermission permission,
       final FsPermission umask, TracingContext tracingContext)
       final FsPermission umask, TracingContext tracingContext)
       throws IOException {
       throws IOException {
     try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) {
     try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) {
+      AbfsClient createClient = getClientHandler().getIngressClient();
       boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
       boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
       LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}",
       LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}",
-              getClient().getFileSystem(),
+              createClient.getFileSystem(),
               path,
               path,
               permission,
               permission,
               umask,
               umask,
@@ -841,7 +870,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
           !isNamespaceEnabled || abfsConfiguration.isEnabledMkdirOverwrite();
           !isNamespaceEnabled || abfsConfiguration.isEnabledMkdirOverwrite();
       Permissions permissions = new Permissions(isNamespaceEnabled,
       Permissions permissions = new Permissions(isNamespaceEnabled,
           permission, umask);
           permission, umask);
-      final AbfsRestOperation op = getClient().createPath(getRelativePath(path),
+      final AbfsRestOperation op = createClient.createPath(getRelativePath(path),
           false, overwrite, permissions, false, null, null, tracingContext);
           false, overwrite, permissions, false, null, null, tracingContext);
       perfInfo.registerResult(op.getResult()).registerSuccess(true);
       perfInfo.registerResult(op.getResult()).registerSuccess(true);
     }
     }
@@ -976,6 +1005,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
               overwrite);
               overwrite);
 
 
       String relativePath = getRelativePath(path);
       String relativePath = getRelativePath(path);
+      AbfsClient writeClient = getClientHandler().getIngressClient();
 
 
       final AbfsRestOperation op = getClient()
       final AbfsRestOperation op = getClient()
           .getPathStatus(relativePath, false, tracingContext, null);
           .getPathStatus(relativePath, false, tracingContext, null);
@@ -1000,8 +1030,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       }
       }
 
 
       AbfsLease lease = maybeCreateLease(relativePath, tracingContext);
       AbfsLease lease = maybeCreateLease(relativePath, tracingContext);
+      final String eTag = extractEtagHeader(op.getResult());
       final ContextEncryptionAdapter contextEncryptionAdapter;
       final ContextEncryptionAdapter contextEncryptionAdapter;
-      if (getClient().getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) {
+      if (writeClient.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) {
         final String encryptionContext = op.getResult()
         final String encryptionContext = op.getResult()
             .getResponseHeader(
             .getResponseHeader(
                 HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT);
                 HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT);
@@ -1010,7 +1041,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
               "File doesn't have encryptionContext.");
               "File doesn't have encryptionContext.");
         }
         }
         contextEncryptionAdapter = new ContextProviderEncryptionAdapter(
         contextEncryptionAdapter = new ContextProviderEncryptionAdapter(
-            getClient().getEncryptionContextProvider(), getRelativePath(path),
+            writeClient.getEncryptionContextProvider(), getRelativePath(path),
             encryptionContext.getBytes(StandardCharsets.UTF_8));
             encryptionContext.getBytes(StandardCharsets.UTF_8));
       } else {
       } else {
         contextEncryptionAdapter = NoContextEncryptionAdapter.getInstance();
         contextEncryptionAdapter = NoContextEncryptionAdapter.getInstance();
@@ -1020,10 +1051,11 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
           populateAbfsOutputStreamContext(
           populateAbfsOutputStreamContext(
               isAppendBlob,
               isAppendBlob,
               lease,
               lease,
-              getClient(),
+              getClientHandler(),
               statistics,
               statistics,
               relativePath,
               relativePath,
               offset,
               offset,
+              eTag,
               contextEncryptionAdapter,
               contextEncryptionAdapter,
               tracingContext));
               tracingContext));
     }
     }

+ 31 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java

@@ -51,6 +51,8 @@ public final class AbfsHttpConstants {
   public static final String DEFAULT_TIMEOUT = "90";
   public static final String DEFAULT_TIMEOUT = "90";
   public static final String APPEND_BLOB_TYPE = "appendblob";
   public static final String APPEND_BLOB_TYPE = "appendblob";
   public static final String LIST = "list";
   public static final String LIST = "list";
+  public static final String BLOCK_BLOB_TYPE = "BlockBlob";
+  public static final String APPEND_BLOCK = "appendblock";
 
 
   //Abfs Http Client Constants for Blob Endpoint APIs.
   //Abfs Http Client Constants for Blob Endpoint APIs.
 
 
@@ -238,7 +240,7 @@ public final class AbfsHttpConstants {
   public static final String PUT_BLOCK_LIST = "PutBlockList";
   public static final String PUT_BLOCK_LIST = "PutBlockList";
 
 
   /**
   /**
-   * Value that differentiates categories of the http_status.
+   * Value that differentiates categories of the HTTP status.
    * <pre>
    * <pre>
    * 100 - 199 : Informational responses
    * 100 - 199 : Informational responses
    * 200 - 299 : Successful responses
    * 200 - 299 : Successful responses
@@ -249,6 +251,28 @@ public final class AbfsHttpConstants {
    */
    */
   public static final Integer HTTP_STATUS_CATEGORY_QUOTIENT = 100;
   public static final Integer HTTP_STATUS_CATEGORY_QUOTIENT = 100;
 
 
+  /**
+   * XML version declaration for the block list.
+   */
+  public static final String XML_VERSION = "<?xml version=\"1.0\" encoding=\"UTF-8\"?>%n";
+
+  /**
+   * Start tag for the block list XML.
+   */
+  public static final String BLOCK_LIST_START_TAG = "<BlockList>%n";
+
+  /**
+   * End tag for the block list XML.
+   */
+  public static final String BLOCK_LIST_END_TAG = "</BlockList>%n";
+
+  /**
+   * Format string for the latest block in the block list XML.
+   * The placeholder will be replaced with the block identifier.
+   */
+  public static final String LATEST_BLOCK_FORMAT = "<Latest>%s</Latest>%n";
+
+
   /**
   /**
    * List of configurations that are related to Customer-Provided-Keys.
    * List of configurations that are related to Customer-Provided-Keys.
    * <ol>
    * <ol>
@@ -289,6 +313,12 @@ public final class AbfsHttpConstants {
   public static final String APACHE_IMPL = "Apache";
   public static final String APACHE_IMPL = "Apache";
   public static final String JDK_FALLBACK = "JDK_fallback";
   public static final String JDK_FALLBACK = "JDK_fallback";
   public static final String KEEP_ALIVE_CACHE_CLOSED = "KeepAliveCache is closed";
   public static final String KEEP_ALIVE_CACHE_CLOSED = "KeepAliveCache is closed";
+  public static final String DFS_FLUSH = "D";
+  public static final String DFS_APPEND = "D";
+  public static final String BLOB_FLUSH = "B";
+  public static final String BLOB_APPEND = "B";
+  public static final String FALLBACK_FLUSH = "FB";
+  public static final String FALLBACK_APPEND = "FB";
 
 
   private AbfsHttpConstants() {}
   private AbfsHttpConstants() {}
 }
 }

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

@@ -145,6 +145,11 @@ public final class FileSystemConfigurations {
    */
    */
   public static final int BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT = 20;
   public static final int BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT = 20;
 
 
+  /**
+   * Length of the block ID used for appends.
+   */
+  public static final int BLOCK_ID_LENGTH = 60;
+
   /**
   /**
    * Buffer blocks to disk.
    * Buffer blocks to disk.
    * Capacity is limited to available disk space.
    * Capacity is limited to available disk space.

+ 6 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java

@@ -104,5 +104,11 @@ public final class HttpHeaderConfigurations {
    */
    */
   public static final String X_MS_BLOB_CONTENT_MD5 = "x-ms-blob-content-md5";
   public static final String X_MS_BLOB_CONTENT_MD5 = "x-ms-blob-content-md5";
 
 
+  /**
+   * Http Request Header for denoting blob type.
+   * {@value}
+   */
+  public static final String X_MS_BLOB_TYPE = "x-ms-blob-type";
+
   private HttpHeaderConfigurations() {}
   private HttpHeaderConfigurations() {}
 }
 }

+ 51 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidIngressServiceException.java

@@ -0,0 +1,51 @@
+/**
+ * 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.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Exception thrown when an invalid ingress service is encountered.
+ *
+ * <p>This exception is used to indicate that the ingress service being used
+ * is not valid or supported. It extends the {@link AbfsRestOperationException}
+ * to provide additional context about the error.</p>
+ *
+ * @see AbfsRestOperationException
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class InvalidIngressServiceException extends AbfsRestOperationException {
+
+  /**
+   * Constructs a new InvalidIngressServiceException with the specified details.
+   *
+   * @param statusCode the HTTP status code
+   * @param errorCode the error code
+   * @param errorMessage the error message
+   * @param innerException the inner exception
+   */
+  public InvalidIngressServiceException(final int statusCode,
+      final String errorCode,
+      final String errorMessage,
+      final Exception innerException) {
+    super(statusCode, errorCode, errorMessage, innerException);
+  }
+}

+ 46 - 8
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java

@@ -36,7 +36,7 @@ public class AppendRequestParameters {
   private final String leaseId;
   private final String leaseId;
   private boolean isExpectHeaderEnabled;
   private boolean isExpectHeaderEnabled;
   private boolean isRetryDueToExpect;
   private boolean isRetryDueToExpect;
-  private final BlobAppendRequestParameters blobParams;
+  private BlobAppendRequestParameters blobParams;
 
 
 
 
   /**
   /**
@@ -129,27 +129,65 @@ public class AppendRequestParameters {
     return isRetryDueToExpect;
     return isRetryDueToExpect;
   }
   }
 
 
+  /**
+   * Retrieves the parameters specific to the append operation on the Blob Endpoint.
+   *
+   * @return the {@link BlobAppendRequestParameters} for the append operation.
+   */
+  public BlobAppendRequestParameters getBlobParams() {
+    return blobParams;
+  }
+
   /**
   /**
    * Returns BlockId of the block blob to be appended.
    * Returns BlockId of the block blob to be appended.
    * @return blockId
    * @return blockId
    */
    */
   public String getBlockId() {
   public String getBlockId() {
-    return blobParams.getBlockId();
+    return getBlobParams().getBlockId();
   }
   }
 
 
   /**
   /**
-   * Returns ETag of the block blob.
-   * @return eTag
+   * Sets whether the retry is due to the Expect header.
+   *
+   * @param retryDueToExpect true if the retry is due to the Expect header, false otherwise
    */
    */
-  public String getETag() {
-    return blobParams.getETag();
-  }
-
   public void setRetryDueToExpect(boolean retryDueToExpect) {
   public void setRetryDueToExpect(boolean retryDueToExpect) {
     isRetryDueToExpect = retryDueToExpect;
     isRetryDueToExpect = retryDueToExpect;
   }
   }
 
 
+  /**
+   * Sets whether the Expect header is enabled.
+   *
+   * @param expectHeaderEnabled true if the Expect header is enabled, false otherwise
+   */
   public void setExpectHeaderEnabled(boolean expectHeaderEnabled) {
   public void setExpectHeaderEnabled(boolean expectHeaderEnabled) {
     isExpectHeaderEnabled = expectHeaderEnabled;
     isExpectHeaderEnabled = expectHeaderEnabled;
   }
   }
+
+  /**
+   * Sets the Block ID for the block blob to be appended.
+   *
+   * @param blockId the Block ID to set
+   */
+  public void setBlockId(final String blockId) {
+    this.getBlobParams().setBlockId(blockId);
+  }
+
+  /**
+   * Sets the ETag for the block blob.
+   *
+   * @param eTag the ETag to set
+   */
+  public void setEtag(final String eTag) {
+    this.getBlobParams().setETag(eTag);
+  }
+
+  /**
+   * Sets the parameters specific to the append operation on the Blob Endpoint.
+   *
+   * @param blobParams the {@link BlobAppendRequestParameters} to set
+   */
+  public void setBlobParams(BlobAppendRequestParameters blobParams) {
+    this.blobParams = blobParams;
+  }
 }
 }

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

@@ -58,6 +58,8 @@ public enum AzureServiceErrorCode {
   ACCOUNT_REQUIRES_HTTPS("AccountRequiresHttps", HttpURLConnection.HTTP_BAD_REQUEST, null),
   ACCOUNT_REQUIRES_HTTPS("AccountRequiresHttps", HttpURLConnection.HTTP_BAD_REQUEST, null),
   MD5_MISMATCH("Md5Mismatch", HttpURLConnection.HTTP_BAD_REQUEST,
   MD5_MISMATCH("Md5Mismatch", HttpURLConnection.HTTP_BAD_REQUEST,
           "The MD5 value specified in the request did not match with the MD5 value calculated by the server."),
           "The MD5 value specified in the request did not match with the MD5 value calculated by the server."),
+  BLOB_OPERATION_NOT_SUPPORTED("BlobOperationNotSupported", HttpURLConnection.HTTP_CONFLICT, null),
+  INVALID_APPEND_OPERATION("InvalidAppendOperation", HttpURLConnection.HTTP_CONFLICT, null),
   UNKNOWN(null, -1, null);
   UNKNOWN(null, -1, null);
 
 
   private final String errorCode;
   private final String errorCode;

+ 8 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/BlobAppendRequestParameters.java

@@ -43,4 +43,12 @@ public class BlobAppendRequestParameters {
   public String getETag() {
   public String getETag() {
     return eTag;
     return eTag;
   }
   }
+
+  public void setBlockId(final String blockId) {
+    this.blockId = blockId;
+  }
+
+  public void setETag(final String eTag) {
+    this.eTag = eTag;
+  }
 }
 }

+ 2 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java

@@ -49,6 +49,8 @@ public interface SASTokenProvider {
   String SET_PERMISSION_OPERATION = "set-permission";
   String SET_PERMISSION_OPERATION = "set-permission";
   String SET_PROPERTIES_OPERATION = "set-properties";
   String SET_PROPERTIES_OPERATION = "set-properties";
   String WRITE_OPERATION = "write";
   String WRITE_OPERATION = "write";
+  // Generic HTTP operation can be used with FixedSASTokenProvider.
+  String FIXED_SAS_STORE_OPERATION = "fixed-sas";
 
 
   /**
   /**
    * Initialize authorizer for Azure Blob File System.
    * Initialize authorizer for Azure Blob File System.

+ 72 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java

@@ -0,0 +1,72 @@
+/**
+ * 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.services;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.commons.codec.binary.Base64;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
+
+/**
+ * Represents a block in Azure Blob Storage used by Azure Data Lake Storage (ADLS).
+ *
+ * <p>Extends {@link AbfsBlock} and provides functionality specific to Azure Blob Storage blocks.
+ * Each block is identified by a unique block ID generated based on the offset and stream ID.</p>
+ */
+public class AbfsBlobBlock extends AbfsBlock {
+
+  private final String blockId;
+
+  /**
+   * Gets the activeBlock and the blockId.
+   *
+   * @param outputStream AbfsOutputStream Instance.
+   * @param offset       Used to generate blockId based on offset.
+   * @throws IOException exception is thrown.
+   */
+  AbfsBlobBlock(AbfsOutputStream outputStream, long offset) throws IOException {
+    super(outputStream, offset);
+    this.blockId = generateBlockId(offset);
+  }
+
+  /**
+   * Helper method that generates blockId.
+   * @param position The offset needed to generate blockId.
+   * @return String representing the block ID generated.
+   */
+  private String generateBlockId(long position) {
+    String streamId = getOutputStream().getStreamID();
+    String streamIdHash = Integer.toString(streamId.hashCode());
+    String blockId = String.format("%d_%s", position, streamIdHash);
+    byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH];
+    System.arraycopy(blockId.getBytes(StandardCharsets.UTF_8), 0, blockIdByteArray, 0, Math.min(BLOCK_ID_LENGTH, blockId.length()));
+    return new String(Base64.encodeBase64(blockIdByteArray), StandardCharsets.UTF_8);
+  }
+
+  /**
+   * Returns blockId for the block.
+   * @return blockId.
+   */
+  public String getBlockId() {
+    return blockId;
+  }
+}
+

+ 271 - 16
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java

@@ -50,7 +50,9 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
 import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ApiVersion;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ApiVersion;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
 import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
@@ -69,15 +71,21 @@ import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 
 
+import static java.net.HttpURLConnection.HTTP_CONFLICT;
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
 import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ACQUIRE_LEASE_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ACQUIRE_LEASE_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_BLOB_TYPE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_BLOCK;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_JSON;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_JSON;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_OCTET_STREAM;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_OCTET_STREAM;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_XML;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_XML;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCKLIST;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCKLIST;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_BLOB_TYPE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_LIST_END_TAG;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_LIST_START_TAG;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_TYPE_COMMITTED;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_TYPE_COMMITTED;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BREAK_LEASE_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BREAK_LEASE_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMMA;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMMA;
@@ -90,6 +98,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_MET
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.LATEST_BLOCK_FORMAT;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.LEASE;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.LEASE;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.LIST;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.LIST;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.METADATA;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.METADATA;
@@ -107,8 +116,10 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_COMMITTED_BLOCKS;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_COMMITTED_BLOCKS;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_HDI_ISFOLDER;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_HDI_ISFOLDER;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_VERSION;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_ASCII;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_ASCII;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_UNICODE;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_UNICODE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ZERO;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ACCEPT;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ACCEPT;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_LENGTH;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_LENGTH;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_MD5;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_MD5;
@@ -120,6 +131,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.L
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.RANGE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.RANGE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.USER_AGENT;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.USER_AGENT;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_BLOB_CONTENT_MD5;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_BLOB_CONTENT_MD5;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_BLOB_TYPE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_SOURCE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_SOURCE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_BREAK_PERIOD;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_BREAK_PERIOD;
@@ -140,6 +152,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARA
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_MAX_RESULTS;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_MAX_RESULTS;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_PREFIX;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_PREFIX;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RESTYPE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RESTYPE;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.PATH_EXISTS;
 
 
 /**
 /**
  * AbfsClient interacting with Blob endpoint.
  * AbfsClient interacting with Blob endpoint.
@@ -328,7 +341,7 @@ public class AbfsBlobClient extends AbfsClient {
       abfsUriQueryBuilder.addQuery(QUERY_PARAM_DELIMITER, FORWARD_SLASH);
       abfsUriQueryBuilder.addQuery(QUERY_PARAM_DELIMITER, FORWARD_SLASH);
     }
     }
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAX_RESULTS, String.valueOf(listMaxResults));
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAX_RESULTS, String.valueOf(listMaxResults));
-    appendSASTokenToQuery(relativePath, SASTokenProvider.LIST_OPERATION, abfsUriQueryBuilder);
+    appendSASTokenToQuery(relativePath, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
 
     final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -377,8 +390,181 @@ public class AbfsBlobClient extends AbfsClient {
       final String eTag,
       final String eTag,
       final ContextEncryptionAdapter contextEncryptionAdapter,
       final ContextEncryptionAdapter contextEncryptionAdapter,
       final TracingContext tracingContext) throws AzureBlobFileSystemException {
       final TracingContext tracingContext) throws AzureBlobFileSystemException {
-    // TODO: [FnsOverBlob][HADOOP-19232] To be implemented as part of ingress support.
-    throw new NotImplementedException("Create Path operation on Blob endpoint yet to be implemented.");
+    return createPath(path, isFile, overwrite, permissions, isAppendBlob, eTag,
+        contextEncryptionAdapter, tracingContext, false);
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/put-blob">Put Blob</a>.
+   * Creates a file or directory (marker file) at the specified path.
+   *
+   * @param path the path of the directory to be created.
+   * @param isFile whether the path is a file.
+   * @param overwrite whether to overwrite if the path already exists.
+   * @param permissions the permissions to set on the path.
+   * @param isAppendBlob whether the path is an append blob.
+   * @param eTag the eTag of the path.
+   * @param contextEncryptionAdapter the context encryption adapter.
+   * @param tracingContext the tracing context.
+   * @param isCreateCalledFromMarkers whether the create is called from markers.
+   * @return the executed rest operation containing the response from the server.
+   * @throws AzureBlobFileSystemException if the rest operation fails.
+   */
+  public AbfsRestOperation createPath(final String path,
+      final boolean isFile,
+      final boolean overwrite,
+      final AzureBlobFileSystemStore.Permissions permissions,
+      final boolean isAppendBlob,
+      final String eTag,
+      final ContextEncryptionAdapter contextEncryptionAdapter,
+      final TracingContext tracingContext,
+      boolean isCreateCalledFromMarkers) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    if (!getIsNamespaceEnabled() && !isCreateCalledFromMarkers) {
+      AbfsHttpOperation op1Result = null;
+      try {
+        op1Result = getPathStatus(path, tracingContext,
+            null, true).getResult();
+      } catch (AbfsRestOperationException ex) {
+        if (ex.getStatusCode() == HTTP_NOT_FOUND) {
+          LOG.debug("No directory/path found: {}", path);
+        } else {
+          LOG.debug("Failed to get path status for: {}", path, ex);
+          throw ex;
+        }
+      }
+      if (op1Result != null) {
+        boolean isDir = checkIsDir(op1Result);
+        if (isFile == isDir) {
+          throw new AbfsRestOperationException(HTTP_CONFLICT,
+              AzureServiceErrorCode.PATH_CONFLICT.getErrorCode(),
+              PATH_EXISTS,
+              null);
+        }
+      }
+      Path parentPath = new Path(path).getParent();
+      if (parentPath != null && !parentPath.isRoot()) {
+        createMarkers(parentPath, overwrite, permissions, isAppendBlob, eTag,
+            contextEncryptionAdapter, tracingContext);
+      }
+    }
+    if (isFile) {
+      addEncryptionKeyRequestHeaders(path, requestHeaders, true,
+          contextEncryptionAdapter, tracingContext);
+    } else {
+      requestHeaders.add(new AbfsHttpHeader(X_MS_META_HDI_ISFOLDER, TRUE));
+    }
+    requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, ZERO));
+    if (isAppendBlob) {
+      requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_TYPE, APPEND_BLOB_TYPE));
+    } else {
+      requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_TYPE, BLOCK_BLOB_TYPE));
+    }
+    if (!overwrite) {
+      requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR));
+    }
+    if (eTag != null && !eTag.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.PutBlob,
+        HTTP_METHOD_PUT, url, requestHeaders);
+    try {
+      op.execute(tracingContext);
+    } catch (AzureBlobFileSystemException ex) {
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw ex;
+      }
+      if (!isFile && op.getResult().getStatusCode() == HTTP_CONFLICT) {
+        // This ensures that we don't throw ex only for existing directory but if a blob exists we throw exception.
+        AbfsHttpOperation opResult = null;
+        try {
+          opResult = this.getPathStatus(path, true, tracingContext, null).getResult();
+        } catch (AbfsRestOperationException e) {
+          if (opResult != null) {
+            LOG.debug("Failed to get path status for: {} during blob type check", path, e);
+            throw e;
+          }
+        }
+        if (opResult != null && checkIsDir(opResult)) {
+          return op;
+        }
+      }
+      throw ex;
+    }
+    return op;
+  }
+
+  /**
+   *  Creates marker blobs for the parent directories of the specified path.
+   *
+   * @param path The path for which parent directories need to be created.
+   * @param overwrite A flag indicating whether existing directories should be overwritten.
+   * @param permissions The permissions to be set for the created directories.
+   * @param isAppendBlob A flag indicating whether the created blob should be of type APPEND_BLOB.
+   * @param eTag The eTag to be matched for conditional requests.
+   * @param contextEncryptionAdapter The encryption adapter for context encryption.
+   * @param tracingContext The tracing context for the operation.
+   * @throws AzureBlobFileSystemException If the creation of any parent directory fails.
+   */
+  private void createMarkers(final Path path,
+      final boolean overwrite,
+      final AzureBlobFileSystemStore.Permissions permissions,
+      final boolean isAppendBlob,
+      final String eTag,
+      final ContextEncryptionAdapter contextEncryptionAdapter,
+      final TracingContext tracingContext) throws AzureBlobFileSystemException {
+    ArrayList<Path> keysToCreateAsFolder = new ArrayList<>();
+    checkParentChainForFile(path, tracingContext,
+        keysToCreateAsFolder);
+    for (Path pathToCreate : keysToCreateAsFolder) {
+      createPath(pathToCreate.toUri().getPath(), false, overwrite, permissions,
+          isAppendBlob, eTag, contextEncryptionAdapter, tracingContext, true);
+    }
+  }
+
+  /**
+   * Checks for the entire parent hierarchy and returns if any directory exists and
+   * throws an exception if any file exists.
+   * @param path path to check the hierarchy for.
+   * @param tracingContext the tracingcontext.
+   */
+  private void checkParentChainForFile(Path path, TracingContext tracingContext,
+      List<Path> keysToCreateAsFolder) throws AzureBlobFileSystemException {
+    AbfsHttpOperation opResult = null;
+    Path current = path;
+    do {
+      try {
+        opResult = getPathStatus(current.toUri().getPath(),
+            tracingContext, null, false).getResult();
+      } catch (AbfsRestOperationException ex) {
+        if (ex.getStatusCode() == HTTP_NOT_FOUND) {
+          LOG.debug("No explicit directory/path found: {}", current);
+        } else {
+          LOG.debug("Exception occurred while getting path status: {}", current, ex);
+          throw ex;
+        }
+      }
+      boolean isDirectory = opResult != null && checkIsDir(opResult);
+      if (opResult != null && !isDirectory) {
+        throw new AbfsRestOperationException(HTTP_CONFLICT,
+            AzureServiceErrorCode.PATH_CONFLICT.getErrorCode(),
+            PATH_EXISTS,
+            null);
+      }
+      if (isDirectory) {
+        return;
+      }
+      keysToCreateAsFolder.add(current);
+      current = current.getParent();
+    } while (current != null && !current.isRoot());
   }
   }
 
 
   /**
   /**
@@ -400,6 +586,7 @@ public class AbfsBlobClient extends AbfsClient {
 
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -427,6 +614,7 @@ public class AbfsBlobClient extends AbfsClient {
 
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -454,6 +642,7 @@ public class AbfsBlobClient extends AbfsClient {
 
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -480,6 +669,7 @@ public class AbfsBlobClient extends AbfsClient {
 
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -540,7 +730,6 @@ public class AbfsBlobClient extends AbfsClient {
     addEncryptionKeyRequestHeaders(path, requestHeaders, false,
     addEncryptionKeyRequestHeaders(path, requestHeaders, false,
         contextEncryptionAdapter, tracingContext);
         contextEncryptionAdapter, tracingContext);
     requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
     requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
-    requestHeaders.add(new AbfsHttpHeader(IF_MATCH, reqParams.getETag()));
     if (reqParams.getLeaseId() != null) {
     if (reqParams.getLeaseId() != null) {
       requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId()));
       requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId()));
     }
     }
@@ -561,7 +750,7 @@ public class AbfsBlobClient extends AbfsClient {
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCK);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCK);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOCKID, reqParams.getBlockId());
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOCKID, reqParams.getBlockId());
 
 
-    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
+    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION,
         abfsUriQueryBuilder, cachedSasToken);
         abfsUriQueryBuilder, cachedSasToken);
 
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
@@ -583,7 +772,7 @@ public class AbfsBlobClient extends AbfsClient {
          if someone has taken dependency on the exception message,
          if someone has taken dependency on the exception message,
          which is created using the error string present in the response header.
          which is created using the error string present in the response header.
       */
       */
-      int responseStatusCode = ((AbfsRestOperationException) e).getStatusCode();
+      int responseStatusCode = e.getStatusCode();
       if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) {
       if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) {
         LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path);
         LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path);
         reqParams.setExpectHeaderEnabled(false);
         reqParams.setExpectHeaderEnabled(false);
@@ -611,6 +800,55 @@ public class AbfsBlobClient extends AbfsClient {
     return op;
     return op;
   }
   }
 
 
+  /**
+   * Appends a block to an append blob.
+   * <a href="../../../../site/markdown/blobEndpoint.md#append-block">Append Block</a>.
+   *
+   * @param path the path of the append blob.
+   * @param requestParameters the parameters for the append request.
+   * @param data the data to be appended.
+   * @param tracingContext the tracing context.
+   * @return the executed rest operation containing the response from the server.
+   * @throws AzureBlobFileSystemException if the rest operation fails.
+   */
+  public AbfsRestOperation appendBlock(final String path,
+      AppendRequestParameters requestParameters,
+      final byte[] data,
+      final TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(data.length)));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_TYPE, APPEND_BLOB_TYPE));
+    if (requestParameters.getLeaseId() != null) {
+      requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, requestParameters.getLeaseId()));
+    }
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, APPEND_BLOCK);
+    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.AppendBlock,
+        HTTP_METHOD_PUT,
+        url,
+        requestHeaders,
+        data,
+        requestParameters.getoffset(),
+        requestParameters.getLength(),
+        sasTokenForReuse);
+
+    try {
+      op.execute(tracingContext);
+    } catch (AzureBlobFileSystemException ex) {
+      LOG.debug("Exception occurred during append block operation for path: {}", path, ex);
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw ex;
+      }
+      throw ex;
+    }
+    return op;
+  }
+
   /**
   /**
    * Blob Endpoint needs blockIds to flush the data.
    * Blob Endpoint needs blockIds to flush the data.
    * This method is not supported on Blob Endpoint.
    * This method is not supported on Blob Endpoint.
@@ -677,7 +915,7 @@ public class AbfsBlobClient extends AbfsClient {
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose));
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose));
-    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
+    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION,
         abfsUriQueryBuilder, cachedSasToken);
         abfsUriQueryBuilder, cachedSasToken);
 
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
@@ -739,13 +977,12 @@ public class AbfsBlobClient extends AbfsClient {
 
 
     AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, METADATA);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, METADATA);
-    appendSASTokenToQuery(path, SASTokenProvider.SET_PROPERTIES_OPERATION, abfsUriQueryBuilder);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
     final AbfsRestOperation op = getAbfsRestOperation(
         AbfsRestOperationType.SetPathProperties,
         AbfsRestOperationType.SetPathProperties,
         HTTP_METHOD_PUT, url, requestHeaders);
         HTTP_METHOD_PUT, url, requestHeaders);
-    op.execute(tracingContext);
     try {
     try {
       op.execute(tracingContext);
       op.execute(tracingContext);
     } catch (AbfsRestOperationException ex) {
     } catch (AbfsRestOperationException ex) {
@@ -756,7 +993,8 @@ public class AbfsBlobClient extends AbfsClient {
       // This path could be present as an implicit directory in FNS.
       // This path could be present as an implicit directory in FNS.
       if (op.getResult().getStatusCode() == HTTP_NOT_FOUND && isNonEmptyListing(path, tracingContext)) {
       if (op.getResult().getStatusCode() == HTTP_NOT_FOUND && isNonEmptyListing(path, tracingContext)) {
         // Implicit path found, create a marker blob at this path and set properties.
         // Implicit path found, create a marker blob at this path and set properties.
-        this.createPath(path, false, false, null, false, null, contextEncryptionAdapter, tracingContext);
+        this.createPath(path, false, false, null, false, null,
+            contextEncryptionAdapter, tracingContext, false);
         // Make sure hdi_isFolder is added to the list of properties to be set.
         // Make sure hdi_isFolder is added to the list of properties to be set.
         boolean hdiIsFolderExists = properties.containsKey(XML_TAG_HDI_ISFOLDER);
         boolean hdiIsFolderExists = properties.containsKey(XML_TAG_HDI_ISFOLDER);
         if (!hdiIsFolderExists) {
         if (!hdiIsFolderExists) {
@@ -812,7 +1050,7 @@ public class AbfsBlobClient extends AbfsClient {
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN,
     abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN,
         String.valueOf(getAbfsConfiguration().isUpnUsed()));
         String.valueOf(getAbfsConfiguration().isUpnUsed()));
-    appendSASTokenToQuery(path, SASTokenProvider.GET_PROPERTIES_OPERATION,
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION,
         abfsUriQueryBuilder);
         abfsUriQueryBuilder);
 
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
@@ -889,7 +1127,7 @@ public class AbfsBlobClient extends AbfsClient {
     }
     }
 
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION,
+    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION,
         abfsUriQueryBuilder, cachedSasToken);
         abfsUriQueryBuilder, cachedSasToken);
 
 
     URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
@@ -1031,7 +1269,7 @@ public class AbfsBlobClient extends AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    String operation = SASTokenProvider.READ_OPERATION;
+    String operation = SASTokenProvider.FIXED_SAS_STORE_OPERATION;
     appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
     appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
 
 
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
@@ -1069,9 +1307,9 @@ public class AbfsBlobClient extends AbfsClient {
     String dstBlobRelativePath = destinationBlobPath.toUri().getPath();
     String dstBlobRelativePath = destinationBlobPath.toUri().getPath();
     String srcBlobRelativePath = sourceBlobPath.toUri().getPath();
     String srcBlobRelativePath = sourceBlobPath.toUri().getPath();
     appendSASTokenToQuery(dstBlobRelativePath,
     appendSASTokenToQuery(dstBlobRelativePath,
-        SASTokenProvider.WRITE_OPERATION, abfsUriQueryBuilderDst);
+        SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilderDst);
     appendSASTokenToQuery(srcBlobRelativePath,
     appendSASTokenToQuery(srcBlobRelativePath,
-        SASTokenProvider.READ_OPERATION, abfsUriQueryBuilderSrc);
+        SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilderSrc);
     final URL url = createRequestUrl(dstBlobRelativePath,
     final URL url = createRequestUrl(dstBlobRelativePath,
         abfsUriQueryBuilderDst.toString());
         abfsUriQueryBuilderDst.toString());
     final String sourcePathUrl = createRequestUrl(srcBlobRelativePath,
     final String sourcePathUrl = createRequestUrl(srcBlobRelativePath,
@@ -1105,7 +1343,7 @@ public class AbfsBlobClient extends AbfsClient {
     AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     String blobRelativePath = blobPath.toUri().getPath();
     String blobRelativePath = blobPath.toUri().getPath();
     appendSASTokenToQuery(blobRelativePath,
     appendSASTokenToQuery(blobRelativePath,
-        SASTokenProvider.DELETE_OPERATION, abfsUriQueryBuilder);
+        SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
     final URL url = createRequestUrl(blobRelativePath, abfsUriQueryBuilder.toString());
     final URL url = createRequestUrl(blobRelativePath, abfsUriQueryBuilder.toString());
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     if (leaseId != null) {
     if (leaseId != null) {
@@ -1469,4 +1707,21 @@ public class AbfsBlobClient extends AbfsClient {
     }
     }
     return false;
     return false;
   }
   }
+
+  /**
+   * Generates an XML string representing the block list.
+   *
+   * @param blockIds the set of block IDs
+   * @return the generated XML string
+   */
+  public static String generateBlockListXml(List<String> blockIds) {
+    StringBuilder stringBuilder = new StringBuilder();
+    stringBuilder.append(String.format(XML_VERSION));
+    stringBuilder.append(String.format(BLOCK_LIST_START_TAG));
+    for (String blockId : blockIds) {
+      stringBuilder.append(String.format(LATEST_BLOCK_FORMAT, blockId));
+    }
+    stringBuilder.append(String.format(BLOCK_LIST_END_TAG));
+    return stringBuilder.toString();
+  }
 }
 }

+ 160 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlock.java

@@ -0,0 +1,160 @@
+/**
+ * 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.services;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.store.DataBlocks;
+
+/**
+ * Return activeBlock with blockId.
+ */
+public class AbfsBlock implements Closeable {
+
+  private final DataBlocks.DataBlock activeBlock;
+  private AbfsOutputStream outputStream;
+  private final long offset;
+  private BlockEntry blockEntry;
+
+  /**
+   * Gets the activeBlock and the blockId.
+   * @param outputStream AbfsOutputStream Instance.
+   * @param offset Used to generate blockId based on offset.
+   * @throws IOException
+   */
+  AbfsBlock(AbfsOutputStream outputStream, long offset) throws IOException {
+    this.outputStream = outputStream;
+    this.offset = offset;
+    DataBlocks.BlockFactory blockFactory = outputStream.getBlockManager().getBlockFactory();
+    long blockCount = outputStream.getBlockManager().getBlockCount();
+    int blockSize = outputStream.getBlockManager().getBlockSize();
+    AbfsOutputStreamStatistics outputStreamStatistics = outputStream.getOutputStreamStatistics();
+    this.activeBlock = blockFactory.create(blockCount, blockSize, outputStreamStatistics);
+  }
+
+  /**
+   * Returns datasize for the block.
+   * @return datasize.
+   */
+  public int dataSize() {
+    return activeBlock.dataSize();
+  }
+
+  /**
+   * Returns an instance of BlockUploadData.
+   *
+   * @return an instance of BlockUploadData.
+   * @throws IOException if an I/O error occurs during the upload process.
+   */
+  public DataBlocks.BlockUploadData startUpload() throws IOException {
+    return activeBlock.startUpload();
+  }
+
+  /**
+   * Return the block has data or not.
+   * @return block has data or not.
+   */
+  public boolean hasData() {
+    return activeBlock.hasData();
+  }
+
+  /**
+   * Writes a series of bytes from the buffer, starting from the specified offset.
+   * Returns the number of bytes written. Only valid in the state Writing.
+   * The base class verifies the state but does no writing.
+   *
+   * @param buffer the buffer containing the bytes to write.
+   * @param offset the offset in the buffer to start writing from.
+   * @param length the number of bytes to write.
+   * @return the number of bytes written.
+   * @throws IOException if an I/O error occurs.
+   */
+  public int write(byte[] buffer, int offset, int length) throws IOException {
+    return activeBlock.write(buffer, offset, length);
+  }
+
+  /**
+   * Returns remainingCapacity.
+   * @return remainingCapacity.
+   */
+  public int remainingCapacity() {
+    return activeBlock.remainingCapacity();
+  }
+
+  /**
+   * Returns the offset of the block.
+   *
+   * @return the offset of the block.
+   */
+  public Long getOffset() {
+    return offset;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (activeBlock != null) {
+      activeBlock.close();
+    }
+  }
+
+  /**
+   * Returns blockId for the block.
+   * @return blockId.
+   */
+  public String getBlockId() {
+    throw new IllegalArgumentException("DFS client does not support blockId");
+  }
+
+  /**
+   * Gets the AbfsOutputStream.
+   *
+   * @return the AbfsOutputStream.
+   */
+  public AbfsOutputStream getOutputStream() {
+    return outputStream;
+  }
+
+  /**
+   * Sets the AbfsOutputStream.
+   *
+   * @param outputStream the AbfsOutputStream to set.
+   */
+  public void setOutputStream(final AbfsOutputStream outputStream) {
+    this.outputStream = outputStream;
+  }
+
+  /**
+   * Returns the block entry.
+   *
+   * @return the block entry.
+   */
+  public BlockEntry getBlockEntry() {
+    return blockEntry;
+  }
+
+  /**
+   * Sets the block entry.
+   *
+   * @param blockEntry the block entry to set.
+   */
+  public void setBlockEntry(final BlockEntry blockEntry) {
+    this.blockEntry = blockEntry;
+  }
+}

+ 47 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlockStatus.java

@@ -0,0 +1,47 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.services;
+
+/**
+ * Enum representing the status of an ABFS block.
+ *
+ * <p>This enum is used to indicate the current status of a block in the Azure Blob File System (ABFS).
+ * The possible statuses are:</p>
+ * <ul>
+ *   <li>NEW - The block is newly created and has not been processed yet.</li>
+ *   <li>SUCCESS - The block has been successfully processed.</li>
+ *   <li>FAILED - The block processing has failed.</li>
+ * </ul>
+ */
+public enum AbfsBlockStatus {
+  /**
+   * The block is newly created and has not been processed yet.
+   */
+  NEW,
+
+  /**
+   * The block has been successfully processed.
+   */
+  SUCCESS,
+
+  /**
+   * The block processing has failed.
+   */
+  FAILED
+}

+ 19 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java

@@ -177,6 +177,7 @@ public abstract class AbfsClient implements Closeable {
   private KeepAliveCache keepAliveCache;
   private KeepAliveCache keepAliveCache;
 
 
   private AbfsApacheHttpClient abfsApacheHttpClient;
   private AbfsApacheHttpClient abfsApacheHttpClient;
+  private static boolean isNamespaceEnabled = false;
 
 
   /**
   /**
    * logging the rename failure if metadata is in an incomplete state.
    * logging the rename failure if metadata is in an incomplete state.
@@ -1618,6 +1619,24 @@ public abstract class AbfsClient implements Closeable {
     return userAgent;
     return userAgent;
   }
   }
 
 
+  /**
+   * Checks if the namespace is enabled.
+   *
+   * @return True if the namespace is enabled, false otherwise.
+   */
+  public static boolean getIsNamespaceEnabled() {
+    return isNamespaceEnabled;
+  }
+
+  /**
+   * Sets the namespace enabled status.
+   *
+   * @param namespaceEnabled True to enable the namespace, false to disable it.
+   */
+  public static void setIsNamespaceEnabled(final boolean namespaceEnabled) {
+    isNamespaceEnabled = namespaceEnabled;
+  }
+
   protected boolean isRenameResilience() {
   protected boolean isRenameResilience() {
     return renameResilience;
     return renameResilience;
   }
   }

+ 29 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java

@@ -41,6 +41,7 @@ public class AbfsClientHandler {
   public static final Logger LOG = LoggerFactory.getLogger(AbfsClientHandler.class);
   public static final Logger LOG = LoggerFactory.getLogger(AbfsClientHandler.class);
 
 
   private AbfsServiceType defaultServiceType;
   private AbfsServiceType defaultServiceType;
+  private AbfsServiceType ingressServiceType;
   private final AbfsDfsClient dfsAbfsClient;
   private final AbfsDfsClient dfsAbfsClient;
   private final AbfsBlobClient blobAbfsClient;
   private final AbfsBlobClient blobAbfsClient;
 
 
@@ -80,6 +81,7 @@ public class AbfsClientHandler {
    */
    */
   private void initServiceType(final AbfsConfiguration abfsConfiguration) {
   private void initServiceType(final AbfsConfiguration abfsConfiguration) {
     this.defaultServiceType = abfsConfiguration.getFsConfiguredServiceType();
     this.defaultServiceType = abfsConfiguration.getFsConfiguredServiceType();
+    this.ingressServiceType = abfsConfiguration.getIngressServiceType();
   }
   }
 
 
   /**
   /**
@@ -90,6 +92,15 @@ public class AbfsClientHandler {
     return getClient(defaultServiceType);
     return getClient(defaultServiceType);
   }
   }
 
 
+  /**
+   * Get the AbfsClient based on the ingress service type.
+   *
+   * @return AbfsClient for the ingress service type.
+   */
+  public AbfsClient getIngressClient() {
+    return getClient(ingressServiceType);
+  }
+
   /**
   /**
    * Get the AbfsClient based on the service type.
    * Get the AbfsClient based on the service type.
    * @param serviceType AbfsServiceType.
    * @param serviceType AbfsServiceType.
@@ -99,6 +110,24 @@ public class AbfsClientHandler {
     return serviceType == AbfsServiceType.DFS ? dfsAbfsClient : blobAbfsClient;
     return serviceType == AbfsServiceType.DFS ? dfsAbfsClient : blobAbfsClient;
   }
   }
 
 
+  /**
+   * Gets the AbfsDfsClient instance.
+   *
+   * @return the AbfsDfsClient instance.
+   */
+  public AbfsDfsClient getDfsClient() {
+    return dfsAbfsClient;
+  }
+
+  /**
+   * Gets the AbfsBlobClient instance.
+   *
+   * @return the AbfsBlobClient instance.
+   */
+  public AbfsBlobClient getBlobClient() {
+    return blobAbfsClient;
+  }
+
   /**
   /**
    * Create the AbfsDfsClient using the url used to configure file system.
    * Create the AbfsDfsClient using the url used to configure file system.
    * If URL is for Blob endpoint, it will be converted to DFS endpoint.
    * If URL is for Blob endpoint, it will be converted to DFS endpoint.

+ 16 - 3
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java

@@ -29,10 +29,12 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public final class AbfsErrors {
 public final class AbfsErrors {
   public static final String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease";
   public static final String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease";
-  public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the"
-      + " resource has expired";
+  public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the resource has expired.";
+  public static final String ERR_LEASE_EXPIRED_BLOB = "A lease ID was specified, but the lease for the blob has expired.";
   public static final String ERR_NO_LEASE_ID_SPECIFIED = "There is currently a lease on the "
   public static final String ERR_NO_LEASE_ID_SPECIFIED = "There is currently a lease on the "
       + "resource and no lease ID was specified in the request";
       + "resource and no lease ID was specified in the request";
+  public static final String ERR_NO_LEASE_ID_SPECIFIED_BLOB = "There is currently a lease on the "
+      + "blob and no lease ID was specified in the request";
   public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path "
   public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path "
       + "detected. Failing request to honor single writer semantics";
       + "detected. Failing request to honor single writer semantics";
   public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease";
   public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease";
@@ -43,11 +45,22 @@ public final class AbfsErrors {
   public static final String ERR_LEASE_DID_NOT_MATCH = "The lease ID specified did not match the "
   public static final String ERR_LEASE_DID_NOT_MATCH = "The lease ID specified did not match the "
       + "lease ID for the resource with the specified lease operation";
       + "lease ID for the resource with the specified lease operation";
   public static final String ERR_LEASE_BROKEN = "The lease ID matched, but the lease has been "
   public static final String ERR_LEASE_BROKEN = "The lease ID matched, but the lease has been "
-    + "broken explicitly and cannot be renewed";
+      + "broken explicitly and cannot be renewed";
   public static final String ERR_LEASE_FUTURE_EXISTS = "There is already an existing lease "
   public static final String ERR_LEASE_FUTURE_EXISTS = "There is already an existing lease "
       + "operation";
       + "operation";
   public static final String ERR_NO_LEASE_THREADS = "Lease desired but no lease threads "
   public static final String ERR_NO_LEASE_THREADS = "Lease desired but no lease threads "
       + "configured, set " + FS_AZURE_LEASE_THREADS;
       + "configured, set " + FS_AZURE_LEASE_THREADS;
   public static final String ERR_CREATE_ON_ROOT = "Cannot create file over root path";
   public static final String ERR_CREATE_ON_ROOT = "Cannot create file over root path";
+  public static final String PATH_EXISTS = "The specified path, or an element of the path, "
+      + "exists and its resource type is invalid for this operation.";
+  public static final String BLOB_OPERATION_NOT_SUPPORTED = "Blob operation is not supported.";
+  public static final String INVALID_APPEND_OPERATION = "The resource was created or modified by the Azure Blob Service API "
+      + "and cannot be appended to by the Azure Data Lake Storage Service API";
+  public static final String CONDITION_NOT_MET = "The condition specified using "
+      + "HTTP conditional header(s) is not met.";
+  /**
+   * Exception message on filesystem init if token-provider-auth-type configs are provided
+   */
+  public static final String UNAUTHORIZED_SAS = "Incorrect SAS token provider configured for non-hierarchical namespace account.";
   private AbfsErrors() {}
   private AbfsErrors() {}
 }
 }

+ 1 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java

@@ -468,7 +468,7 @@ public abstract class AbfsHttpOperation implements AbfsPerfLoggable {
       // Ignore errors that occur while attempting to parse the storage
       // Ignore errors that occur while attempting to parse the storage
       // error, since the response may have been handled by the HTTP driver
       // error, since the response may have been handled by the HTTP driver
       // or for other reasons have an unexpected
       // or for other reasons have an unexpected
-      log.debug("ExpectedError: ", ex);
+      log.debug("Error parsing storage error response", ex);
     }
     }
   }
   }
 
 

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

@@ -25,7 +25,12 @@ import java.net.HttpURLConnection;
 import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.Future;
 import java.util.concurrent.Future;
 import java.util.UUID;
 import java.util.UUID;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
 
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidIngressServiceException;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.impl.BackReference;
 import org.apache.hadoop.fs.impl.BackReference;
@@ -53,6 +58,7 @@ import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.Syncable;
 
 
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN;
 import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE;
 import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE;
 import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;
 import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;
@@ -68,7 +74,7 @@ import static org.apache.hadoop.util.Preconditions.checkState;
 public class AbfsOutputStream extends OutputStream implements Syncable,
 public class AbfsOutputStream extends OutputStream implements Syncable,
     StreamCapabilities, IOStatisticsSource {
     StreamCapabilities, IOStatisticsSource {
 
 
-  private final AbfsClient client;
+  private volatile AbfsClient client;
   private final String path;
   private final String path;
   /** The position in the file being uploaded, where the next block would be
   /** The position in the file being uploaded, where the next block would be
    * uploaded.
    * uploaded.
@@ -117,24 +123,35 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
   /** Factory for blocks. */
   /** Factory for blocks. */
   private final DataBlocks.BlockFactory blockFactory;
   private final DataBlocks.BlockFactory blockFactory;
 
 
-  /** Current data block. Null means none currently active. */
-  private DataBlocks.DataBlock activeBlock;
-
   /** Count of blocks uploaded. */
   /** Count of blocks uploaded. */
   private long blockCount = 0;
   private long blockCount = 0;
 
 
-  /** The size of a single block. */
-  private final int blockSize;
-
   /** Executor service to carry out the parallel upload requests. */
   /** Executor service to carry out the parallel upload requests. */
   private final ListeningExecutorService executorService;
   private final ListeningExecutorService executorService;
 
 
+  /** The etag of the blob. */
+  private final String eTag;
+
   /** ABFS instance to be held by the output stream to avoid GC close. */
   /** ABFS instance to be held by the output stream to avoid GC close. */
   private final BackReference fsBackRef;
   private final BackReference fsBackRef;
 
 
+  /** The service type at initialization. */
+  private final AbfsServiceType serviceTypeAtInit;
+
+  /** Indicates whether DFS to Blob fallback is enabled. */
+  private final boolean isDFSToBlobFallbackEnabled;
+
+  /** The current executing service type. */
+  private AbfsServiceType currentExecutingServiceType;
+
+  /** The handler for managing Azure ingress, marked as volatile to ensure visibility across threads. */
+  private volatile AzureIngressHandler ingressHandler;
+
+  /** The handler for managing Abfs client operations. */
+  private final AbfsClientHandler clientHandler;
+
   public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
   public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
       throws IOException {
       throws IOException {
-    this.client = abfsOutputStreamContext.getClient();
     this.statistics = abfsOutputStreamContext.getStatistics();
     this.statistics = abfsOutputStreamContext.getStatistics();
     this.path = abfsOutputStreamContext.getPath();
     this.path = abfsOutputStreamContext.getPath();
     this.position = abfsOutputStreamContext.getPosition();
     this.position = abfsOutputStreamContext.getPosition();
@@ -144,7 +161,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     this.disableOutputStreamFlush = abfsOutputStreamContext
     this.disableOutputStreamFlush = abfsOutputStreamContext
             .isDisableOutputStreamFlush();
             .isDisableOutputStreamFlush();
     this.enableSmallWriteOptimization
     this.enableSmallWriteOptimization
-        = abfsOutputStreamContext.isEnableSmallWriteOptimization();
+        = abfsOutputStreamContext.isSmallWriteSupported();
     this.isAppendBlob = abfsOutputStreamContext.isAppendBlob();
     this.isAppendBlob = abfsOutputStreamContext.isAppendBlob();
     this.lastError = null;
     this.lastError = null;
     this.lastFlushOffset = 0;
     this.lastFlushOffset = 0;
@@ -155,6 +172,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     this.outputStreamStatistics = abfsOutputStreamContext.getStreamStatistics();
     this.outputStreamStatistics = abfsOutputStreamContext.getStreamStatistics();
     this.fsBackRef = abfsOutputStreamContext.getFsBackRef();
     this.fsBackRef = abfsOutputStreamContext.getFsBackRef();
     this.contextEncryptionAdapter = abfsOutputStreamContext.getEncryptionAdapter();
     this.contextEncryptionAdapter = abfsOutputStreamContext.getEncryptionAdapter();
+    this.eTag = abfsOutputStreamContext.getETag();
 
 
     if (this.isAppendBlob) {
     if (this.isAppendBlob) {
       this.maxConcurrentRequestCount = 1;
       this.maxConcurrentRequestCount = 1;
@@ -177,12 +195,185 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     this.tracingContext.setOperation(FSOperationType.WRITE);
     this.tracingContext.setOperation(FSOperationType.WRITE);
     this.ioStatistics = outputStreamStatistics.getIOStatistics();
     this.ioStatistics = outputStreamStatistics.getIOStatistics();
     this.blockFactory = abfsOutputStreamContext.getBlockFactory();
     this.blockFactory = abfsOutputStreamContext.getBlockFactory();
-    this.blockSize = bufferSize;
-    // create that first block. This guarantees that an open + close sequence
-    // writes a 0-byte entry.
-    createBlockIfNeeded();
+    this.isDFSToBlobFallbackEnabled
+        = abfsOutputStreamContext.isDFSToBlobFallbackEnabled();
+    this.serviceTypeAtInit = abfsOutputStreamContext.getIngressServiceType();
+    this.currentExecutingServiceType = abfsOutputStreamContext.getIngressServiceType();
+    this.clientHandler = abfsOutputStreamContext.getClientHandler();
+    createIngressHandler(serviceTypeAtInit,
+        abfsOutputStreamContext.getBlockFactory(), bufferSize, false, null);
+  }
+
+  /**
+   * Retrieves the current ingress handler.
+   *
+   * @return the current {@link AzureIngressHandler}.
+   */
+  public AzureIngressHandler getIngressHandler() {
+    return ingressHandler;
+  }
+
+  private final Lock lock = new ReentrantLock();
+
+  private volatile boolean switchCompleted = false;
+
+  /**
+   * Creates or retrieves an existing Azure ingress handler based on the service type and provided parameters.
+   * <p>
+   * If the `ingressHandler` is already initialized and the switch operation is complete, the existing
+   * handler is returned without acquiring a lock to minimize performance overhead.
+   * If the `ingressHandler` is initialized but the switch is incomplete, a lock is acquired to safely modify
+   * or create a new handler. Double-checked locking is used to ensure thread safety while minimizing the
+   * time spent in the critical section.
+   * If the `ingressHandler` is `null`, the handler is safely initialized outside of the lock as no other
+   * thread would be modifying it.
+   * </p>
+   *
+   * @param serviceType   The type of Azure service to handle (e.g., ABFS, Blob, etc.).
+   * @param blockFactory  The factory to create data blocks used in the handler.
+   * @param bufferSize    The buffer size used by the handler for data processing.
+   * @param isSwitch      A flag indicating whether a switch operation is in progress.
+   * @param blockManager  The manager responsible for handling blocks of data during processing.
+   *
+   * @return The initialized or existing Azure ingress handler.
+   * @throws IOException If an I/O error occurs during handler creation or data processing.
+   */
+  private AzureIngressHandler createIngressHandler(AbfsServiceType serviceType,
+      DataBlocks.BlockFactory blockFactory,
+      int bufferSize, boolean isSwitch, AzureBlockManager blockManager) throws IOException {
+    if (ingressHandler != null) {
+      if (switchCompleted) {
+        return ingressHandler; // Return the handler if it's already initialized and the switch is completed
+      }
+      // If the switch is incomplete, lock to safely modify
+      lock.lock();
+      try {
+        // Double-check the condition after acquiring the lock
+        if (switchCompleted) {
+          return ingressHandler; // Return the handler if it's now completed
+        }
+        // If the switch is still incomplete, create a new handler
+        return createNewHandler(serviceType, blockFactory, bufferSize, isSwitch, blockManager);
+      } finally {
+        lock.unlock();
+      }
+    }
+    // If ingressHandler is null, no lock is needed; safely initialize it outside the lock
+    return createNewHandler(serviceType, blockFactory, bufferSize, isSwitch, blockManager);
   }
   }
 
 
+  // Helper method to create a new handler, used in both scenarios (locked and unlocked)
+  private AzureIngressHandler createNewHandler(AbfsServiceType serviceType,
+      DataBlocks.BlockFactory blockFactory,
+      int bufferSize,
+      boolean isSwitch,
+      AzureBlockManager blockManager) throws IOException {
+    this.client = clientHandler.getClient(serviceType);
+    if (isDFSToBlobFallbackEnabled && serviceTypeAtInit != AbfsServiceType.DFS) {
+      throw new InvalidConfigurationValueException(
+          "The ingress service type must be configured as DFS");
+    }
+    if (isDFSToBlobFallbackEnabled && !isSwitch) {
+      ingressHandler = new AzureDfsToBlobIngressFallbackHandler(this,
+          blockFactory, bufferSize, eTag, clientHandler);
+    } else if (serviceType == AbfsServiceType.BLOB) {
+      ingressHandler = new AzureBlobIngressHandler(this, blockFactory,
+          bufferSize, eTag, clientHandler, blockManager);
+    } else {
+      ingressHandler = new AzureDFSIngressHandler(this, blockFactory,
+          bufferSize, eTag, clientHandler);
+    }
+    if (isSwitch) {
+      switchCompleted = true;
+    }
+    return ingressHandler;
+  }
+
+  /**
+   * Switches the current ingress handler and service type if necessary.
+   *
+   * @throws IOException if there is an error creating the new ingress handler.
+   */
+  protected void switchHandler() throws IOException {
+    if (serviceTypeAtInit != currentExecutingServiceType) {
+      LOG.debug("Handler switch not required as serviceTypeAtInit {} is different from currentExecutingServiceType {}. "
+              + "This check prevents the handler from being switched more than once.",
+          serviceTypeAtInit, currentExecutingServiceType);
+      return;
+    }
+    if (serviceTypeAtInit == AbfsServiceType.BLOB) {
+      currentExecutingServiceType = AbfsServiceType.DFS;
+    } else {
+      currentExecutingServiceType = AbfsServiceType.BLOB;
+    }
+    LOG.info("Switching ingress handler to different service type: {}", currentExecutingServiceType);
+    ingressHandler = createIngressHandler(currentExecutingServiceType,
+        blockFactory, bufferSize, true, getBlockManager());
+  }
+
+  /**
+   * Buffers data in the given block.
+   *
+   * @param block the block to buffer data into.
+   * @param data the data to buffer.
+   * @param off the offset in the data array.
+   * @param length the length of data to buffer.
+   * @return the number of bytes buffered.
+   * @throws IOException if there is an error buffering the data.
+   */
+  private int bufferData(AbfsBlock block,
+      final byte[] data,
+      final int off,
+      final int length)
+      throws IOException {
+    return getIngressHandler().bufferData(block, data, off, length);
+  }
+
+  /**
+   * Performs a remote write operation.
+   *
+   * @param blockToUpload the block to upload.
+   * @param uploadData the data to upload.
+   * @param reqParams the parameters for the append request.
+   * @param tracingContext the tracing context for the operation.
+   * @return the result of the remote write operation.
+   * @throws IOException if there is an error during the remote write.
+   */
+  private AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
+      DataBlocks.BlockUploadData uploadData,
+      AppendRequestParameters reqParams,
+      TracingContext tracingContext)
+      throws IOException {
+    return getIngressHandler().remoteWrite(blockToUpload, uploadData, reqParams,
+        tracingContext);
+  }
+
+  /**
+   * Flushes data remotely.
+   *
+   * @param offset the offset to flush.
+   * @param retainUncommitedData whether to retain uncommitted data.
+   * @param isClose whether this is a close operation.
+   * @param leaseId the lease ID for the operation.
+   * @param tracingContext the tracing context for the operation.
+   * @return the result of the remote flush operation.
+   * @throws IOException if there is an error during the remote flush.
+   */
+  private AbfsRestOperation remoteFlush(final long offset,
+      final boolean retainUncommitedData,
+      final boolean isClose,
+      final String leaseId,
+      TracingContext tracingContext)
+      throws IOException {
+    return getIngressHandler().remoteFlush(offset, retainUncommitedData,
+        isClose, leaseId, tracingContext);
+  }
+
+  /**
+   * Creates a new output stream ID.
+   *
+   * @return the newly created output stream ID.
+   */
   private String createOutputStreamId() {
   private String createOutputStreamId() {
     return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN);
     return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN);
   }
   }
@@ -226,6 +417,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
   @Override
   @Override
   public synchronized void write(final byte[] data, final int off, final int length)
   public synchronized void write(final byte[] data, final int off, final int length)
       throws IOException {
       throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
     // validate if data is not null and index out of bounds.
     // validate if data is not null and index out of bounds.
     DataBlocks.validateWriteArgs(data, off, length);
     DataBlocks.validateWriteArgs(data, off, length);
     maybeThrowLastError();
     maybeThrowLastError();
@@ -237,8 +431,13 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     if (hasLease() && isLeaseFreed()) {
     if (hasLease() && isLeaseFreed()) {
       throw new PathIOException(path, ERR_WRITE_WITHOUT_LEASE);
       throw new PathIOException(path, ERR_WRITE_WITHOUT_LEASE);
     }
     }
-    DataBlocks.DataBlock block = createBlockIfNeeded();
-    int written = block.write(data, off, length);
+    if (length == 0) {
+      LOG.debug("No data to write, length is 0 for path: {}", path);
+      return;
+    }
+
+    AbfsBlock block = createBlockIfNeeded(position);
+    int written = bufferData(block, data, off, length);
     int remainingCapacity = block.remainingCapacity();
     int remainingCapacity = block.remainingCapacity();
 
 
     if (written < length) {
     if (written < length) {
@@ -264,14 +463,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
    * @return the active block; null if there isn't one.
    * @return the active block; null if there isn't one.
    * @throws IOException on any failure to create
    * @throws IOException on any failure to create
    */
    */
-  private synchronized DataBlocks.DataBlock createBlockIfNeeded()
+  private synchronized AbfsBlock createBlockIfNeeded(long position)
       throws IOException {
       throws IOException {
-    if (activeBlock == null) {
-      blockCount++;
-      activeBlock = blockFactory
-          .create(blockCount, this.blockSize, outputStreamStatistics);
-    }
-    return activeBlock;
+    return getBlockManager().createBlock(position);
   }
   }
 
 
   /**
   /**
@@ -281,13 +475,17 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
    *                     initializing the upload, or if a previous operation has failed.
    *                     initializing the upload, or if a previous operation has failed.
    */
    */
   private synchronized void uploadCurrentBlock() throws IOException {
   private synchronized void uploadCurrentBlock() throws IOException {
-    checkState(hasActiveBlock(), "No active block");
-    LOG.debug("Writing block # {}", blockCount);
+    checkState(getBlockManager().hasActiveBlock(),
+        "No active block");
+    LOG.debug("Writing block # {}", getBlockManager().getBlockCount());
     try {
     try {
-      uploadBlockAsync(getActiveBlock(), false, false);
+      uploadBlockAsync(getBlockManager().getActiveBlock(),
+          false, false);
     } finally {
     } finally {
-      // set the block to null, so the next write will create a new block.
-      clearActiveBlock();
+      if (getBlockManager().hasActiveBlock()) {
+        // set the block to null, so the next write will create a new block.
+        getBlockManager().clearActiveBlock();
+      }
     }
     }
   }
   }
 
 
@@ -298,11 +496,11 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
    * @param blockToUpload    block to upload.
    * @param blockToUpload    block to upload.
    * @throws IOException     upload failure
    * @throws IOException     upload failure
    */
    */
-  private void uploadBlockAsync(DataBlocks.DataBlock blockToUpload,
+  private void uploadBlockAsync(AbfsBlock blockToUpload,
       boolean isFlush, boolean isClose)
       boolean isFlush, boolean isClose)
       throws IOException {
       throws IOException {
     if (this.isAppendBlob) {
     if (this.isAppendBlob) {
-      writeAppendBlobCurrentBufferToService();
+      getIngressHandler().writeAppendBlobCurrentBufferToService();
       return;
       return;
     }
     }
     if (!blockToUpload.hasData()) {
     if (!blockToUpload.hasData()) {
@@ -319,9 +517,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     final Future<Void> job =
     final Future<Void> job =
         executorService.submit(() -> {
         executorService.submit(() -> {
           AbfsPerfTracker tracker =
           AbfsPerfTracker tracker =
-              client.getAbfsPerfTracker();
+              getClient().getAbfsPerfTracker();
           try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
           try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
-              "writeCurrentBufferToService", "append")) {
+              "writeCurrentBufferToService", APPEND_ACTION)) {
             AppendRequestParameters.Mode
             AppendRequestParameters.Mode
                 mode = APPEND_MODE;
                 mode = APPEND_MODE;
             if (isFlush & isClose) {
             if (isFlush & isClose) {
@@ -338,9 +536,15 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
              */
              */
             AppendRequestParameters reqParams = new AppendRequestParameters(
             AppendRequestParameters reqParams = new AppendRequestParameters(
                 offset, 0, bytesLength, mode, false, leaseId, isExpectHeaderEnabled);
                 offset, 0, bytesLength, mode, false, leaseId, isExpectHeaderEnabled);
-            AbfsRestOperation op = getClient().append(path,
-                blockUploadData.toByteArray(), reqParams, cachedSasToken.get(),
-                contextEncryptionAdapter, new TracingContext(tracingContext));
+            AbfsRestOperation op;
+            try {
+              op = remoteWrite(blockToUpload, blockUploadData, reqParams, tracingContext);
+            } catch (InvalidIngressServiceException ex) {
+              LOG.debug("InvalidIngressServiceException caught for path: {}, switching handler and retrying remoteWrite.", getPath());
+              switchHandler();
+              // retry the operation with switched handler.
+              op = remoteWrite(blockToUpload, blockUploadData, reqParams, tracingContext);
+            }
             cachedSasToken.update(op.getSasToken());
             cachedSasToken.update(op.getSasToken());
             perfInfo.registerResult(op.getResult());
             perfInfo.registerResult(op.getResult());
             perfInfo.registerSuccess(true);
             perfInfo.registerSuccess(true);
@@ -361,7 +565,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
    * @param ex Exception caught.
    * @param ex Exception caught.
    * @throws IOException Throws the lastError.
    * @throws IOException Throws the lastError.
    */
    */
-  private void failureWhileSubmit(Exception ex) throws IOException {
+  void failureWhileSubmit(Exception ex) throws IOException {
     if (ex instanceof AbfsRestOperationException) {
     if (ex instanceof AbfsRestOperationException) {
       if (((AbfsRestOperationException) ex).getStatusCode()
       if (((AbfsRestOperationException) ex).getStatusCode()
           == HttpURLConnection.HTTP_NOT_FOUND) {
           == HttpURLConnection.HTTP_NOT_FOUND) {
@@ -376,43 +580,15 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     throw lastError;
     throw lastError;
   }
   }
 
 
-  /**
-   * Synchronized accessor to the active block.
-   *
-   * @return the active block; null if there isn't one.
-   */
-  private synchronized DataBlocks.DataBlock getActiveBlock() {
-    return activeBlock;
-  }
-
-  /**
-   * Predicate to query whether or not there is an active block.
-   *
-   * @return true if there is an active block.
-   */
-  private synchronized boolean hasActiveBlock() {
-    return activeBlock != null;
-  }
-
   /**
   /**
    * Is there an active block and is there any data in it to upload?
    * Is there an active block and is there any data in it to upload?
    *
    *
    * @return true if there is some data to upload in an active block else false.
    * @return true if there is some data to upload in an active block else false.
    */
    */
-  private boolean hasActiveBlockDataToUpload() {
-    return hasActiveBlock() && getActiveBlock().hasData();
-  }
-
-  /**
-   * Clear the active block.
-   */
-  private void clearActiveBlock() {
-    if (activeBlock != null) {
-      LOG.debug("Clearing active block");
-    }
-    synchronized (this) {
-      activeBlock = null;
-    }
+  boolean hasActiveBlockDataToUpload() {
+    AzureBlockManager blockManager = getBlockManager();
+    AbfsBlock activeBlock = blockManager.getActiveBlock();
+    return blockManager.hasActiveBlock() && activeBlock.hasData();
   }
   }
 
 
   /**
   /**
@@ -472,10 +648,20 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     }
     }
   }
   }
 
 
+  /**
+   * Retrieves the stream ID associated with this output stream.
+   *
+   * @return the stream ID of this output stream.
+   */
   public String getStreamID() {
   public String getStreamID() {
     return outputStreamId;
     return outputStreamId;
   }
   }
 
 
+  /**
+   * Registers a listener for this output stream.
+   *
+   * @param listener1 the listener to register.
+   */
   public void registerListener(Listener listener1) {
   public void registerListener(Listener listener1) {
     listener = listener1;
     listener = listener1;
     tracingContext.setListener(listener);
     tracingContext.setListener(listener);
@@ -522,13 +708,23 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
       bufferIndex = 0;
       bufferIndex = 0;
       closed = true;
       closed = true;
       writeOperations.clear();
       writeOperations.clear();
-      if (hasActiveBlock()) {
-        clearActiveBlock();
+      if (getBlockManager().hasActiveBlock()) {
+        getBlockManager().clearActiveBlock();
       }
       }
     }
     }
     LOG.debug("Closing AbfsOutputStream : {}", this);
     LOG.debug("Closing AbfsOutputStream : {}", this);
   }
   }
 
 
+  /**
+   * Flushes the buffered data to the Azure Blob Storage service.
+   * This method checks if a small write optimization can be applied, and if so, delegates
+   * the flush operation to {@link #smallWriteOptimizedflushInternal(boolean)}.
+   * Otherwise, it uploads the active block synchronously, flushes the written bytes to
+   * the service, and resets the number of appends to the server since the last flush.
+   *
+   * @param isClose indicates whether this flush operation is part of a close operation.
+   * @throws IOException if an I/O error occurs during the flush operation.
+   */
   private synchronized void flushInternal(boolean isClose) throws IOException {
   private synchronized void flushInternal(boolean isClose) throws IOException {
     maybeThrowLastError();
     maybeThrowLastError();
 
 
@@ -550,17 +746,35 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     numOfAppendsToServerSinceLastFlush = 0;
     numOfAppendsToServerSinceLastFlush = 0;
   }
   }
 
 
+  /**
+   * Flushes the buffered data to the Azure Blob Storage service with small write optimization.
+   * This method uploads the active block asynchronously, waits for appends to complete, shrinks
+   * the write operation queue, checks for any previous errors, and resets the number of appends
+   * to the server since the last flush.
+   *
+   * @param isClose indicates whether this flush operation is part of a close operation.
+   * @throws IOException if an I/O error occurs during the flush operation.
+   */
   private synchronized void smallWriteOptimizedflushInternal(boolean isClose) throws IOException {
   private synchronized void smallWriteOptimizedflushInternal(boolean isClose) throws IOException {
     // writeCurrentBufferToService will increment numOfAppendsToServerSinceLastFlush
     // writeCurrentBufferToService will increment numOfAppendsToServerSinceLastFlush
-    uploadBlockAsync(getActiveBlock(), true, isClose);
+    uploadBlockAsync(getBlockManager().getActiveBlock(),
+        true, isClose);
     waitForAppendsToComplete();
     waitForAppendsToComplete();
     shrinkWriteOperationQueue();
     shrinkWriteOperationQueue();
     maybeThrowLastError();
     maybeThrowLastError();
     numOfAppendsToServerSinceLastFlush = 0;
     numOfAppendsToServerSinceLastFlush = 0;
   }
   }
 
 
+  /**
+   * Asynchronously flushes the buffered data to the Azure Blob Storage service.
+   * This method checks for any previous errors, uploads the current block if needed,
+   * waits for appends to complete, and then performs an async flush operation.
+   *
+   * @throws IOException if an I/O error occurs during the flush operation.
+   */
   private synchronized void flushInternalAsync() throws IOException {
   private synchronized void flushInternalAsync() throws IOException {
     maybeThrowLastError();
     maybeThrowLastError();
+    // Upload the current block if there is active block data.
     if (hasActiveBlockDataToUpload()) {
     if (hasActiveBlockDataToUpload()) {
       uploadCurrentBlock();
       uploadCurrentBlock();
     }
     }
@@ -569,50 +783,16 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
   }
   }
 
 
   /**
   /**
-   * Appending the current active data block to service. Clearing the active
-   * data block and releasing all buffered data.
-   * @throws IOException if there is any failure while starting an upload for
-   *                     the dataBlock or while closing the BlockUploadData.
+   * Waits for all write operations (appends) to complete.
+   * This method iterates through the list of write operations and waits for their tasks
+   * to finish. If an error occurs during the operation, it is handled appropriately.
+   *
+   * @throws IOException if an I/O error occurs while waiting for appends to complete.
    */
    */
-  private void writeAppendBlobCurrentBufferToService() throws IOException {
-    DataBlocks.DataBlock activeBlock = getActiveBlock();
-    // No data, return.
-    if (!hasActiveBlockDataToUpload()) {
-      return;
-    }
-
-    final int bytesLength = activeBlock.dataSize();
-    DataBlocks.BlockUploadData uploadData = activeBlock.startUpload();
-    clearActiveBlock();
-    outputStreamStatistics.writeCurrentBuffer();
-    outputStreamStatistics.bytesToUpload(bytesLength);
-    final long offset = position;
-    position += bytesLength;
-    AbfsPerfTracker tracker = client.getAbfsPerfTracker();
-    try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
-        "writeCurrentBufferToService", "append")) {
-      AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0,
-          bytesLength, APPEND_MODE, true, leaseId, isExpectHeaderEnabled);
-      AbfsRestOperation op = getClient().append(path, uploadData.toByteArray(),
-          reqParams, cachedSasToken.get(), contextEncryptionAdapter,
-          new TracingContext(tracingContext));
-      cachedSasToken.update(op.getSasToken());
-      outputStreamStatistics.uploadSuccessful(bytesLength);
-
-      perfInfo.registerResult(op.getResult());
-      perfInfo.registerSuccess(true);
-      return;
-    } catch (Exception ex) {
-      outputStreamStatistics.uploadFailed(bytesLength);
-      failureWhileSubmit(ex);
-    } finally {
-      IOUtils.close(uploadData, activeBlock);
-    }
-  }
-
   private synchronized void waitForAppendsToComplete() throws IOException {
   private synchronized void waitForAppendsToComplete() throws IOException {
     for (WriteOperation writeOperation : writeOperations) {
     for (WriteOperation writeOperation : writeOperations) {
       try {
       try {
+        // Wait for the write operation task to complete.
         writeOperation.task.get();
         writeOperation.task.get();
       } catch (Exception ex) {
       } catch (Exception ex) {
         outputStreamStatistics.uploadFailed(writeOperation.length);
         outputStreamStatistics.uploadFailed(writeOperation.length);
@@ -631,20 +811,46 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     }
     }
   }
   }
 
 
+  /**
+   * Flushes the written bytes to the Azure Blob Storage service, ensuring all
+   * appends are completed. This method is typically called during a close operation.
+   *
+   * @param isClose indicates whether this flush is happening as part of a close operation.
+   * @throws IOException if an I/O error occurs during the flush operation.
+   */
   private synchronized void flushWrittenBytesToService(boolean isClose) throws IOException {
   private synchronized void flushWrittenBytesToService(boolean isClose) throws IOException {
+    // Ensure all appends are completed before flushing.
     waitForAppendsToComplete();
     waitForAppendsToComplete();
+    // Flush the written bytes to the service.
     flushWrittenBytesToServiceInternal(position, false, isClose);
     flushWrittenBytesToServiceInternal(position, false, isClose);
   }
   }
 
 
+  /**
+   * Asynchronously flushes the written bytes to the Azure Blob Storage service.
+   * This method ensures that the write operation queue is managed and only flushes
+   * if there are uncommitted data beyond the last flush offset.
+   *
+   * @throws IOException if an I/O error occurs during the flush operation.
+   */
   private synchronized void flushWrittenBytesToServiceAsync() throws IOException {
   private synchronized void flushWrittenBytesToServiceAsync() throws IOException {
+    // Manage the write operation queue to ensure efficient writes
     shrinkWriteOperationQueue();
     shrinkWriteOperationQueue();
 
 
+    // Only flush if there are uncommitted data beyond the last flush offset
     if (this.lastTotalAppendOffset > this.lastFlushOffset) {
     if (this.lastTotalAppendOffset > this.lastFlushOffset) {
       this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true,
       this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true,
         false/*Async flush on close not permitted*/);
         false/*Async flush on close not permitted*/);
     }
     }
   }
   }
 
 
+  /**
+   * Flushes the written bytes to the Azure Blob Storage service.
+   *
+   * @param offset                the offset up to which data needs to be flushed.
+   * @param retainUncommitedData whether to retain uncommitted data after flush.
+   * @param isClose               whether this flush is happening as part of a close operation.
+   * @throws IOException if an I/O error occurs.
+   */
   private synchronized void flushWrittenBytesToServiceInternal(final long offset,
   private synchronized void flushWrittenBytesToServiceInternal(final long offset,
       final boolean retainUncommitedData, final boolean isClose) throws IOException {
       final boolean retainUncommitedData, final boolean isClose) throws IOException {
     // flush is called for appendblob only on close
     // flush is called for appendblob only on close
@@ -652,24 +858,43 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
       return;
       return;
     }
     }
 
 
+    // Tracker to monitor performance metrics
     AbfsPerfTracker tracker = client.getAbfsPerfTracker();
     AbfsPerfTracker tracker = client.getAbfsPerfTracker();
     try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
     try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
             "flushWrittenBytesToServiceInternal", "flush")) {
             "flushWrittenBytesToServiceInternal", "flush")) {
-      AbfsRestOperation op = getClient().flush(path, offset, retainUncommitedData,
-          isClose, cachedSasToken.get(), leaseId, contextEncryptionAdapter,
-          new TracingContext(tracingContext));
-      cachedSasToken.update(op.getSasToken());
-      perfInfo.registerResult(op.getResult()).registerSuccess(true);
-    } catch (AzureBlobFileSystemException ex) {
-      if (ex instanceof AbfsRestOperationException) {
-        if (((AbfsRestOperationException) ex).getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
+      AbfsRestOperation op;
+      try {
+        // Attempt to flush data to the remote service.
+        op = remoteFlush(offset, retainUncommitedData, isClose, leaseId,
+            tracingContext);
+      } catch (InvalidIngressServiceException ex) {
+        LOG.debug("InvalidIngressServiceException caught for path: {}, switching handler and retrying remoteFlush.", getPath());
+        // If an invalid ingress service is encountered, switch handler and retry.
+        switchHandler();
+        op = remoteFlush(offset, retainUncommitedData, isClose, leaseId,
+            tracingContext);
+      } catch (AzureBlobFileSystemException ex) {
+        // Handle specific Azure Blob FileSystem exceptions
+        if (ex instanceof AbfsRestOperationException
+            && ((AbfsRestOperationException) ex).getStatusCode()
+                == HttpURLConnection.HTTP_NOT_FOUND) {
           throw new FileNotFoundException(ex.getMessage());
           throw new FileNotFoundException(ex.getMessage());
         }
         }
+        // Store the last error and rethrow it
+        lastError = new IOException(ex);
+        throw lastError;
       }
       }
-      lastError = new IOException(ex);
-      throw lastError;
+
+      if (op != null) {
+        // Update the cached SAS token if the operation was successful
+        cachedSasToken.update(op.getSasToken());
+        // Register the result and mark the operation as successful
+        perfInfo.registerResult(op.getResult()).registerSuccess(true);
+      }
+
+      // Update the last flush offset
+      this.lastFlushOffset = offset;
     }
     }
-    this.lastFlushOffset = offset;
   }
   }
 
 
   /**
   /**
@@ -786,18 +1011,165 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     return sb.toString();
     return sb.toString();
   }
   }
 
 
+  /**
+   * Gets the reference to the file system back.
+   *
+   * @return The back reference to the file system.
+   */
   @VisibleForTesting
   @VisibleForTesting
   BackReference getFsBackRef() {
   BackReference getFsBackRef() {
     return fsBackRef;
     return fsBackRef;
   }
   }
 
 
+  /**
+   * Gets the executor service used for asynchronous operations.
+   *
+   * @return The executor service.
+   */
   @VisibleForTesting
   @VisibleForTesting
   ListeningExecutorService getExecutorService() {
   ListeningExecutorService getExecutorService() {
     return executorService;
     return executorService;
   }
   }
 
 
+  /**
+   * Gets the Azure Blob Storage client.
+   *
+   * @return The Azure Blob Storage client.
+   */
   @VisibleForTesting
   @VisibleForTesting
   AbfsClient getClient() {
   AbfsClient getClient() {
     return client;
     return client;
   }
   }
+
+  /**
+   * Gets the Azure Blob Storage clientHandler.
+   *
+   * @return The Azure Blob Storage clientHandler.
+   */
+  public AbfsClientHandler getClientHandler() {
+    return clientHandler;
+  }
+
+  /**
+   * Gets the path associated with this stream.
+   *
+   * @return The path of the stream.
+   */
+  public String getPath() {
+    return this.path;
+  }
+
+  /**
+   * Gets the current position in the stream.
+   *
+   * @return The current position in the stream.
+   */
+  public synchronized long getPosition() {
+    return position;
+  }
+
+  /**
+   * Sets the position in the stream.
+   *
+   * @param position The position to set.
+   */
+  public synchronized void setPosition(final long position) {
+    this.position = position;
+  }
+
+  /**
+   * Gets the cached SAS token string for authentication.
+   *
+   * @return The cached SAS token string.
+   */
+  public String getCachedSasTokenString() {
+    return cachedSasToken.get();
+  }
+
+  /**
+   * Gets the context encryption adapter.
+   *
+   * @return The context encryption adapter.
+   */
+  public ContextEncryptionAdapter getContextEncryptionAdapter() {
+    return contextEncryptionAdapter;
+  }
+
+  /**
+   * Gets the Azure Block Manager associated with this stream.
+   *
+   * @return The Azure Block Manager.
+   */
+  public AzureBlockManager getBlockManager() {
+    return getIngressHandler().getBlockManager();
+  }
+
+  /**
+   * Gets the tracing context for operations.
+   *
+   * @return The tracing context.
+   */
+  public TracingContext getTracingContext() {
+    return tracingContext;
+  }
+
+  /**
+   * Checks if the DFS to blob fallback mechanism is enabled.
+   *
+   * @return True if the DFS to blob fallback is enabled, otherwise false.
+   */
+  public boolean isDFSToBlobFallbackEnabled() {
+    return isDFSToBlobFallbackEnabled;
+  }
+
+  /**
+   * Checks if the 'Expect' header is enabled for HTTP requests.
+   *
+   * @return True if the 'Expect' header is enabled, otherwise false.
+   */
+  public boolean isExpectHeaderEnabled() {
+    return isExpectHeaderEnabled;
+  }
+
+  /**
+   * Gets the lease ID associated with the stream.
+   *
+   * @return The lease ID.
+   */
+  public String getLeaseId() {
+    return leaseId;
+  }
+
+  /**
+   * Gets the cached SAS token object.
+   *
+   * @return The cached SAS token object.
+   */
+  public CachedSASToken getCachedSasToken() {
+    return cachedSasToken;
+  }
+
+  /**
+   * Checks if the stream is associated with an append blob.
+   *
+   * @return True if the stream is for an append blob, otherwise false.
+   */
+  public boolean isAppendBlob() {
+    return isAppendBlob;
+  }
+
+  /**
+   * Checks if all write operation tasks are done.
+   *
+   * @return True if all write operation tasks are done, false otherwise.
+   */
+  @VisibleForTesting
+  public Boolean areWriteOperationsTasksDone() {
+    for (WriteOperation writeOperation : writeOperations) {
+      if (!writeOperation.task.isDone()) {
+        return false;
+      }
+    }
+    return true;
+  }
 }
 }

+ 59 - 8
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.azurebfs.services;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ExecutorService;
 
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.impl.BackReference;
 import org.apache.hadoop.fs.impl.BackReference;
@@ -57,8 +58,6 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
 
 
   private int blockOutputActiveBlocks;
   private int blockOutputActiveBlocks;
 
 
-  private AbfsClient client;
-
   private long position;
   private long position;
 
 
   private FileSystem.Statistics statistics;
   private FileSystem.Statistics statistics;
@@ -72,6 +71,14 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
   /** A BackReference to the FS instance that created this OutputStream. */
   /** A BackReference to the FS instance that created this OutputStream. */
   private BackReference fsBackRef;
   private BackReference fsBackRef;
 
 
+  private AbfsServiceType ingressServiceType;
+
+  private boolean isDFSToBlobFallbackEnabled;
+
+  private String eTag;
+
+  private AbfsClientHandler clientHandler;
+
   public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) {
   public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) {
     super(sasTokenRenewPeriodForStreamsInSeconds);
     super(sasTokenRenewPeriodForStreamsInSeconds);
   }
   }
@@ -128,9 +135,9 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
   }
   }
 
 
 
 
-  public AbfsOutputStreamContext withClient(
-      final AbfsClient client) {
-    this.client = client;
+  public AbfsOutputStreamContext withClientHandler(
+      final AbfsClientHandler clientHandler) {
+    this.clientHandler = clientHandler;
     return this;
     return this;
   }
   }
 
 
@@ -164,12 +171,31 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
     return this;
     return this;
   }
   }
 
 
+  public AbfsOutputStreamContext withETag(
+      final String eTag) {
+    this.eTag = eTag;
+    return this;
+  }
+
   public AbfsOutputStreamContext withAbfsBackRef(
   public AbfsOutputStreamContext withAbfsBackRef(
       final BackReference fsBackRef) {
       final BackReference fsBackRef) {
     this.fsBackRef = fsBackRef;
     this.fsBackRef = fsBackRef;
     return this;
     return this;
   }
   }
 
 
+  public AbfsOutputStreamContext withIngressServiceType(
+      final AbfsServiceType serviceType) {
+    this.ingressServiceType = serviceType;
+    return this;
+  }
+
+  public AbfsOutputStreamContext withDFSToBlobFallbackEnabled(
+      final boolean isDFSToBlobFallbackEnabled) {
+    this.isDFSToBlobFallbackEnabled = isDFSToBlobFallbackEnabled;
+    return this;
+  }
+
+
   public AbfsOutputStreamContext build() {
   public AbfsOutputStreamContext build() {
     // Validation of parameters to be done here.
     // Validation of parameters to be done here.
     if (streamStatistics == null) {
     if (streamStatistics == null) {
@@ -261,9 +287,6 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
     return blockOutputActiveBlocks;
     return blockOutputActiveBlocks;
   }
   }
 
 
-  public AbfsClient getClient() {
-    return client;
-  }
 
 
   public FileSystem.Statistics getStatistics() {
   public FileSystem.Statistics getStatistics() {
     return statistics;
     return statistics;
@@ -288,4 +311,32 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
   public BackReference getFsBackRef() {
   public BackReference getFsBackRef() {
     return fsBackRef;
     return fsBackRef;
   }
   }
+
+  public AbfsServiceType getIngressServiceType() {
+    return ingressServiceType;
+  }
+
+  public boolean isDFSToBlobFallbackEnabled() {
+    return isDFSToBlobFallbackEnabled;
+  }
+
+  public String getETag() {
+    return eTag;
+  }
+
+  public AbfsClientHandler getClientHandler() {
+    return clientHandler;
+  }
+
+  /**
+   * Checks if small write is supported based on the current configuration.
+   *
+   * @return true if small write is supported, false otherwise.
+   */
+  protected boolean isSmallWriteSupported() {
+    if (!enableSmallWriteOptimization) {
+      return false;
+    }
+    return !(ingressServiceType == AbfsServiceType.BLOB || isDFSToBlobFallbackEnabled);
+  }
 }
 }

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

@@ -56,5 +56,6 @@ public enum AbfsRestOperationType {
     GetBlobProperties,
     GetBlobProperties,
     SetBlobMetadata,
     SetBlobMetadata,
     DeleteBlob,
     DeleteBlob,
-    CopyBlob
+    CopyBlob,
+    AppendBlock
 }
 }

+ 194 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java

@@ -0,0 +1,194 @@
+/**
+ * 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.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.store.DataBlocks;
+
+/**
+ * Manages Azure Blob blocks for append operations.
+ */
+public class AzureBlobBlockManager extends AzureBlockManager {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsOutputStream.class);
+
+
+  /** The list of already committed blocks is stored in this list. */
+  private List<String> committedBlockEntries = new ArrayList<>();
+
+  /** The list to store blockId, position, and status. */
+  private final LinkedList<BlockEntry> blockEntryList = new LinkedList<>();
+
+
+  /**
+   * Constructs an AzureBlobBlockManager.
+   *
+   * @param abfsOutputStream the output stream
+   * @param blockFactory the block factory
+   * @param bufferSize the buffer size
+   * @throws AzureBlobFileSystemException if an error occurs
+   */
+  public AzureBlobBlockManager(AbfsOutputStream abfsOutputStream,
+      DataBlocks.BlockFactory blockFactory,
+      int bufferSize)
+      throws AzureBlobFileSystemException {
+    super(abfsOutputStream, blockFactory, bufferSize);
+    if (abfsOutputStream.getPosition() > 0 && !abfsOutputStream.isAppendBlob()) {
+      this.committedBlockEntries = getBlockList(abfsOutputStream.getTracingContext());
+    }
+    LOG.debug("Created a new Blob Block Manager for AbfsOutputStream instance {} for path {}",
+        abfsOutputStream.getStreamID(), abfsOutputStream.getPath());
+  }
+
+  /**
+   * Creates a new block.
+   *
+   * @param position the position
+   * @return the created block
+   * @throws IOException if an I/O error occurs
+   */
+  @Override
+  protected synchronized AbfsBlock createBlockInternal(long position)
+      throws IOException {
+    if (getActiveBlock() == null) {
+      setBlockCount(getBlockCount() + 1);
+      AbfsBlock activeBlock = new AbfsBlobBlock(getAbfsOutputStream(), position);
+      activeBlock.setBlockEntry(addNewEntry(activeBlock.getBlockId(), activeBlock.getOffset()));
+      setActiveBlock(activeBlock);
+    }
+    return getActiveBlock();
+  }
+
+  /**
+   * Returns block id's which are committed for the blob.
+   *
+   * @param tracingContext Tracing context object.
+   * @return list of committed block id's.
+   * @throws AzureBlobFileSystemException if an error occurs
+   */
+  private List<String> getBlockList(TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    List<String> committedBlockIdList = new ArrayList<>();
+    AbfsBlobClient blobClient = getAbfsOutputStream().getClientHandler().getBlobClient();
+    final AbfsRestOperation op = blobClient
+        .getBlockList(getAbfsOutputStream().getPath(), tracingContext);
+    if (op != null && op.getResult() != null) {
+      committedBlockIdList = op.getResult().getBlockIdList();
+    }
+    return committedBlockIdList;
+  }
+
+  /**
+   * Adds a new block entry to the block entry list.
+   * The block entry is added only if the position of the new block
+   * is greater than the position of the last block in the list.
+   *
+   * @param blockId The ID of the new block to be added.
+   * @param position The position of the new block in the stream.
+   * @return The newly added {@link BlockEntry}.
+   * @throws IOException If the position of the new block is not greater than the last block in the list.
+   */
+  private synchronized BlockEntry addNewEntry(String blockId, long position) throws IOException {
+    if (!blockEntryList.isEmpty()) {
+      BlockEntry lastEntry = blockEntryList.getLast();
+      if (position <= lastEntry.getPosition()) {
+        throw new IOException("New block position " + position  + " must be greater than the last block position "
+            + lastEntry.getPosition() + " for path " + getAbfsOutputStream().getPath());
+      }
+    }
+    BlockEntry blockEntry = new BlockEntry(blockId, position, AbfsBlockStatus.NEW);
+    blockEntryList.addLast(blockEntry);
+    LOG.debug("Added block {} at position {} with status NEW.", blockId, position);
+    return blockEntry;
+  }
+
+  /**
+   * Updates the status of an existing block entry to SUCCESS.
+   * This method is used to mark a block as successfully processed.
+   *
+   * @param block The {@link AbfsBlock} whose status needs to be updated to SUCCESS.
+   */
+  protected synchronized void updateEntry(AbfsBlock block) {
+    BlockEntry blockEntry = block.getBlockEntry();
+    blockEntry.setStatus(AbfsBlockStatus.SUCCESS);
+    LOG.debug("Added block {} at position {} with status SUCCESS.", block.getBlockId(), blockEntry.getPosition());
+  }
+
+  /**
+   * Prepares the list of blocks to commit.
+   *
+   * @return whether we have some data to commit or not.
+   * @throws IOException if an I/O error occurs
+   */
+  protected synchronized boolean hasListToCommit() throws IOException {
+    // Adds all the committed blocks if available to the list of blocks to be added in putBlockList.
+    if (blockEntryList.isEmpty()) {
+      return false; // No entries to commit
+    }
+    while (!blockEntryList.isEmpty()) {
+      BlockEntry current = blockEntryList.poll();
+      if (current.getStatus() != AbfsBlockStatus.SUCCESS) {
+        LOG.debug(
+            "Block {} with position {} has status {}, flush cannot proceed.",
+            current.getBlockId(), current.getPosition(), current.getStatus());
+        throw new IOException("Flush failed. Block " + current.getBlockId()
+            + " with position " + current.getPosition() + " has status "
+            + current.getStatus() + "for path " + getAbfsOutputStream().getPath());
+      }
+      if (!blockEntryList.isEmpty()) {
+        BlockEntry next = blockEntryList.getFirst();
+        if (current.getPosition() >= next.getPosition()) {
+          String errorMessage =
+              "Position check failed. Current block position is greater than or equal to the next block's position.\n"
+                  + "Current Block Entry:\n"
+                  + "Block ID: " + current.getBlockId()
+                  + ", Position: " + current.getPosition()
+                  + ", Status: " + current.getStatus()
+                  + ", Path: " + getAbfsOutputStream().getPath()
+                  + ", StreamID: " + getAbfsOutputStream().getStreamID()
+                  + ", Next block position: " + next.getPosition()
+                  + "\n";
+          throw new IOException(errorMessage);
+        }
+      }
+      committedBlockEntries.add(current.getBlockId());
+      LOG.debug("Block {} added to committed entries.", current.getBlockId());
+    }
+    return true;
+  }
+
+  /**
+   * Returns the block ID list.
+   *
+   * @return the block ID list
+   */
+  protected List<String> getBlockIdList() {
+    return committedBlockEntries;
+  }
+}

+ 348 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java

@@ -0,0 +1,348 @@
+/**
+ * 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.services;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+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.InvalidIngressServiceException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobAppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.store.DataBlocks;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_APPEND;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_FLUSH;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient.generateBlockListXml;
+
+public class AzureBlobIngressHandler extends AzureIngressHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsOutputStream.class);
+
+  private volatile String eTag;
+
+  private final AzureBlobBlockManager blobBlockManager;
+
+  private final AbfsBlobClient blobClient;
+
+  private final AbfsClientHandler clientHandler;
+
+  /**
+   * Constructs an AzureBlobIngressHandler.
+   *
+   * @param abfsOutputStream the AbfsOutputStream.
+   * @param blockFactory the block factory.
+   * @param bufferSize the buffer size.
+   * @param eTag the eTag.
+   * @param clientHandler the client handler.
+   * @param blockManager the block manager.
+   * @throws AzureBlobFileSystemException if an error occurs.
+   */
+  public AzureBlobIngressHandler(AbfsOutputStream abfsOutputStream,
+      DataBlocks.BlockFactory blockFactory,
+      int bufferSize, String eTag, AbfsClientHandler clientHandler, AzureBlockManager blockManager)
+      throws AzureBlobFileSystemException {
+    super(abfsOutputStream);
+    this.eTag = eTag;
+    if (blockManager instanceof AzureBlobBlockManager) {
+      this.blobBlockManager = (AzureBlobBlockManager) blockManager;
+    } else {
+      this.blobBlockManager = new AzureBlobBlockManager(abfsOutputStream,
+          blockFactory, bufferSize);
+    }
+    this.clientHandler = clientHandler;
+    this.blobClient = clientHandler.getBlobClient();
+    LOG.trace("Created a new BlobIngress Handler for AbfsOutputStream instance {} for path {}",
+        abfsOutputStream.getStreamID(), abfsOutputStream.getPath());
+  }
+
+  /**
+   * Buffers data into the specified block.
+   *
+   * @param block the block to buffer data into.
+   * @param data  the data to be buffered.
+   * @param off   the start offset in the data.
+   * @param length the number of bytes to buffer.
+   * @return the number of bytes buffered.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  protected int bufferData(AbfsBlock block,
+      final byte[] data,
+      final int off,
+      final int length)
+      throws IOException {
+    LOG.trace("Buffering data of length {} to block at offset {}", length, off);
+    return block.write(data, off, length);
+  }
+
+  /**
+   * Performs a remote write operation.
+   *
+   * @param blockToUpload the block to upload.
+   * @param uploadData    the data to upload.
+   * @param reqParams     the request parameters.
+   * @param tracingContext the tracing context.
+   * @return the resulting AbfsRestOperation.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  protected AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
+      DataBlocks.BlockUploadData uploadData,
+      AppendRequestParameters reqParams,
+      TracingContext tracingContext)
+      throws IOException {
+    BlobAppendRequestParameters blobParams = new BlobAppendRequestParameters(blockToUpload.getBlockId(), getETag());
+    reqParams.setBlobParams(blobParams);
+    AbfsRestOperation op;
+    String threadIdStr = String.valueOf(Thread.currentThread().getId());
+    TracingContext tracingContextAppend = new TracingContext(tracingContext);
+    tracingContextAppend.setIngressHandler(BLOB_APPEND + " T " + threadIdStr);
+    tracingContextAppend.setPosition(String.valueOf(blockToUpload.getOffset()));
+    try {
+      LOG.trace("Starting remote write for block with ID {} and offset {}",
+          blockToUpload.getBlockId(), blockToUpload.getOffset());
+      op = getClient().append(getAbfsOutputStream().getPath(), uploadData.toByteArray(),
+          reqParams,
+          getAbfsOutputStream().getCachedSasTokenString(),
+          getAbfsOutputStream().getContextEncryptionAdapter(),
+          tracingContextAppend);
+      blobBlockManager.updateEntry(blockToUpload);
+    } catch (AbfsRestOperationException ex) {
+      LOG.error("Error in remote write requiring handler switch for path {}", getAbfsOutputStream().getPath(), ex);
+      if (shouldIngressHandlerBeSwitched(ex)) {
+        throw getIngressHandlerSwitchException(ex);
+      }
+      LOG.error("Error in remote write for path {} and offset {}", getAbfsOutputStream().getPath(),
+          blockToUpload.getOffset(), ex);
+      throw ex;
+    }
+    return op;
+  }
+
+  /**
+   * Flushes data to the remote store.
+   *
+   * @param offset               the offset to flush.
+   * @param retainUncommitedData whether to retain uncommitted data.
+   * @param isClose              whether this is a close operation.
+   * @param leaseId              the lease ID.
+   * @param tracingContext       the tracing context.
+   * @return the resulting AbfsRestOperation.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  protected synchronized AbfsRestOperation remoteFlush(final long offset,
+      final boolean retainUncommitedData,
+      final boolean isClose,
+      final String leaseId,
+      TracingContext tracingContext)
+      throws IOException {
+    AbfsRestOperation op;
+    if (getAbfsOutputStream().isAppendBlob()) {
+      return null;
+    }
+    if (!blobBlockManager.hasListToCommit()) {
+      return null;
+    }
+    try {
+      // Generate the xml with the list of blockId's to generate putBlockList call.
+      String blockListXml = generateBlockListXml(
+          blobBlockManager.getBlockIdList());
+      TracingContext tracingContextFlush = new TracingContext(tracingContext);
+      tracingContextFlush.setIngressHandler(BLOB_FLUSH);
+      tracingContextFlush.setPosition(String.valueOf(offset));
+      LOG.trace("Flushing data at offset {} for path {}", offset, getAbfsOutputStream().getPath());
+      op = getClient().flush(blockListXml.getBytes(StandardCharsets.UTF_8),
+          getAbfsOutputStream().getPath(),
+          isClose, getAbfsOutputStream().getCachedSasTokenString(), leaseId,
+          getETag(), getAbfsOutputStream().getContextEncryptionAdapter(), tracingContextFlush);
+      setETag(op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG));
+    } catch (AbfsRestOperationException ex) {
+      LOG.error("Error in remote flush requiring handler switch for path {}", getAbfsOutputStream().getPath(), ex);
+      if (shouldIngressHandlerBeSwitched(ex)) {
+        throw getIngressHandlerSwitchException(ex);
+      }
+      LOG.error("Error in remote flush for path {} and offset {}", getAbfsOutputStream().getPath(), offset, ex);
+      throw ex;
+    }
+    return op;
+  }
+
+  /**
+   * Method to perform a remote write operation for appending data to an append blob in Azure Blob Storage.
+   *
+   * <p>This method is intended to be implemented by subclasses to handle the specific
+   * case of appending data to an append blob. It takes in the path of the append blob,
+   * the data to be uploaded, the block of data, and additional parameters required for
+   * the append operation.</p>
+   *
+   * @param path           The path of the append blob to which data is to be appended.
+   * @param uploadData     The data to be uploaded as part of the append operation.
+   * @param block          The block of data to append.
+   * @param reqParams      The additional parameters required for the append operation.
+   * @param tracingContext The tracing context for the operation.
+   * @return An {@link AbfsRestOperation} object representing the remote write operation.
+   * @throws IOException If an I/O error occurs during the append operation.
+   */
+  protected AbfsRestOperation remoteAppendBlobWrite(String path,
+      DataBlocks.BlockUploadData uploadData,
+      AbfsBlock block,
+      AppendRequestParameters reqParams,
+      TracingContext tracingContext) throws IOException {
+    // Perform the remote append operation using the blob client.
+    AbfsRestOperation op = null;
+    try {
+      op = blobClient.appendBlock(path, reqParams, uploadData.toByteArray(), tracingContext);
+    } catch (AbfsRestOperationException ex) {
+      LOG.error("Error in remote write requiring handler switch for path {}",
+          getAbfsOutputStream().getPath(), ex);
+      if (shouldIngressHandlerBeSwitched(ex)) {
+        throw getIngressHandlerSwitchException(ex);
+      }
+      LOG.error("Error in remote write for path {} and offset {}",
+          getAbfsOutputStream().getPath(),
+          block.getOffset(), ex);
+      throw ex;
+    }
+    return op;
+  }
+
+  /**
+   * Sets the eTag of the blob.
+   *
+   * @param eTag the eTag to set.
+   */
+  void setETag(String eTag) {
+    this.eTag = eTag;
+  }
+
+  /**
+   * Gets the eTag value of the blob.
+   *
+   * @return the eTag.
+   */
+  @VisibleForTesting
+  @Override
+  public String getETag() {
+    return eTag;
+  }
+
+  /**
+   * Writes the current buffer to the service. .
+   *
+   */
+  @Override
+  protected void writeAppendBlobCurrentBufferToService() throws IOException {
+    AbfsBlock activeBlock = blobBlockManager.getActiveBlock();
+
+    // No data, return immediately.
+    if (!getAbfsOutputStream().hasActiveBlockDataToUpload()) {
+      return;
+    }
+
+    // Prepare data for upload.
+    final int bytesLength = activeBlock.dataSize();
+    DataBlocks.BlockUploadData uploadData = activeBlock.startUpload();
+
+    // Clear active block and update statistics.
+    if (blobBlockManager.hasActiveBlock()) {
+      blobBlockManager.clearActiveBlock();
+    }
+    getAbfsOutputStream().getOutputStreamStatistics().writeCurrentBuffer();
+    getAbfsOutputStream().getOutputStreamStatistics().bytesToUpload(bytesLength);
+
+    // Update the stream position.
+    final long offset = getAbfsOutputStream().getPosition();
+    getAbfsOutputStream().setPosition(offset + bytesLength);
+
+    // Perform the upload within a performance tracking context.
+    try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(
+        blobClient.getAbfsPerfTracker(),
+        "writeCurrentBufferToService", APPEND_ACTION)) {
+      LOG.trace("Writing current buffer to service at offset {} and path {}", offset, getAbfsOutputStream().getPath());
+      AppendRequestParameters reqParams = new AppendRequestParameters(
+          offset, 0, bytesLength, AppendRequestParameters.Mode.APPEND_MODE,
+          true, getAbfsOutputStream().getLeaseId(), getAbfsOutputStream().isExpectHeaderEnabled());
+
+      AbfsRestOperation op;
+      try {
+        op = remoteAppendBlobWrite(getAbfsOutputStream().getPath(), uploadData,
+            activeBlock, reqParams,
+            new TracingContext(getAbfsOutputStream().getTracingContext()));
+      } catch (InvalidIngressServiceException ex) {
+        LOG.debug("InvalidIngressServiceException caught for path: {}, switching handler and retrying remoteAppendBlobWrite.", getAbfsOutputStream().getPath());
+        getAbfsOutputStream().switchHandler();
+        op = getAbfsOutputStream().getIngressHandler()
+            .remoteAppendBlobWrite(getAbfsOutputStream().getPath(), uploadData,
+                activeBlock, reqParams,
+                new TracingContext(getAbfsOutputStream().getTracingContext()));
+      } finally {
+        // Ensure the upload data stream is closed.
+        IOUtils.closeStreams(uploadData, activeBlock);
+      }
+
+      if (op != null) {
+        // Update the SAS token and log the successful upload.
+        getAbfsOutputStream().getCachedSasToken().update(op.getSasToken());
+        getAbfsOutputStream().getOutputStreamStatistics()
+            .uploadSuccessful(bytesLength);
+
+        // Register performance information.
+        perfInfo.registerResult(op.getResult());
+        perfInfo.registerSuccess(true);
+      }
+    }
+  }
+
+  /**
+   * Gets the block manager.
+   *
+   * @return the block manager.
+   */
+  @Override
+  public AzureBlockManager getBlockManager() {
+    return blobBlockManager;
+  }
+
+  /**
+   * Gets the blob client.
+   *
+   * @return the blob client.
+   */
+  @Override
+  public AbfsBlobClient getClient() {
+    return blobClient;
+  }
+
+  @VisibleForTesting
+  public AbfsClientHandler getClientHandler() {
+    return clientHandler;
+  }
+}

+ 170 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java

@@ -0,0 +1,170 @@
+/**
+ * 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.services;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.store.DataBlocks;
+
+/**
+ * Abstract base class for managing Azure Data Lake Storage (ADLS) blocks.
+ */
+public abstract class AzureBlockManager {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsOutputStream.class);
+
+  /** Factory for blocks. */
+  private final DataBlocks.BlockFactory blockFactory;
+
+  /** Current data block. Null means none currently active. */
+  private AbfsBlock activeBlock;
+
+  /** Count of blocks uploaded. */
+  private long blockCount = 0;
+
+  /** The size of a single block. */
+  private final int blockSize;
+
+  private AbfsOutputStream abfsOutputStream;
+
+  /**
+   * Constructs an AzureBlockManager.
+   *
+   * @param abfsOutputStream the output stream associated with this block manager
+   * @param blockFactory the factory to create blocks
+   * @param blockSize the size of each block
+   */
+  protected AzureBlockManager(AbfsOutputStream abfsOutputStream,
+      DataBlocks.BlockFactory blockFactory,
+      final int blockSize) {
+    this.abfsOutputStream = abfsOutputStream;
+    this.blockFactory = blockFactory;
+    this.blockSize = blockSize;
+  }
+
+  /**
+   * Creates a new block at the given position.
+   *
+   * @param position the position in the output stream where the block should be created
+   * @return the created block
+   * @throws IOException if an I/O error occurs
+   */
+  protected final synchronized AbfsBlock createBlock(final long position)
+      throws IOException {
+    return createBlockInternal(position);
+  }
+
+  /**
+   * Internal method to create a new block at the given position.
+   *
+   * @param position the position in the output stream where the block should be created.
+   * @return the created block.
+   * @throws IOException if an I/O error occurs.
+   */
+  protected abstract AbfsBlock createBlockInternal(long position)
+      throws IOException;
+
+  /**
+   * Gets the active block.
+   *
+   * @return the active block
+   */
+  protected synchronized AbfsBlock getActiveBlock() {
+    return activeBlock;
+  }
+
+  /**
+   * Sets the active block.
+   *
+   * @param activeBlock the block to set as active
+   */
+  public synchronized void setActiveBlock(final AbfsBlock activeBlock) {
+    this.activeBlock = activeBlock;
+  }
+
+  /**
+   * Checks if there is an active block.
+   *
+   * @return true if there is an active block, false otherwise
+   */
+  protected synchronized boolean hasActiveBlock() {
+    return activeBlock != null;
+  }
+
+  /**
+   * Gets the block factory.
+   *
+   * @return the block factory
+   */
+  protected DataBlocks.BlockFactory getBlockFactory() {
+    return blockFactory;
+  }
+
+  /**
+   * Gets the count of blocks uploaded.
+   *
+   * @return the block count
+   */
+  protected long getBlockCount() {
+    return blockCount;
+  }
+
+  /**
+   * Sets the count of blocks uploaded.
+   *
+   * @param blockCount the count of blocks to set
+   */
+  public void setBlockCount(final long blockCount) {
+    this.blockCount = blockCount;
+  }
+
+  /**
+   * Gets the block size.
+   *
+   * @return the block size
+   */
+  protected int getBlockSize() {
+    return blockSize;
+  }
+
+  /**
+   * Gets the AbfsOutputStream associated with this block manager.
+   *
+   * @return the AbfsOutputStream
+   */
+  protected AbfsOutputStream getAbfsOutputStream() {
+    return abfsOutputStream;
+  }
+
+  /**
+   * Clears the active block.
+   */
+  void clearActiveBlock() {
+    synchronized (this) {
+      if (activeBlock != null) {
+        LOG.debug("Clearing active block");
+      }
+      activeBlock = null;
+    }
+  }
+}

+ 89 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java

@@ -0,0 +1,89 @@
+/**
+ * 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.services;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.store.DataBlocks;
+
+/**
+ * Manages Azure Data Lake Storage (ADLS) blocks for append operations.
+ */
+public class AzureDFSBlockManager extends AzureBlockManager {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsOutputStream.class);
+
+  /**
+   * Constructs an AzureDFSBlockManager.
+   *
+   * @param abfsOutputStream the output stream associated with this block manager
+   * @param blockFactory the factory to create blocks
+   * @param blockSize the size of each block
+   */
+  public AzureDFSBlockManager(AbfsOutputStream abfsOutputStream,
+      DataBlocks.BlockFactory blockFactory,
+      int blockSize) {
+    super(abfsOutputStream, blockFactory, blockSize);
+    LOG.trace(
+        "Created a new DFS Block Manager for AbfsOutputStream instance {} for path {}",
+        abfsOutputStream.getStreamID(), abfsOutputStream.getPath());
+  }
+
+  /**
+   * Creates a new block at the given position if none exists.
+   *
+   * @param position the position in the output stream where the block should be created
+   * @return the created block
+   * @throws IOException if an I/O error occurs
+   */
+  @Override
+  protected synchronized AbfsBlock createBlockInternal(long position)
+      throws IOException {
+    if (getActiveBlock() == null) {
+      setBlockCount(getBlockCount() + 1);
+      AbfsBlock activeBlock = new AbfsBlock(getAbfsOutputStream(), position);
+      setActiveBlock(activeBlock);
+    }
+    return getActiveBlock();
+  }
+
+  /**
+   * Gets the active block.
+   *
+   * @return the active block
+   */
+  @Override
+  protected synchronized AbfsBlock getActiveBlock() {
+    return super.getActiveBlock();
+  }
+
+  /**
+   * Checks if there is an active block.
+   *
+   * @return true if there is an active block, false otherwise
+   */
+  @Override
+  protected synchronized boolean hasActiveBlock() {
+    return super.hasActiveBlock();
+  }
+}

+ 280 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java

@@ -0,0 +1,280 @@
+/**
+ * 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.services;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.store.DataBlocks;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DFS_APPEND;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DFS_FLUSH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+
+/**
+ * The BlobFsOutputStream for Rest AbfsClient.
+ */
+public class AzureDFSIngressHandler extends AzureIngressHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsOutputStream.class);
+
+  private AzureDFSBlockManager dfsBlockManager;
+
+  private final AbfsDfsClient dfsClient;
+
+  private String eTag;
+
+  /**
+   * Constructs an AzureDFSIngressHandler.
+   *
+   * @param abfsOutputStream the AbfsOutputStream instance.
+   * @param clientHandler the AbfsClientHandler instance.
+   */
+  public AzureDFSIngressHandler(AbfsOutputStream abfsOutputStream,
+      AbfsClientHandler clientHandler) {
+    super(abfsOutputStream);
+    this.dfsClient = clientHandler.getDfsClient();
+  }
+
+  /**
+   * Constructs an AzureDFSIngressHandler with specified parameters.
+   *
+   * @param abfsOutputStream the AbfsOutputStream.
+   * @param blockFactory the block factory.
+   * @param bufferSize the buffer size.
+   * @param eTag the eTag.
+   * @param clientHandler the client handler.
+   */
+  public AzureDFSIngressHandler(AbfsOutputStream abfsOutputStream,
+      DataBlocks.BlockFactory blockFactory,
+      int bufferSize, String eTag, AbfsClientHandler clientHandler) {
+    this(abfsOutputStream, clientHandler);
+    this.eTag = eTag;
+    this.dfsBlockManager = new AzureDFSBlockManager(abfsOutputStream,
+        blockFactory, bufferSize);
+    LOG.trace(
+        "Created a new DFSIngress Handler for AbfsOutputStream instance {} for path {}",
+        abfsOutputStream.getStreamID(), abfsOutputStream.getPath());
+  }
+
+  /**
+   * Buffers data into the specified block.
+   *
+   * @param block the block to buffer data into.
+   * @param data  the data to be buffered.
+   * @param off   the start offset in the data.
+   * @param length the number of bytes to buffer.
+   * @return the number of bytes buffered.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  public int bufferData(AbfsBlock block,
+      final byte[] data,
+      final int off,
+      final int length)
+      throws IOException {
+    LOG.trace("Buffering data of length {} to block at offset {}", length, off);
+    return block.write(data, off, length);
+  }
+
+  /**
+   * Performs a remote write operation.
+   *
+   * @param blockToUpload the block to upload.
+   * @param uploadData    the data to upload.
+   * @param reqParams     the request parameters.
+   * @param tracingContext the tracing context.
+   * @return the resulting AbfsRestOperation.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  protected AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
+      DataBlocks.BlockUploadData uploadData,
+      AppendRequestParameters reqParams,
+      TracingContext tracingContext) throws IOException {
+    TracingContext tracingContextAppend = new TracingContext(tracingContext);
+    String threadIdStr = String.valueOf(Thread.currentThread().getId());
+    if (tracingContextAppend.getIngressHandler().equals(EMPTY_STRING)) {
+      tracingContextAppend.setIngressHandler(DFS_APPEND + " T " + threadIdStr);
+      tracingContextAppend.setPosition(
+          String.valueOf(blockToUpload.getOffset()));
+    }
+    LOG.trace("Starting remote write for block with offset {} and path {}",
+        blockToUpload.getOffset(),
+        getAbfsOutputStream().getPath());
+    return getClient().append(getAbfsOutputStream().getPath(),
+        uploadData.toByteArray(), reqParams,
+        getAbfsOutputStream().getCachedSasTokenString(),
+        getAbfsOutputStream().getContextEncryptionAdapter(),
+        tracingContextAppend);
+  }
+
+  /**
+   * Method to perform a remote write operation for appending data to an append blob in Azure Blob Storage.
+   *
+   * <p>This method is intended to be implemented by subclasses to handle the specific
+   * case of appending data to an append blob. It takes in the path of the append blob,
+   * the data to be uploaded, the block of data, and additional parameters required for
+   * the append operation.</p>
+   *
+   * @param path           The path of the append blob to which data is to be appended.
+   * @param uploadData     The data to be uploaded as part of the append operation.
+   * @param block          The block of data to append.
+   * @param reqParams      The additional parameters required for the append operation.
+   * @param tracingContext The tracing context for the operation.
+   * @return An {@link AbfsRestOperation} object representing the remote write operation.
+   * @throws IOException If an I/O error occurs during the append operation.
+   */
+  @Override
+  protected AbfsRestOperation remoteAppendBlobWrite(String path, DataBlocks.BlockUploadData uploadData,
+      AbfsBlock block, AppendRequestParameters reqParams,
+      TracingContext tracingContext) throws IOException {
+    return remoteWrite(block, uploadData, reqParams, tracingContext);
+  }
+
+  /**
+   * Flushes data to the remote store.
+   *
+   * @param offset               the offset to flush.
+   * @param retainUncommitedData whether to retain uncommitted data.
+   * @param isClose              whether this is a close operation.
+   * @param leaseId              the lease ID.
+   * @param tracingContext       the tracing context.
+   * @return the resulting AbfsRestOperation.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  protected synchronized AbfsRestOperation remoteFlush(final long offset,
+      final boolean retainUncommitedData,
+      final boolean isClose,
+      final String leaseId,
+      TracingContext tracingContext)
+      throws IOException {
+    TracingContext tracingContextFlush = new TracingContext(tracingContext);
+    if (tracingContextFlush.getIngressHandler().equals(EMPTY_STRING)) {
+      tracingContextFlush.setIngressHandler(DFS_FLUSH);
+      tracingContextFlush.setPosition(String.valueOf(offset));
+    }
+    LOG.trace("Flushing data at offset {} and path {}", offset, getAbfsOutputStream().getPath());
+    return getClient()
+        .flush(getAbfsOutputStream().getPath(), offset, retainUncommitedData,
+            isClose,
+            getAbfsOutputStream().getCachedSasTokenString(), leaseId,
+            getAbfsOutputStream().getContextEncryptionAdapter(),
+            tracingContextFlush);
+  }
+
+  /**
+   * Appending the current active data block to the service. Clearing the active
+   * data block and releasing all buffered data.
+   *
+   * @throws IOException if there is any failure while starting an upload for
+   *                     the data block or while closing the BlockUploadData.
+   */
+  @Override
+  protected void writeAppendBlobCurrentBufferToService() throws IOException {
+    AbfsBlock activeBlock = dfsBlockManager.getActiveBlock();
+
+    // No data, return immediately.
+    if (!getAbfsOutputStream().hasActiveBlockDataToUpload()) {
+      return;
+    }
+
+    // Prepare data for upload.
+    final int bytesLength = activeBlock.dataSize();
+    DataBlocks.BlockUploadData uploadData = activeBlock.startUpload();
+
+    // Clear active block and update statistics.
+    if (dfsBlockManager.hasActiveBlock()) {
+      dfsBlockManager.clearActiveBlock();
+    }
+    getAbfsOutputStream().getOutputStreamStatistics().writeCurrentBuffer();
+    getAbfsOutputStream().getOutputStreamStatistics().bytesToUpload(bytesLength);
+
+    // Update the stream position.
+    final long offset = getAbfsOutputStream().getPosition();
+    getAbfsOutputStream().setPosition(offset + bytesLength);
+
+    // Perform the upload within a performance tracking context.
+    try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(
+        dfsClient.getAbfsPerfTracker(),
+        "writeCurrentBufferToService", APPEND_ACTION)) {
+      LOG.trace("Writing current buffer to service at offset {} and path {}", offset, getAbfsOutputStream().getPath());
+      AppendRequestParameters reqParams = new AppendRequestParameters(
+          offset, 0, bytesLength, AppendRequestParameters.Mode.APPEND_MODE,
+          true, getAbfsOutputStream().getLeaseId(), getAbfsOutputStream().isExpectHeaderEnabled());
+
+      // Perform the remote write operation.
+      AbfsRestOperation op = remoteWrite(activeBlock, uploadData, reqParams,
+          new TracingContext(getAbfsOutputStream().getTracingContext()));
+
+      // Update the SAS token and log the successful upload.
+      getAbfsOutputStream().getCachedSasToken().update(op.getSasToken());
+      getAbfsOutputStream().getOutputStreamStatistics().uploadSuccessful(bytesLength);
+
+      // Register performance information.
+      perfInfo.registerResult(op.getResult());
+      perfInfo.registerSuccess(true);
+    } catch (Exception ex) {
+      LOG.error("Failed to upload current buffer of length {} and path {}", bytesLength, getAbfsOutputStream().getPath(), ex);
+      getAbfsOutputStream().getOutputStreamStatistics().uploadFailed(bytesLength);
+      getAbfsOutputStream().failureWhileSubmit(ex);
+    } finally {
+      // Ensure the upload data stream is closed.
+      IOUtils.closeStreams(uploadData, activeBlock);
+    }
+  }
+
+  /**
+   * Gets the block manager.
+   *
+   * @return the block manager.
+   */
+  @Override
+  public AzureBlockManager getBlockManager() {
+    return dfsBlockManager;
+  }
+
+  /**
+   * Gets the dfs client.
+   *
+   * @return the dfs client.
+   */
+  @Override
+  public AbfsDfsClient getClient() {
+    return dfsClient;
+  }
+
+  /**
+   * Gets the eTag value of the blob.
+   *
+   * @return the eTag.
+   */
+  @Override
+  public String getETag() {
+    return eTag;
+  }
+}

+ 265 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDfsToBlobIngressFallbackHandler.java

@@ -0,0 +1,265 @@
+/**
+ * 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.services;
+
+import java.io.IOException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+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.InvalidIngressServiceException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.store.DataBlocks;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FALLBACK_APPEND;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FALLBACK_FLUSH;
+
+/**
+ * Handles the fallback mechanism for Azure Blob Ingress operations.
+ */
+public class AzureDfsToBlobIngressFallbackHandler extends AzureDFSIngressHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsOutputStream.class);
+
+  private final AzureBlobBlockManager blobBlockManager;
+
+  private final String eTag;
+
+  private final Lock lock = new ReentrantLock();
+
+  /**
+   * Constructs an AzureDfsToBlobIngressFallbackHandler.
+   *
+   * @param abfsOutputStream the AbfsOutputStream.
+   * @param blockFactory the block factory.
+   * @param bufferSize the buffer size.
+   * @param eTag the eTag.
+   * @param clientHandler the client handler.
+   * @throws AzureBlobFileSystemException if an error occurs.
+   */
+  public AzureDfsToBlobIngressFallbackHandler(AbfsOutputStream abfsOutputStream,
+      DataBlocks.BlockFactory blockFactory,
+      int bufferSize, String eTag, AbfsClientHandler clientHandler) throws AzureBlobFileSystemException {
+    super(abfsOutputStream, clientHandler);
+    this.eTag = eTag;
+    this.blobBlockManager = new AzureBlobBlockManager(abfsOutputStream,
+        blockFactory, bufferSize);
+    LOG.trace(
+        "Created a new BlobFallbackIngress Handler for AbfsOutputStream instance {} for path {}",
+        abfsOutputStream.getStreamID(), abfsOutputStream.getPath());
+  }
+
+  /**
+   * Buffers data into the specified block.
+   *
+   * @param block the block to buffer data into.
+   * @param data  the data to be buffered.
+   * @param off   the start offset in the data.
+   * @param length the number of bytes to buffer.
+   * @return the number of bytes buffered.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  public int bufferData(AbfsBlock block,
+      final byte[] data,
+      final int off,
+      final int length) throws IOException {
+    LOG.trace("Buffering data of length {} to block at offset {}", length, off);
+    return super.bufferData(block, data, off, length);
+  }
+
+  /**
+   * Performs a remote write operation.
+   *
+   * @param blockToUpload the block to upload.
+   * @param uploadData    the data to upload.
+   * @param reqParams     the request parameters.
+   * @param tracingContext the tracing context.
+   * @return the resulting AbfsRestOperation.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  protected AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
+      DataBlocks.BlockUploadData uploadData,
+      AppendRequestParameters reqParams,
+      TracingContext tracingContext) throws IOException {
+    AbfsRestOperation op;
+    TracingContext tracingContextAppend = new TracingContext(tracingContext);
+    String threadIdStr = String.valueOf(Thread.currentThread().getId());
+    tracingContextAppend.setIngressHandler(FALLBACK_APPEND + " T " + threadIdStr);
+    tracingContextAppend.setPosition(String.valueOf(blockToUpload.getOffset()));
+    try {
+      op = super.remoteWrite(blockToUpload, uploadData, reqParams,
+          tracingContextAppend);
+      blobBlockManager.updateEntry(blockToUpload);
+    } catch (AbfsRestOperationException ex) {
+      if (shouldIngressHandlerBeSwitched(ex)) {
+        LOG.error("Error in remote write requiring handler switch for path {}", getAbfsOutputStream().getPath(), ex);
+        throw getIngressHandlerSwitchException(ex);
+      }
+      LOG.error("Error in remote write for path {} and offset {}", getAbfsOutputStream().getPath(),
+          blockToUpload.getOffset(), ex);
+      throw ex;
+    }
+    return op;
+  }
+
+  /**
+   * Flushes data to the remote store.
+   *
+   * @param offset               the offset to flush.
+   * @param retainUncommitedData whether to retain uncommitted data.
+   * @param isClose              whether this is a close operation.
+   * @param leaseId              the lease ID.
+   * @param tracingContext       the tracing context.
+   * @return the resulting AbfsRestOperation.
+   * @throws IOException if an I/O error occurs.
+   */
+  @Override
+  protected synchronized AbfsRestOperation remoteFlush(final long offset,
+      final boolean retainUncommitedData,
+      final boolean isClose,
+      final String leaseId,
+      TracingContext tracingContext) throws IOException {
+    AbfsRestOperation op;
+    if (!blobBlockManager.hasListToCommit()) {
+      return null;
+    }
+    try {
+      TracingContext tracingContextFlush = new TracingContext(tracingContext);
+      tracingContextFlush.setIngressHandler(FALLBACK_FLUSH);
+      tracingContextFlush.setPosition(String.valueOf(offset));
+      op = super.remoteFlush(offset, retainUncommitedData, isClose, leaseId,
+          tracingContextFlush);
+    } catch (AbfsRestOperationException ex) {
+      if (shouldIngressHandlerBeSwitched(ex)) {
+        LOG.error("Error in remote flush requiring handler switch for path {}", getAbfsOutputStream().getPath(), ex);
+        throw getIngressHandlerSwitchException(ex);
+      }
+      LOG.error("Error in remote flush for path {} and offset {}", getAbfsOutputStream().getPath(), offset, ex);
+      throw ex;
+    }
+    return op;
+  }
+
+  /**
+   * Gets the block manager.
+   *
+   * @return the block manager.
+   */
+  @Override
+  public AzureBlockManager getBlockManager() {
+    return blobBlockManager;
+  }
+
+  /**
+   * Gets the eTag value of the blob.
+   *
+   * @return the eTag.
+   */
+  @VisibleForTesting
+  public String getETag() {
+    lock.lock();
+    try {
+      return eTag;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Appending the current active data block to the service. Clearing the active
+   * data block and releasing all buffered data.
+   *
+   * @throws IOException if there is any failure while starting an upload for
+   *                     the data block or while closing the BlockUploadData.
+   */
+  @Override
+  protected void writeAppendBlobCurrentBufferToService() throws IOException {
+    AbfsBlock activeBlock = blobBlockManager.getActiveBlock();
+
+    // No data, return immediately.
+    if (!getAbfsOutputStream().hasActiveBlockDataToUpload()) {
+      return;
+    }
+
+    // Prepare data for upload.
+    final int bytesLength = activeBlock.dataSize();
+    DataBlocks.BlockUploadData uploadData = activeBlock.startUpload();
+
+    // Clear active block and update statistics.
+    if (blobBlockManager.hasActiveBlock()) {
+      blobBlockManager.clearActiveBlock();
+    }
+    getAbfsOutputStream().getOutputStreamStatistics().writeCurrentBuffer();
+    getAbfsOutputStream().getOutputStreamStatistics().bytesToUpload(bytesLength);
+
+    // Update the stream position.
+    final long offset = getAbfsOutputStream().getPosition();
+    getAbfsOutputStream().setPosition(offset + bytesLength);
+
+    // Perform the upload within a performance tracking context.
+    try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(
+        getClient().getAbfsPerfTracker(),
+        "writeCurrentBufferToService", APPEND_ACTION)) {
+      LOG.trace("Writing current buffer to service at offset {} and path {}", offset, getAbfsOutputStream().getPath());
+      AppendRequestParameters reqParams = new AppendRequestParameters(
+          offset, 0, bytesLength, AppendRequestParameters.Mode.APPEND_MODE,
+          true, getAbfsOutputStream().getLeaseId(), getAbfsOutputStream().isExpectHeaderEnabled());
+
+      // Perform the remote write operation.
+      AbfsRestOperation op;
+      try {
+        op = remoteAppendBlobWrite(getAbfsOutputStream().getPath(), uploadData,
+            activeBlock, reqParams,
+            new TracingContext(getAbfsOutputStream().getTracingContext()));
+      } catch (InvalidIngressServiceException ex) {
+        LOG.debug("InvalidIngressServiceException caught for path: {}, switching handler and retrying remoteAppendBlobWrite.", getAbfsOutputStream().getPath());
+        getAbfsOutputStream().switchHandler();
+        op = getAbfsOutputStream().getIngressHandler()
+            .remoteAppendBlobWrite(getAbfsOutputStream().getPath(), uploadData,
+                activeBlock, reqParams,
+                new TracingContext(getAbfsOutputStream().getTracingContext()));
+      } finally {
+        // Ensure the upload data stream is closed.
+        IOUtils.closeStreams(uploadData, activeBlock);
+      }
+
+      if (op != null) {
+        // Update the SAS token and log the successful upload.
+        getAbfsOutputStream().getCachedSasToken().update(op.getSasToken());
+        getAbfsOutputStream().getOutputStreamStatistics()
+            .uploadSuccessful(bytesLength);
+
+        // Register performance information.
+        perfInfo.registerResult(op.getResult());
+        perfInfo.registerSuccess(true);
+      }
+    }
+  }
+}

+ 209 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java

@@ -0,0 +1,209 @@
+/**
+ * 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.services;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidIngressServiceException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.store.DataBlocks;
+
+import static java.net.HttpURLConnection.HTTP_CONFLICT;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.BLOB_OPERATION_NOT_SUPPORTED;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.INVALID_APPEND_OPERATION;
+
+/**
+ * Abstract base class for handling ingress operations for Azure Data Lake Storage (ADLS).
+ */
+public abstract class AzureIngressHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsOutputStream.class);
+
+  /** The output stream associated with this handler */
+  private AbfsOutputStream abfsOutputStream;
+
+  /**
+   * Constructs an AzureIngressHandler.
+   *
+   * @param abfsOutputStream the output stream associated with this handler
+   */
+  protected AzureIngressHandler(AbfsOutputStream abfsOutputStream) {
+    this.abfsOutputStream = abfsOutputStream;
+  }
+
+  /**
+   * Gets the AbfsOutputStream associated with this handler.
+   *
+   * @return the AbfsOutputStream
+   */
+  public AbfsOutputStream getAbfsOutputStream() {
+    return abfsOutputStream;
+  }
+
+  /**
+   * Sets the AbfsOutputStream associated with this handler.
+   *
+   * @param abfsOutputStream the AbfsOutputStream to set
+   */
+  public void setAbfsOutputStream(final AbfsOutputStream abfsOutputStream) {
+    this.abfsOutputStream = abfsOutputStream;
+  }
+
+  /**
+   * Gets the eTag value of the blob.
+   *
+   * @return the eTag.
+   */
+  public abstract String getETag();
+
+  /**
+   * Buffers data into the specified block.
+   *
+   * @param block the block to buffer data into
+   * @param data the data to buffer
+   * @param off the start offset in the data
+   * @param length the number of bytes to buffer
+   * @return the number of bytes buffered
+   * @throws IOException if an I/O error occurs
+   */
+  protected abstract int bufferData(AbfsBlock block,
+      byte[] data, int off, int length) throws IOException;
+
+  /**
+   * Performs a remote write operation to upload a block.
+   *
+   * @param blockToUpload the block to upload
+   * @param uploadData the data to upload
+   * @param reqParams the request parameters for the append operation
+   * @param tracingContext the tracing context
+   * @return the result of the REST operation
+   * @throws IOException if an I/O error occurs
+   */
+  protected abstract AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
+      DataBlocks.BlockUploadData uploadData,
+      AppendRequestParameters reqParams,
+      TracingContext tracingContext) throws IOException;
+
+  /**
+   * Performs a remote flush operation.
+   *
+   * @param offset the offset to flush to
+   * @param retainUncommittedData whether to retain uncommitted data
+   * @param isClose whether this is a close operation
+   * @param leaseId the lease ID
+   * @param tracingContext the tracing context
+   * @return the result of the REST operation
+   * @throws IOException if an I/O error occurs
+   */
+  protected abstract AbfsRestOperation remoteFlush(long offset,
+      boolean retainUncommittedData,
+      boolean isClose,
+      String leaseId,
+      TracingContext tracingContext) throws IOException;
+
+  /**
+   * Writes the current buffer to the service for an append blob.
+   *
+   * @throws IOException if an I/O error occurs
+   */
+  protected abstract void writeAppendBlobCurrentBufferToService()
+      throws IOException;
+
+  /**
+   * Abstract method to perform a remote write operation for appending data to an append blob in Azure Blob Storage.
+   *
+   * <p>This method is intended to be implemented by subclasses to handle the specific
+   * case of appending data to an append blob. It takes in the path of the append blob,
+   * the data to be uploaded, the block of data, and additional parameters required for
+   * the append operation.</p>
+   *
+   * @param path           The path of the append blob to which data is to be appended.
+   * @param uploadData     The data to be uploaded as part of the append operation.
+   * @param block          The block of data to append.
+   * @param reqParams      The additional parameters required for the append operation.
+   * @param tracingContext The tracing context for the operation.
+   * @return An {@link AbfsRestOperation} object representing the remote write operation.
+   * @throws IOException If an I/O error occurs during the append operation.
+   */
+  protected abstract AbfsRestOperation remoteAppendBlobWrite(String path,
+      DataBlocks.BlockUploadData uploadData,
+      AbfsBlock block,
+      AppendRequestParameters reqParams,
+      TracingContext tracingContext) throws IOException;
+
+  /**
+   * Determines if the ingress handler should be switched based on the given exception.
+   *
+   * @param ex the exception that occurred
+   * @return true if the ingress handler should be switched, false otherwise
+   */
+  protected boolean shouldIngressHandlerBeSwitched(AbfsRestOperationException ex) {
+    if (ex == null || ex.getErrorCode() == null) {
+      return false;
+    }
+    String errorCode = ex.getErrorCode().getErrorCode();
+    if (errorCode != null) {
+      return ex.getStatusCode() == HTTP_CONFLICT
+          && (Objects.equals(errorCode, AzureServiceErrorCode.BLOB_OPERATION_NOT_SUPPORTED.getErrorCode())
+              || Objects.equals(errorCode, AzureServiceErrorCode.INVALID_APPEND_OPERATION.getErrorCode()));
+    }
+    return false;
+  }
+
+  /**
+   * Constructs an InvalidIngressServiceException that includes the current handler class name in the exception message.
+   *
+   * @param e the original AbfsRestOperationException that triggered this exception.
+   * @return an InvalidIngressServiceException with the status code, error code, original message, and handler class name.
+   */
+  protected InvalidIngressServiceException getIngressHandlerSwitchException(
+      AbfsRestOperationException e) {
+    if (e.getMessage().contains(BLOB_OPERATION_NOT_SUPPORTED)) {
+      return new InvalidIngressServiceException(e.getStatusCode(),
+          AzureServiceErrorCode.BLOB_OPERATION_NOT_SUPPORTED.getErrorCode(),
+          BLOB_OPERATION_NOT_SUPPORTED + " " + getClass().getName(), e);
+    } else {
+      return new InvalidIngressServiceException(e.getStatusCode(),
+          AzureServiceErrorCode.INVALID_APPEND_OPERATION.getErrorCode(),
+          INVALID_APPEND_OPERATION + " " + getClass().getName(), e);
+    }
+  }
+
+  /**
+   * Gets the block manager associated with this handler.
+   *
+   * @return the block manager
+   */
+  protected abstract AzureBlockManager getBlockManager();
+
+  /**
+   * Gets the client associated with this handler.
+   *
+   * @return the block manager
+   */
+  public abstract AbfsClient getClient();
+}

+ 86 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlockEntry.java

@@ -0,0 +1,86 @@
+/**
+ * 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.services;
+
+/**
+ * Represents an entry for a block, containing a block ID, its position in the stream, and its status.
+ */
+public class BlockEntry {
+  private final String blockId;
+  private AbfsBlockStatus status;
+  private long position;
+
+  /**
+   * Constructs a new {@code BlockEntry}.
+   *
+   * @param blockId The unique identifier for the block.
+   * @param position The position of the block in the stream.
+   * @param status The current status of the block.
+   */
+  public BlockEntry(String blockId, long position, AbfsBlockStatus status) {
+    this.blockId = blockId;
+    this.position = position;
+    this.status = status;
+  }
+
+  /**
+   * Returns the block ID of this {@code BlockEntry}.
+   *
+   * @return The block ID.
+   */
+  public String getBlockId() {
+    return blockId;
+  }
+
+  /**
+   * Returns the position of the block in the stream.
+   *
+   * @return The block's position.
+   */
+  public long getPosition() {
+    return position;
+  }
+
+  /**
+   * Returns the current status of the block.
+   *
+   * @return The block's status.
+   */
+  public AbfsBlockStatus getStatus() {
+    return status;
+  }
+
+  /**
+   * Sets the status of the block.
+   *
+   * @param status The new status to be set.
+   */
+  public void setStatus(AbfsBlockStatus status) {
+    this.status = status;
+  }
+
+  /**
+   * Sets the position of the block in the stream.
+   *
+   * @param position The new position to be set.
+   */
+  public void setPosition(final long position) {
+    this.position = position;
+  }
+}

+ 2 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java

@@ -30,4 +30,6 @@ public interface Listener {
   void updatePrimaryRequestID(String primaryRequestID);
   void updatePrimaryRequestID(String primaryRequestID);
   Listener getClone();
   Listener getClone();
   void setOperation(FSOperationType operation);
   void setOperation(FSOperationType operation);
+  void updateIngressHandler(String ingressHandler);
+  void updatePosition(String position);
 }
 }

+ 51 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java

@@ -63,6 +63,8 @@ public class TracingContext {
   private Listener listener = null;  // null except when testing
   private Listener listener = null;  // null except when testing
   //final concatenated ID list set into x-ms-client-request-id header
   //final concatenated ID list set into x-ms-client-request-id header
   private String header = EMPTY_STRING;
   private String header = EMPTY_STRING;
+  private String ingressHandler = EMPTY_STRING;
+  private String position = EMPTY_STRING;
   private String metricResults = EMPTY_STRING;
   private String metricResults = EMPTY_STRING;
   private String metricHeader = EMPTY_STRING;
   private String metricHeader = EMPTY_STRING;
 
 
@@ -131,6 +133,8 @@ public class TracingContext {
     this.retryCount = 0;
     this.retryCount = 0;
     this.primaryRequestId = originalTracingContext.primaryRequestId;
     this.primaryRequestId = originalTracingContext.primaryRequestId;
     this.format = originalTracingContext.format;
     this.format = originalTracingContext.format;
+    this.position = originalTracingContext.getPosition();
+    this.ingressHandler = originalTracingContext.getIngressHandler();
     if (originalTracingContext.listener != null) {
     if (originalTracingContext.listener != null) {
       this.listener = originalTracingContext.listener.getClone();
       this.listener = originalTracingContext.listener.getClone();
     }
     }
@@ -193,6 +197,12 @@ public class TracingContext {
               + ":" + opType + ":" + retryCount;
               + ":" + opType + ":" + retryCount;
       header = addFailureReasons(header, previousFailure, retryPolicyAbbreviation);
       header = addFailureReasons(header, previousFailure, retryPolicyAbbreviation);
       header += (":" + httpOperation.getTracingContextSuffix());
       header += (":" + httpOperation.getTracingContextSuffix());
+      if (!(ingressHandler.equals(EMPTY_STRING))) {
+        header += ":" + ingressHandler;
+      }
+      if (!(position.equals(EMPTY_STRING))) {
+        header += ":" + position;
+      }
       metricHeader += !(metricResults.trim().isEmpty()) ? metricResults  : "";
       metricHeader += !(metricResults.trim().isEmpty()) ? metricResults  : "";
       break;
       break;
     case TWO_ID_FORMAT:
     case TWO_ID_FORMAT:
@@ -257,4 +267,45 @@ public class TracingContext {
     return header;
     return header;
   }
   }
 
 
+  /**
+   * Gets the ingress handler.
+   *
+   * @return the ingress handler as a String.
+   */
+  public String getIngressHandler() {
+    return ingressHandler;
+  }
+
+  /**
+   * Gets the position.
+   *
+   * @return the position as a String.
+   */
+  public String getPosition() {
+    return position;
+  }
+
+  /**
+   * Sets the ingress handler.
+   *
+   * @param ingressHandler the ingress handler to set, must not be null.
+   */
+  public void setIngressHandler(final String ingressHandler) {
+    this.ingressHandler = ingressHandler;
+    if (listener != null) {
+      listener.updateIngressHandler(ingressHandler);
+    }
+  }
+
+  /**
+   * Sets the position.
+   *
+   * @param position the position to set, must not be null.
+   */
+  public void setPosition(final String position) {
+    this.position = position;
+    if (listener != null) {
+      listener.updatePosition(position);
+    }
+  }
 }
 }

+ 6 - 1
hadoop-tools/hadoop-azure/src/site/markdown/blobEndpoint.md

@@ -35,6 +35,7 @@ The API includes the operations listed in the following table.
 | [Delete Blob](#delete-blob)                           | Path          | Deletes the blob at specified path.                                                         |
 | [Delete Blob](#delete-blob)                           | Path          | Deletes the blob at specified path.                                                         |
 | [Get Block List](#get-block-list)                     | Path          | Retrieves the list of blocks that have been uploaded as part of a block blob.               |
 | [Get Block List](#get-block-list)                     | Path          | Retrieves the list of blocks that have been uploaded as part of a block blob.               |
 | [Copy Blob](#copy-blob)                               | Path          | Copies a blob to a destination within the storage account.                                  |
 | [Copy Blob](#copy-blob)                               | Path          | Copies a blob to a destination within the storage account.                                  |
+| [Append Block](#append-block)                         | Path          | Commits a new block of data to the end of an existing append blob.                          |
 
 
 ## Create Container
 ## Create Container
 The Create Container operation creates a new container under the specified account. If the container with the same name
 The Create Container operation creates a new container under the specified account. If the container with the same name
@@ -99,4 +100,8 @@ Rest API Documentation: [Get Block List](https://docs.microsoft.com/en-us/rest/a
 
 
 ## Copy Blob
 ## Copy Blob
 The Copy Blob operation copies a blob to a destination within the storage account.
 The Copy Blob operation copies a blob to a destination within the storage account.
-Rest API Documentation: [Copy Blob](https://docs.microsoft.com/en-us/rest/api/storageservices/copy-blob)
+Rest API Documentation: [Copy Blob](https://docs.microsoft.com/en-us/rest/api/storageservices/copy-blob)
+
+## Append Block
+The Append Block operation commits a new block of data to the end of an existing append blob.
+Rest API Documentaion: [Append Block](https://learn.microsoft.com/en-us/rest/api/storageservices/append-block)

+ 65 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 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.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
@@ -50,6 +51,7 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.utils.AzcopyToolHelper;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
 import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
 import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
@@ -58,7 +60,11 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 
 
 import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX;
 import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX;
+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.ConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_BLOB_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.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -103,7 +109,10 @@ public abstract class AbstractAbfsIntegrationTest extends
     assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT_NAME,
     assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT_NAME,
             accountName != null && !accountName.isEmpty());
             accountName != null && !accountName.isEmpty());
 
 
-    abfsConfig = new AbfsConfiguration(rawConfig, accountName);
+    final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
+    URI defaultUri = null;
+
+    abfsConfig = new AbfsConfiguration(rawConfig, accountName, identifyAbfsServiceTypeFromUrl(abfsUrl));
 
 
     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();
     assumeValidAuthConfigsPresent();
@@ -111,9 +120,6 @@ public abstract class AbstractAbfsIntegrationTest extends
     abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME
     abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME
             : FileSystemUriSchemes.ABFS_SECURE_SCHEME;
             : FileSystemUriSchemes.ABFS_SECURE_SCHEME;
 
 
-    final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
-    URI defaultUri = null;
-
     try {
     try {
       defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
       defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
     } catch (Exception ex) {
     } catch (Exception ex) {
@@ -436,6 +442,13 @@ public abstract class AbstractAbfsIntegrationTest extends
         FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX, isAlwaysHttpsUsed);
         FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX, isAlwaysHttpsUsed);
   }
   }
 
 
+  private AbfsServiceType identifyAbfsServiceTypeFromUrl(String defaultUri) {
+    if (defaultUri.contains(ABFS_BLOB_DOMAIN_NAME)) {
+      return AbfsServiceType.BLOB;
+    }
+    return AbfsServiceType.DFS;
+  }
+
   private static String convertTestUrls(
   private static String convertTestUrls(
       final String url,
       final String url,
       final String fromNonSecureScheme,
       final String fromNonSecureScheme,
@@ -575,4 +588,52 @@ public abstract class AbstractAbfsIntegrationTest extends
   protected boolean isAppendBlobEnabled() {
   protected boolean isAppendBlobEnabled() {
     return getRawConfiguration().getBoolean(FS_AZURE_TEST_APPENDBLOB_ENABLED, false);
     return getRawConfiguration().getBoolean(FS_AZURE_TEST_APPENDBLOB_ENABLED, false);
   }
   }
+
+  protected AbfsServiceType getAbfsServiceType() {
+    return abfsConfig.getFsConfiguredServiceType();
+  }
+
+  /**
+   * Returns the service type to be used for Ingress Operations irrespective of account type.
+   * Default value is the same as the service type configured for the file system.
+   * @return the service type.
+   */
+  public AbfsServiceType getIngressServiceType() {
+    return abfsConfig.getIngressServiceType();
+  }
+
+  /**
+   * Create directory with implicit parent directory.
+   * @param path path to create. Can be relative or absolute.
+   */
+  protected void createAzCopyFolder(Path path) throws Exception {
+    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.BLOB);
+    assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_FIXED_SAS_TOKEN);
+    String sasToken = getRawConfiguration().get(FS_AZURE_TEST_FIXED_SAS_TOKEN);
+    AzcopyToolHelper azcopyHelper = AzcopyToolHelper.getInstance(sasToken);
+    azcopyHelper.createFolderUsingAzcopy(getAzcopyAbsolutePath(path));
+  }
+
+  /**
+   * Create file with implicit parent directory.
+   * @param path path to create. Can be relative or absolute.
+   */
+  protected void createAzCopyFile(Path path) throws Exception {
+    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.BLOB);
+    assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_FIXED_SAS_TOKEN);
+    String sasToken = getRawConfiguration().get(FS_AZURE_TEST_FIXED_SAS_TOKEN);
+    AzcopyToolHelper azcopyHelper = AzcopyToolHelper.getInstance(sasToken);
+    azcopyHelper.createFileUsingAzcopy(getAzcopyAbsolutePath(path));
+  }
+
+  private String getAzcopyAbsolutePath(Path path) throws IOException {
+    String pathFromContainerRoot = getFileSystem().makeQualified(path).toUri().getPath();
+    return HTTPS_SCHEME + COLON + FORWARD_SLASH + FORWARD_SLASH
+        + accountName + FORWARD_SLASH + fileSystemName + pathFromContainerRoot;
+  }
+
+  protected void assumeBlobServiceType() {
+    Assume.assumeTrue("Blob service type is required for this test",
+        getAbfsServiceType() == AbfsServiceType.BLOB);
+  }
 }
 }

+ 33 - 5
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java

@@ -31,8 +31,10 @@ import java.util.Random;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobAppendRequestParameters;
 import org.apache.hadoop.fs.azurebfs.security.EncodingHelper;
 import org.apache.hadoop.fs.azurebfs.security.EncodingHelper;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientUtils;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientUtils;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
@@ -61,7 +63,10 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Lists;
 
 
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_LIST_END_TAG;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_LIST_START_TAG;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CPK_IN_NON_HNS_ACCOUNT_ERROR_MESSAGE;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CPK_IN_NON_HNS_ACCOUNT_ERROR_MESSAGE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_VERSION;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA;
@@ -87,6 +92,7 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
 
 
   private final byte[] cpk = new byte[ENCRYPTION_KEY_LEN];
   private final byte[] cpk = new byte[ENCRYPTION_KEY_LEN];
   private final String cpkSHAEncoded;
   private final String cpkSHAEncoded;
+  private static final String BLOCK_ID = "MF8tNDE1MjkzOTE4AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA";
 
 
   private List<AzureBlobFileSystem> fileSystemsOpenedInTest = new ArrayList<>();
   private List<AzureBlobFileSystem> fileSystemsOpenedInTest = new ArrayList<>();
 
 
@@ -185,6 +191,12 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
     }
     }
   }
   }
 
 
+  protected static String generateBlockListXml() {
+    return XML_VERSION
+        + BLOCK_LIST_START_TAG
+        + BLOCK_LIST_END_TAG;
+  }
+
   private void validateCpkResponseHeadersForCombination(final AzureBlobFileSystem fs)
   private void validateCpkResponseHeadersForCombination(final AzureBlobFileSystem fs)
       throws Exception {
       throws Exception {
     Path testPath = path("/testFile");
     Path testPath = path("/testFile");
@@ -244,7 +256,9 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
       Path testPath, EncryptionContextProvider ecp)
       Path testPath, EncryptionContextProvider ecp)
       throws Exception {
       throws Exception {
     AbfsClient client = fs.getAbfsClient();
     AbfsClient client = fs.getAbfsClient();
+    AbfsClient ingressClient = fs.getAbfsStore().getClientHandler().getIngressClient();
     AbfsClientUtils.setEncryptionContextProvider(client, ecp);
     AbfsClientUtils.setEncryptionContextProvider(client, ecp);
+    AbfsClientUtils.setEncryptionContextProvider(ingressClient, ecp);
     if (isExceptionCase) {
     if (isExceptionCase) {
       LambdaTestUtils.intercept(IOException.class, () -> {
       LambdaTestUtils.intercept(IOException.class, () -> {
         switch (operation) {
         switch (operation) {
@@ -310,12 +324,26 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
           }
           }
         }
         }
       case WRITE:
       case WRITE:
-        return client.flush(path, 3, false, false, null,
-          null, encryptionAdapter, getTestTracingContext(fs, false));
+        if (ingressClient instanceof AbfsDfsClient) {
+          return ingressClient.flush(path, 3, false, false, null,
+              null, encryptionAdapter, getTestTracingContext(fs, false));
+        } else {
+          byte[] buffer = generateBlockListXml().getBytes(StandardCharsets.UTF_8);
+          return ingressClient.flush(buffer, path, false, null,
+              null, null, encryptionAdapter, getTestTracingContext(fs, false));
+        }
       case APPEND:
       case APPEND:
-        return client.append(path, "val".getBytes(),
-            new AppendRequestParameters(3, 0, 3, APPEND_MODE, false, null, true),
-            null, encryptionAdapter, getTestTracingContext(fs, false));
+        if (ingressClient instanceof AbfsDfsClient) {
+          return ingressClient.append(path, "val".getBytes(),
+              new AppendRequestParameters(3, 0, 3, APPEND_MODE, false, null,
+                  true),
+              null, encryptionAdapter, getTestTracingContext(fs, false));
+        } else {
+          return ingressClient.append(path, "val".getBytes(),
+              new AppendRequestParameters(3, 0, 3, APPEND_MODE, false, null,
+                  true, new BlobAppendRequestParameters(BLOCK_ID, null)),
+              null, encryptionAdapter, getTestTracingContext(fs, false));
+        }
       case SET_ACL:
       case SET_ACL:
         return client.setAcl(path, AclEntry.aclSpecToString(
         return client.setAcl(path, AclEntry.aclSpecToString(
           Lists.newArrayList(aclEntry(ACCESS, USER, ALL))),
           Lists.newArrayList(aclEntry(ACCESS, USER, ALL))),

+ 48 - 11
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java

@@ -28,6 +28,9 @@ import org.junit.Test;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
@@ -37,6 +40,7 @@ import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.BYTES_RECEIVED;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.GET_RESPONSES;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.GET_RESPONSES;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.SEND_REQUESTS;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.SEND_REQUESTS;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH;
 
 
 public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
 public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
 
 
@@ -47,6 +51,21 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
   public ITestAbfsNetworkStatistics() throws Exception {
   public ITestAbfsNetworkStatistics() throws Exception {
   }
   }
 
 
+  /**
+   * Counts the number of directories in the given path.
+   *
+   * @param path The path to be checked.
+   * @return The number of directories in the path.
+   */
+  private int countDirectory(String path) {
+    int index = path.indexOf(getFileSystemName());
+    if (index == -1) {
+      return 0;
+    }
+    return (int) path.substring(index + getFileSystemName().length()).chars()
+        .filter(ch -> ch == FORWARD_SLASH.charAt(0)).count();
+  }
+
   /**
   /**
    * Testing connections_made, send_request and bytes_send statistics in
    * Testing connections_made, send_request and bytes_send statistics in
    * {@link AbfsRestOperation}.
    * {@link AbfsRestOperation}.
@@ -59,12 +78,15 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
     AzureBlobFileSystem fs = getFileSystem();
     AzureBlobFileSystem fs = getFileSystem();
     Map<String, Long> metricMap;
     Map<String, Long> metricMap;
     Path sendRequestPath = path(getMethodName());
     Path sendRequestPath = path(getMethodName());
+    String path = sendRequestPath.toString();
+    int directory = countDirectory(path);
     String testNetworkStatsString = "http_send";
     String testNetworkStatsString = "http_send";
 
 
-    metricMap = fs.getInstrumentationMap();
+    metricMap = getInstrumentationMap(fs);
     long expectedConnectionsMade = metricMap.get(CONNECTIONS_MADE.getStatName());
     long expectedConnectionsMade = metricMap.get(CONNECTIONS_MADE.getStatName());
     long expectedRequestsSent = metricMap.get(SEND_REQUESTS.getStatName());
     long expectedRequestsSent = metricMap.get(SEND_REQUESTS.getStatName());
     long expectedBytesSent = 0;
     long expectedBytesSent = 0;
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
 
 
     // --------------------------------------------------------------------
     // --------------------------------------------------------------------
      // Operation: Creating AbfsOutputStream
      // Operation: Creating AbfsOutputStream
@@ -72,8 +94,14 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
         sendRequestPath)) {
         sendRequestPath)) {
        // Network stats calculation: For Creating AbfsOutputStream:
        // Network stats calculation: For Creating AbfsOutputStream:
        // 1 create request = 1 connection made and 1 send request
        // 1 create request = 1 connection made and 1 send request
-      expectedConnectionsMade++;
-      expectedRequestsSent++;
+      if (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs)) {
+        expectedRequestsSent += (directory);
+        // Per directory, we have 2 calls :- GetBlobProperties and PutBlob and 1 ListBlobs call (implicit check) for the path.
+        expectedConnectionsMade += ((directory * 2) + 1);
+      } else {
+        expectedRequestsSent++;
+        expectedConnectionsMade++;
+      }
       // --------------------------------------------------------------------
       // --------------------------------------------------------------------
 
 
       // Operation: Write small data
       // Operation: Write small data
@@ -103,7 +131,7 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
        *   1 append = 1 connection and 1 send request
        *   1 append = 1 connection and 1 send request
        */
        */
       if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(sendRequestPath).toString())
       if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(sendRequestPath).toString())
-          || (this.getConfiguration().isSmallWriteOptimizationEnabled())) {
+          || (fs.getAbfsStore().getAbfsConfiguration().isSmallWriteOptimizationEnabled())) {
         expectedConnectionsMade++;
         expectedConnectionsMade++;
         expectedRequestsSent++;
         expectedRequestsSent++;
       } else {
       } else {
@@ -114,7 +142,7 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
       // --------------------------------------------------------------------
       // --------------------------------------------------------------------
 
 
       // Assertions
       // Assertions
-      metricMap = fs.getInstrumentationMap();
+      metricMap = getInstrumentationMap(fs);
       assertAbfsStatistics(CONNECTIONS_MADE,
       assertAbfsStatistics(CONNECTIONS_MADE,
           expectedConnectionsMade, metricMap);
           expectedConnectionsMade, metricMap);
       assertAbfsStatistics(SEND_REQUESTS, expectedRequestsSent,
       assertAbfsStatistics(SEND_REQUESTS, expectedRequestsSent,
@@ -127,8 +155,11 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
     // Operation: AbfsOutputStream close.
     // Operation: AbfsOutputStream close.
     // Network Stats calculation: 1 flush (with close) is send.
     // Network Stats calculation: 1 flush (with close) is send.
     // 1 flush request = 1 connection and 1 send request
     // 1 flush request = 1 connection and 1 send request
-    expectedConnectionsMade++;
-    expectedRequestsSent++;
+    // Flush with no data is a no-op for blob endpoint, hence update only for dfs endpoint.
+    if (client instanceof AbfsDfsClient) {
+      expectedConnectionsMade++;
+      expectedRequestsSent++;
+    }
     // --------------------------------------------------------------------
     // --------------------------------------------------------------------
 
 
     // Operation: Re-create the file / create overwrite scenario
     // Operation: Re-create the file / create overwrite scenario
@@ -144,11 +175,17 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
        *    create overwrite=false (will fail in this case as file is indeed present)
        *    create overwrite=false (will fail in this case as file is indeed present)
        *    + getFileStatus to fetch the file ETag
        *    + getFileStatus to fetch the file ETag
        *    + create overwrite=true
        *    + create overwrite=true
-       *    = 3 connections and 2 send requests
+       *    = 3 connections and 2 send requests in case of Dfs Client
+       *    = 7 connections (5 GBP and 2 PutBlob calls) in case of Blob Client
        */
        */
-      if (this.getConfiguration().isConditionalCreateOverwriteEnabled()) {
-        expectedConnectionsMade += 3;
-        expectedRequestsSent += 2;
+      if (fs.getAbfsStore().getAbfsConfiguration().isConditionalCreateOverwriteEnabled()) {
+        if (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs)) {
+          expectedRequestsSent += 2;
+          expectedConnectionsMade += 7;
+        } else {
+          expectedConnectionsMade += 3;
+          expectedRequestsSent += 2;
+        }
       } else {
       } else {
         expectedConnectionsMade += 1;
         expectedConnectionsMade += 1;
         expectedRequestsSent += 1;
         expectedRequestsSent += 1;

+ 91 - 42
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java

@@ -24,6 +24,7 @@ import org.assertj.core.api.Assertions;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.oauth2.RetryTestTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.RetryTestTokenProvider;
@@ -64,12 +65,21 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest
 
 
       // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId)
       // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId)
       Assertions.assertThat(errorFields)
       Assertions.assertThat(errorFields)
-          .describedAs("Number of Fields in exception message are not as expected")
+          .describedAs(
+              "Number of Fields in exception message are not as expected")
           .hasSize(5);
           .hasSize(5);
       // Check status message, status code, HTTP Request Type and URL.
       // Check status message, status code, HTTP Request Type and URL.
-      Assertions.assertThat(errorFields[0].trim())
-          .describedAs("Error Message Field in exception message is wrong")
-          .isEqualTo("Operation failed: \"The specified path does not exist.\"");
+      if (getAbfsServiceType() == AbfsServiceType.BLOB) {
+        Assertions.assertThat(errorFields[0].trim())
+            .describedAs("Error Message Field in exception message is wrong")
+            .contains(
+                "Operation failed: \"The specified blob does not exist.\"");
+      } else {
+        Assertions.assertThat(errorFields[0].trim())
+            .describedAs("Error Message Field in exception message is wrong")
+            .isEqualTo(
+                "Operation failed: \"The specified path does not exist.\"");
+      }
       Assertions.assertThat(errorFields[1].trim())
       Assertions.assertThat(errorFields[1].trim())
           .describedAs("Status Code Field in exception message "
           .describedAs("Status Code Field in exception message "
               + "should be \"404\"")
               + "should be \"404\"")
@@ -91,46 +101,85 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest
     try {
     try {
       fs.listFiles(nonExistedFilePath2, false);
       fs.listFiles(nonExistedFilePath2, false);
     } catch (Exception ex) {
     } catch (Exception ex) {
-      // verify its format
       String errorMessage = ex.getLocalizedMessage();
       String errorMessage = ex.getLocalizedMessage();
       String[] errorFields = errorMessage.split(",");
       String[] errorFields = errorMessage.split(",");
-      // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId), StorageErrorCode, StorageErrorMessage.
-      Assertions.assertThat(errorFields)
-          .describedAs("Number of Fields in exception message are not as expected")
-          .hasSize(7);
-      // Check status message, status code, HTTP Request Type and URL.
-      Assertions.assertThat(errorFields[0].trim())
-          .describedAs("Error Message Field in exception message is wrong")
-          .isEqualTo("Operation failed: \"The specified path does not exist.\"");
-      Assertions.assertThat(errorFields[1].trim())
-          .describedAs("Status Code Field in exception message"
-              + " should be \"404\"")
-          .isEqualTo("404");
-      Assertions.assertThat(errorFields[2].trim())
-          .describedAs("Http Rest Method Field in exception message"
-              + " should be \"GET\"")
-          .isEqualTo("GET");
-      Assertions.assertThat(errorFields[3].trim())
-          .describedAs("Url Field in exception message"
-              + " should start with \"http\"")
-          .startsWith("http");
-      Assertions.assertThat(errorFields[4].trim())
-          .describedAs("ActivityId Field in exception message"
-              + " should start with \"rId:\"")
-          .startsWith("rId:");
-      // Check storage error code and storage error message.
-      Assertions.assertThat(errorFields[5].trim())
-          .describedAs("StorageErrorCode Field in exception message"
-              + " should be \"PathNotFound\"")
-          .isEqualTo("PathNotFound");
-      Assertions.assertThat(errorFields[6].trim())
-          .describedAs("StorageErrorMessage Field in exception message"
-              + " should contain \"RequestId\"")
-          .contains("RequestId");
-      Assertions.assertThat(errorFields[6].trim())
-          .describedAs("StorageErrorMessage Field in exception message"
-              + " should contain \"Time\"")
-          .contains("Time");
+      if (getAbfsServiceType() == AbfsServiceType.DFS) {
+        // verify its format
+        // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId), StorageErrorCode, StorageErrorMessage.
+        Assertions.assertThat(errorFields)
+            .describedAs(
+                "Number of Fields in exception message are not as expected")
+            .hasSize(7);
+        Assertions.assertThat(errorFields[0].trim())
+            .describedAs("Error Message Field in exception message is wrong")
+            .isEqualTo(
+                "Operation failed: \"The specified path does not exist.\"");
+        Assertions.assertThat(errorFields[1].trim())
+            .describedAs("Status Code Field in exception message"
+                + " should be \"404\"")
+            .isEqualTo("404");
+        Assertions.assertThat(errorFields[2].trim())
+            .describedAs("Http Rest Method Field in exception message"
+                + " should be \"GET\"")
+            .isEqualTo("GET");
+        Assertions.assertThat(errorFields[3].trim())
+            .describedAs("Url Field in exception message"
+                + " should start with \"http\"")
+            .startsWith("http");
+        Assertions.assertThat(errorFields[4].trim())
+            .describedAs("ActivityId Field in exception message"
+                + " should start with \"rId:\"")
+            .startsWith("rId:");
+        // Check storage error code and storage error message.
+        Assertions.assertThat(errorFields[5].trim())
+            .describedAs("StorageErrorCode Field in exception message"
+                + " should be \"PathNotFound\"")
+            .isEqualTo("PathNotFound");
+        Assertions.assertThat(errorFields[6].trim())
+            .describedAs("StorageErrorMessage Field in exception message"
+                + " should contain \"RequestId\"")
+            .contains("RequestId");
+        Assertions.assertThat(errorFields[6].trim())
+            .describedAs("StorageErrorMessage Field in exception message"
+                + " should contain \"Time\"")
+            .contains("Time");
+      } else {
+        // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId)
+        Assertions.assertThat(errorFields)
+            .describedAs(
+                "Number of Fields in exception message are not as expected")
+            .hasSize(5);
+        // Check status message, status code, HTTP Request Type and URL.
+        if (getAbfsStore(fs).getAbfsConfiguration().enableAbfsListIterator()) {
+          Assertions.assertThat(errorFields[0].trim())
+              .describedAs(
+                  "Error Message Field in exception message is wrong")
+              .contains(
+                  "Operation failed: \"The specified container does not exist.\"");
+        } else {
+          Assertions.assertThat(errorFields[0].trim())
+              .describedAs(
+                  "Error Message Field in exception message is wrong")
+              .contains(
+                  "Operation failed: \"The specified blob does not exist.\"");
+        }
+        Assertions.assertThat(errorFields[1].trim())
+            .describedAs("Status Code Field in exception message "
+                + "should be \"404\"")
+            .isEqualTo("404");
+        Assertions.assertThat(errorFields[2].trim())
+            .describedAs("Http Rest Method Field in exception message "
+                + "should be \"HEAD\"")
+            .isEqualTo("HEAD");
+        Assertions.assertThat(errorFields[3].trim())
+            .describedAs("Url Field in exception message"
+                + " should start with \"http\"")
+            .startsWith("http");
+        Assertions.assertThat(errorFields[4].trim())
+            .describedAs("ActivityId Field in exception message "
+                + "should start with \"rId:\"")
+            .startsWith("rId:");
+      }
     }
     }
   }
   }
 
 

+ 1056 - 30
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java

@@ -21,39 +21,89 @@ package org.apache.hadoop.fs.azurebfs;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
+import java.lang.reflect.Field;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Random;
 import java.util.Random;
 import java.util.Set;
 import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
 
 
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 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.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 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.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
+import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClientHandler;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AzureBlobIngressHandler;
+import org.apache.hadoop.fs.azurebfs.services.AzureDFSIngressHandler;
+import org.apache.hadoop.fs.azurebfs.services.AzureIngressHandler;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.store.BlockUploadStatistics;
 import org.apache.hadoop.fs.store.BlockUploadStatistics;
 import org.apache.hadoop.fs.store.DataBlocks;
 import org.apache.hadoop.fs.store.DataBlocks;
+import org.apache.hadoop.test.LambdaTestUtils;
 
 
+import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
+import static java.net.HttpURLConnection.HTTP_OK;
+import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLOCKS_BUFFER;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLOCKS_BUFFER;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INFINITE_LEASE_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INGRESS_SERVICE_TYPE;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient.generateBlockListXml;
 import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BUFFER_ARRAY;
 import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BUFFER_ARRAY;
 import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BUFFER_DISK;
 import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BUFFER_DISK;
 import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BYTEBUFFER;
 import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BYTEBUFFER;
 import static org.apache.hadoop.fs.store.DataBlocks.DataBlock.DestState.Closed;
 import static org.apache.hadoop.fs.store.DataBlocks.DataBlock.DestState.Closed;
 import static org.apache.hadoop.fs.store.DataBlocks.DataBlock.DestState.Writing;
 import static org.apache.hadoop.fs.store.DataBlocks.DataBlock.DestState.Writing;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.mockito.ArgumentMatchers.anyString;
 
 
 /**
 /**
  * Test append operations.
  * Test append operations.
  */
  */
 public class ITestAzureBlobFileSystemAppend extends
 public class ITestAzureBlobFileSystemAppend extends
     AbstractAbfsIntegrationTest {
     AbstractAbfsIntegrationTest {
+
   private static final String TEST_FILE_PATH = "testfile";
   private static final String TEST_FILE_PATH = "testfile";
+
   private static final String TEST_FOLDER_PATH = "testFolder";
   private static final String TEST_FOLDER_PATH = "testFolder";
 
 
+  private static final int TEN = 10;
+  private static final int TWENTY = 20;
+  private static final int THIRTY = 30;
+  private static final int HUNDRED = 100;
+
   public ITestAzureBlobFileSystemAppend() throws Exception {
   public ITestAzureBlobFileSystemAppend() throws Exception {
     super();
     super();
   }
   }
@@ -69,7 +119,7 @@ public class ITestAzureBlobFileSystemAppend extends
   @Test
   @Test
   public void testAppendWithLength0() throws Exception {
   public void testAppendWithLength0() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();
     final AzureBlobFileSystem fs = getFileSystem();
-    try(FSDataOutputStream stream = fs.create(path(TEST_FILE_PATH))) {
+    try (FSDataOutputStream stream = fs.create(path(TEST_FILE_PATH))) {
       final byte[] b = new byte[1024];
       final byte[] b = new byte[1024];
       new Random().nextBytes(b);
       new Random().nextBytes(b);
       stream.write(b, 1000, 0);
       stream.write(b, 1000, 0);
@@ -104,7 +154,7 @@ public class ITestAzureBlobFileSystemAppend extends
     fs.registerListener(new TracingHeaderValidator(
     fs.registerListener(new TracingHeaderValidator(
         fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
         fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
         fs.getFileSystemId(), FSOperationType.APPEND, false, 0));
         fs.getFileSystemId(), FSOperationType.APPEND, false, 0));
-    fs.append(testPath, 10);
+    fs.append(testPath, TEN);
   }
   }
 
 
   @Test
   @Test
@@ -116,37 +166,1013 @@ public class ITestAzureBlobFileSystemAppend extends
     for (String blockBufferType : blockBufferTypes) {
     for (String blockBufferType : blockBufferTypes) {
       Configuration configuration = new Configuration(getRawConfiguration());
       Configuration configuration = new Configuration(getRawConfiguration());
       configuration.set(DATA_BLOCKS_BUFFER, blockBufferType);
       configuration.set(DATA_BLOCKS_BUFFER, blockBufferType);
-      AzureBlobFileSystem fs = Mockito.spy(
-          (AzureBlobFileSystem) FileSystem.newInstance(configuration));
+      try (AzureBlobFileSystem fs = Mockito.spy(
+          (AzureBlobFileSystem) FileSystem.newInstance(configuration))) {
+        AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+        Mockito.doReturn(store).when(fs).getAbfsStore();
+        DataBlocks.DataBlock[] dataBlock = new DataBlocks.DataBlock[1];
+        Mockito.doAnswer(getBlobFactoryInvocation -> {
+          DataBlocks.BlockFactory factory = Mockito.spy(
+              (DataBlocks.BlockFactory) getBlobFactoryInvocation.callRealMethod());
+          Mockito.doAnswer(factoryCreateInvocation -> {
+                dataBlock[0] = Mockito.spy(
+                    (DataBlocks.DataBlock) factoryCreateInvocation.callRealMethod());
+                return dataBlock[0];
+              })
+              .when(factory)
+              .create(Mockito.anyLong(), Mockito.anyInt(), Mockito.any(
+                  BlockUploadStatistics.class));
+          return factory;
+        }).when(store).getBlockFactory();
+        try (OutputStream os = fs.create(
+            new Path(getMethodName() + "_" + blockBufferType))) {
+          os.write(new byte[1]);
+          Assertions.assertThat(dataBlock[0].getState())
+              .describedAs(
+                  "On write of data in outputStream, state should become Writing")
+              .isEqualTo(Writing);
+          os.close();
+          Mockito.verify(dataBlock[0], Mockito.times(1)).close();
+          Assertions.assertThat(dataBlock[0].getState())
+              .describedAs(
+                  "On close of outputStream, state should become Closed")
+              .isEqualTo(Closed);
+        }
+      }
+    }
+  }
+
+  /**
+   * Creates a file over DFS and attempts to append over Blob.
+   * It should fallback to DFS when appending to the file fails.
+   *
+   * @throws IOException if an I/O error occurs.
+   */
+  @Test
+  public void testCreateOverDfsAppendOverBlob() throws IOException {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path testPath = path(TEST_FILE_PATH);
+    AzureBlobFileSystemStore.Permissions permissions
+        = new AzureBlobFileSystemStore.Permissions(false,
+        FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+    fs.getAbfsStore().getClientHandler().getDfsClient().
+        createPath(makeQualified(testPath).toUri().getPath(), true, false,
+            permissions, false, null,
+            null, getTestTracingContext(fs, true));
+    fs.getAbfsStore()
+        .getAbfsConfiguration()
+        .set(FS_AZURE_INGRESS_SERVICE_TYPE, AbfsServiceType.BLOB.name());
+    FSDataOutputStream outputStream = fs.append(testPath);
+    AzureIngressHandler ingressHandler
+        = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler();
+    AbfsClient client = ingressHandler.getClient();
+    Assertions.assertThat(client)
+        .as("Blob client was not used before fallback")
+        .isInstanceOf(AbfsBlobClient.class);
+    outputStream.write(TEN);
+    outputStream.hsync();
+    outputStream.write(TWENTY);
+    outputStream.hsync();
+    outputStream.write(THIRTY);
+    outputStream.hsync();
+    AzureIngressHandler ingressHandlerFallback
+        = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler();
+    AbfsClient clientFallback = ingressHandlerFallback.getClient();
+    Assertions.assertThat(clientFallback)
+        .as("DFS client was not used after fallback")
+        .isInstanceOf(AbfsDfsClient.class);
+  }
+
+  /**
+   * Creates a file over Blob and attempts to append over DFS.
+   * It should fallback to Blob when appending to the file fails.
+   *
+   * @throws IOException if an I/O error occurs.
+   */
+  @Test
+  public void testCreateOverBlobAppendOverDfs() throws IOException {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    Configuration conf = getRawConfiguration();
+    conf.setBoolean(FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK, true);
+    conf.set(FS_AZURE_INGRESS_SERVICE_TYPE,
+        String.valueOf(AbfsServiceType.DFS));
+    try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
+        conf)) {
+      Path testPath = path(TEST_FILE_PATH);
+      AzureBlobFileSystemStore.Permissions permissions
+          = new AzureBlobFileSystemStore.Permissions(false,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      fs.getAbfsStore()
+          .getAbfsConfiguration()
+          .setBoolean(FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK, true);
+      fs.getAbfsStore()
+          .getAbfsConfiguration()
+          .set(FS_AZURE_INGRESS_SERVICE_TYPE,
+              String.valueOf(AbfsServiceType.DFS));
+      fs.getAbfsStore().getClientHandler().getBlobClient().
+          createPath(makeQualified(testPath).toUri().getPath(), true, false,
+              permissions, false, null,
+              null, getTestTracingContext(fs, true));
+      FSDataOutputStream outputStream = fs.append(testPath);
+      outputStream.write(TEN);
+      outputStream.hsync();
+      outputStream.write(TWENTY);
+      outputStream.hsync();
+      outputStream.write(THIRTY);
+      outputStream.hsync();
+    }
+  }
+
+  /**
+   * Creates an Append Blob over Blob and attempts to append over DFS.
+   * It should fallback to Blob when appending to the file fails.
+   *
+   * @throws IOException if an I/O error occurs.
+   */
+  @Test
+  public void testCreateAppendBlobOverBlobEndpointAppendOverDfs()
+      throws IOException, NoSuchFieldException, IllegalAccessException {
+    Configuration conf = getRawConfiguration();
+    conf.setBoolean(FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK, true);
+    conf.set(FS_AZURE_INGRESS_SERVICE_TYPE,
+        String.valueOf(AbfsServiceType.DFS));
+    try (AzureBlobFileSystem fs = Mockito.spy(
+        (AzureBlobFileSystem) FileSystem.newInstance(conf))) {
       AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
       AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
-      Mockito.doReturn(store).when(fs).getAbfsStore();
-      DataBlocks.DataBlock[] dataBlock = new DataBlocks.DataBlock[1];
-      Mockito.doAnswer(getBlobFactoryInvocation -> {
-        DataBlocks.BlockFactory factory = Mockito.spy(
-            (DataBlocks.BlockFactory) getBlobFactoryInvocation.callRealMethod());
-        Mockito.doAnswer(factoryCreateInvocation -> {
-              dataBlock[0] = Mockito.spy(
-                  (DataBlocks.DataBlock) factoryCreateInvocation.callRealMethod());
-              return dataBlock[0];
-            })
-            .when(factory)
-            .create(Mockito.anyLong(), Mockito.anyInt(), Mockito.any(
-                BlockUploadStatistics.class));
-        return factory;
-      }).when(store).getBlockFactory();
-      try (OutputStream os = fs.create(
-          new Path(getMethodName() + "_" + blockBufferType))) {
-        os.write(new byte[1]);
-        Assertions.assertThat(dataBlock[0].getState())
-            .describedAs(
-                "On write of data in outputStream, state should become Writing")
-            .isEqualTo(Writing);
+      Mockito.doReturn(true).when(store).isAppendBlobKey(anyString());
+
+      // Set abfsStore as our mocked value.
+      Field privateField = AzureBlobFileSystem.class.getDeclaredField(
+          "abfsStore");
+      privateField.setAccessible(true);
+      privateField.set(fs, store);
+      Path testPath = path(TEST_FILE_PATH);
+      AzureBlobFileSystemStore.Permissions permissions
+          = new AzureBlobFileSystemStore.Permissions(false,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      fs.getAbfsStore()
+          .getAbfsConfiguration()
+          .setBoolean(FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK, true);
+      fs.getAbfsStore()
+          .getAbfsConfiguration()
+          .set(FS_AZURE_INGRESS_SERVICE_TYPE,
+              String.valueOf(AbfsServiceType.DFS));
+      fs.getAbfsStore().getClientHandler().getBlobClient().
+          createPath(makeQualified(testPath).toUri().getPath(), true, false,
+              permissions, true, null,
+              null, getTestTracingContext(fs, true));
+      FSDataOutputStream outputStream = fs.append(testPath);
+      outputStream.write(TEN);
+      outputStream.hsync();
+      outputStream.write(TWENTY);
+      outputStream.hsync();
+      outputStream.write(THIRTY);
+      outputStream.hsync();
+    }
+  }
+
+  /**
+   * Creates an append Blob over DFS and attempts to append over Blob.
+   * It should fallback to DFS when appending to the file fails.
+   *
+   * @throws IOException if an I/O error occurs.
+   */
+  @Test
+  public void testCreateAppendBlobOverDfsEndpointAppendOverBlob()
+      throws IOException, NoSuchFieldException, IllegalAccessException {
+    Assume.assumeTrue(
+        "FNS does not support append blob creation for DFS endpoint",
+        getIsNamespaceEnabled(getFileSystem()));
+    final AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    Mockito.doReturn(true).when(store).isAppendBlobKey(anyString());
+
+    // Set abfsStore as our mocked value.
+    Field privateField = AzureBlobFileSystem.class.getDeclaredField(
+        "abfsStore");
+    privateField.setAccessible(true);
+    privateField.set(fs, store);
+    Path testPath = path(TEST_FILE_PATH);
+    AzureBlobFileSystemStore.Permissions permissions
+        = new AzureBlobFileSystemStore.Permissions(false,
+        FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+    fs.getAbfsStore().getClientHandler().getDfsClient().
+        createPath(makeQualified(testPath).toUri().getPath(), true, false,
+            permissions, true, null,
+            null, getTestTracingContext(fs, true));
+    fs.getAbfsStore()
+        .getAbfsConfiguration()
+        .set(FS_AZURE_INGRESS_SERVICE_TYPE, AbfsServiceType.BLOB.name());
+    FSDataOutputStream outputStream = fs.append(testPath);
+    AzureIngressHandler ingressHandler
+        = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler();
+    AbfsClient client = ingressHandler.getClient();
+    Assertions.assertThat(client)
+        .as("Blob client was not used before fallback")
+        .isInstanceOf(AbfsBlobClient.class);
+    outputStream.write(TEN);
+    outputStream.hsync();
+    outputStream.write(TWENTY);
+    outputStream.hsync();
+    outputStream.write(THIRTY);
+    outputStream.flush();
+    AzureIngressHandler ingressHandlerFallback
+        = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler();
+    AbfsClient clientFallback = ingressHandlerFallback.getClient();
+    Assertions.assertThat(clientFallback)
+        .as("DFS client was not used after fallback")
+        .isInstanceOf(AbfsDfsClient.class);
+  }
+
+
+  /**
+   * Tests the correct retrieval of the AzureIngressHandler based on the configured ingress service type.
+   *
+   * @throws IOException if an I/O error occurs
+   */
+  @Test
+  public void testValidateIngressHandler() throws IOException {
+    Configuration configuration = getRawConfiguration();
+    configuration.set(FS_AZURE_INGRESS_SERVICE_TYPE,
+        AbfsServiceType.BLOB.name());
+    try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
+        configuration)) {
+      Path testPath = path(TEST_FILE_PATH);
+      AzureBlobFileSystemStore.Permissions permissions
+          = new AzureBlobFileSystemStore.Permissions(false,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      fs.getAbfsStore().getClientHandler().getBlobClient().
+          createPath(makeQualified(testPath).toUri().getPath(), true,
+              false,
+              permissions, false, null,
+              null, getTestTracingContext(fs, true));
+      FSDataOutputStream outputStream = fs.append(testPath);
+      AzureIngressHandler ingressHandler
+          = ((AbfsOutputStream) outputStream.getWrappedStream()).getIngressHandler();
+      Assertions.assertThat(ingressHandler)
+          .as("Blob Ingress handler instance is not correct")
+          .isInstanceOf(AzureBlobIngressHandler.class);
+      AbfsClient client = ingressHandler.getClient();
+      Assertions.assertThat(client)
+          .as("Blob client was not used correctly")
+          .isInstanceOf(AbfsBlobClient.class);
+
+      Path testPath1 = new Path("testFile1");
+      fs.getAbfsStore().getClientHandler().getBlobClient().
+          createPath(makeQualified(testPath1).toUri().getPath(), true,
+              false,
+              permissions, false, null,
+              null, getTestTracingContext(fs, true));
+      fs.getAbfsStore()
+          .getAbfsConfiguration()
+          .set(FS_AZURE_INGRESS_SERVICE_TYPE, AbfsServiceType.DFS.name());
+      FSDataOutputStream outputStream1 = fs.append(testPath1);
+      AzureIngressHandler ingressHandler1
+          = ((AbfsOutputStream) outputStream1.getWrappedStream()).getIngressHandler();
+      Assertions.assertThat(ingressHandler1)
+          .as("DFS Ingress handler instance is not correct")
+          .isInstanceOf(AzureDFSIngressHandler.class);
+      AbfsClient client1 = ingressHandler1.getClient();
+      Assertions.assertThat(client1)
+          .as("Dfs client was not used correctly")
+          .isInstanceOf(AbfsDfsClient.class);
+    }
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testAppendImplicitDirectory() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path folderPath = new Path(TEST_FOLDER_PATH);
+    fs.mkdirs(folderPath);
+    fs.append(folderPath.getParent());
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testAppendFileNotExists() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path folderPath = new Path(TEST_FOLDER_PATH);
+    fs.append(folderPath);
+  }
+
+  /**
+   * Create directory over dfs endpoint and append over blob endpoint.
+   * Should return error as append is not supported for directory.
+   * **/
+  @Test(expected = IOException.class)
+  public void testCreateExplicitDirectoryOverDfsAppendOverBlob()
+      throws IOException {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path folderPath = path(TEST_FOLDER_PATH);
+    AzureBlobFileSystemStore.Permissions permissions
+        = new AzureBlobFileSystemStore.Permissions(false,
+        FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+    fs.getAbfsStore().getClientHandler().getDfsClient().
+        createPath(makeQualified(folderPath).toUri().getPath(), false, false,
+            permissions, false, null,
+            null, getTestTracingContext(fs, true));
+    FSDataOutputStream outputStream = fs.append(folderPath);
+    outputStream.write(TEN);
+    outputStream.hsync();
+  }
+
+  /**
+   * Recreate file between append and flush. Etag mismatch happens.
+   **/
+  @Test(expected = IOException.class)
+  public void testRecreateAppendAndFlush() throws IOException {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path filePath = path(TEST_FILE_PATH);
+    fs.create(filePath);
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB);
+    FSDataOutputStream outputStream = fs.append(filePath);
+    outputStream.write(TEN);
+    try (AzureBlobFileSystem fs1
+        = (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration());
+    FSDataOutputStream outputStream1 = fs1.create(filePath)) {
+      outputStream.hsync();
+    }
+  }
+
+  /**
+   * Recreate directory between append and flush. Etag mismatch happens.
+   **/
+  @Test(expected = IOException.class)
+  public void testRecreateDirectoryAppendAndFlush() throws IOException {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path filePath = path(TEST_FILE_PATH);
+    fs.create(filePath);
+    FSDataOutputStream outputStream = fs.append(filePath);
+    outputStream.write(TEN);
+    try (AzureBlobFileSystem fs1
+        = (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration())) {
+      fs1.mkdirs(filePath);
+      outputStream.hsync();
+    }
+  }
+
+  /**
+   * Checks a list of futures for exceptions.
+   *
+   * This method iterates over a list of futures, waits for each task to complete,
+   * and handles any exceptions thrown by the lambda expressions. If a
+   * RuntimeException is caught, it increments the exceptionCaught counter.
+   * If an unexpected exception is caught, it prints the exception to the standard error.
+   * Finally, it asserts that no RuntimeExceptions were caught.
+   *
+   * @param futures The list of futures to check for exceptions.
+   */
+  private void checkFuturesForExceptions(List<Future<?>> futures, int exceptionVal) {
+    int exceptionCaught = 0;
+    for (Future<?> future : futures) {
+      try {
+        future.get(); // wait for the task to complete and handle any exceptions thrown by the lambda expression
+      } catch (ExecutionException e) {
+        Throwable cause = e.getCause();
+        if (cause instanceof RuntimeException) {
+          exceptionCaught++;
+        } else {
+          System.err.println("Unexpected exception caught: " + cause);
+        }
+      } catch (InterruptedException e) {
+        // handle interruption
+      }
+    }
+    assertEquals(exceptionCaught, exceptionVal);
+  }
+
+  /**
+   * Verify that parallel write with same offset from different output streams will not throw exception.
+   **/
+  @Test
+  public void testParallelWriteSameOffsetDifferentOutputStreams()
+      throws Exception {
+    Configuration configuration = getRawConfiguration();
+    configuration.set(FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE, "false");
+     try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
+        configuration)) {
+       ExecutorService executorService = Executors.newFixedThreadPool(5);
+       List<Future<?>> futures = new ArrayList<>();
+
+       final byte[] b = new byte[8 * ONE_MB];
+       new Random().nextBytes(b);
+       final Path filePath = path(TEST_FILE_PATH);
+       // Create three output streams
+       FSDataOutputStream out1 = fs.create(filePath);
+       FSDataOutputStream out2 = fs.append(filePath);
+       FSDataOutputStream out3 = fs.append(filePath);
+
+       // Submit tasks to write to each output stream with the same offset
+       futures.add(executorService.submit(() -> {
+         try {
+           out1.write(b, TEN, 2 * HUNDRED);
+         } catch (IOException e) {
+           throw new RuntimeException(e);
+         }
+       }));
+
+       futures.add(executorService.submit(() -> {
+         try {
+           out2.write(b, TEN, 2 * HUNDRED);
+         } catch (IOException e) {
+           throw new RuntimeException(e);
+         }
+       }));
+
+       futures.add(executorService.submit(() -> {
+         try {
+           out3.write(b, TEN, 2 * HUNDRED);
+         } catch (IOException e) {
+           throw new RuntimeException(e);
+         }
+       }));
+
+       checkFuturesForExceptions(futures, 0);
+     }
+  }
+
+  /**
+   * Verify that parallel write for different content length will not throw exception.
+   **/
+  @Test
+  public void testParallelWriteDifferentContentLength() throws Exception {
+    Configuration configuration = getRawConfiguration();
+    configuration.set(FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE, "false");
+    try (FileSystem fs = FileSystem.newInstance(configuration)) {
+      ExecutorService executorService = Executors.newFixedThreadPool(5);
+      List<Future<?>> futures = new ArrayList<>();
+
+      final Path filePath = path(TEST_FILE_PATH);
+      // Create three output streams with different content length
+      FSDataOutputStream out1 = fs.create(filePath);
+      final byte[] b1 = new byte[8 * ONE_MB];
+      new Random().nextBytes(b1);
+
+      FSDataOutputStream out2 = fs.append(filePath);
+      FSDataOutputStream out3 = fs.append(filePath);
+
+      // Submit tasks to write to each output stream
+      futures.add(executorService.submit(() -> {
+        try {
+          out1.write(b1, TEN, 2 * HUNDRED);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }));
+
+      futures.add(executorService.submit(() -> {
+        try {
+          out2.write(b1, TWENTY, 3 * HUNDRED);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }));
+
+      futures.add(executorService.submit(() -> {
+        try {
+          out3.write(b1, THIRTY, 4 * HUNDRED);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }));
+
+      checkFuturesForExceptions(futures, 0);
+    }
+  }
+
+  /**
+   * Verify that parallel write for different content length will not throw exception.
+   **/
+  @Test
+  public void testParallelWriteOutputStreamClose() throws Exception {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    AzureBlobFileSystem fs = getFileSystem();
+    final Path secondarytestfile = new Path("secondarytestfile");
+    ExecutorService executorService = Executors.newFixedThreadPool(2);
+    List<Future<?>> futures = new ArrayList<>();
+
+    FSDataOutputStream out1 = fs.create(secondarytestfile);
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB);
+    AbfsOutputStream outputStream1 = (AbfsOutputStream) out1.getWrappedStream();
+    String fileETag = outputStream1.getIngressHandler().getETag();
+    final byte[] b1 = new byte[8 * ONE_MB];
+    new Random().nextBytes(b1);
+    final byte[] b2 = new byte[8 * ONE_MB];
+    new Random().nextBytes(b2);
+
+    FSDataOutputStream out2 = fs.append(secondarytestfile);
+
+    // Submit tasks to write to each output stream
+    futures.add(executorService.submit(() -> {
+      try {
+        out1.write(b1, 0, 2 * HUNDRED);
+        out1.close();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }));
+
+    futures.add(executorService.submit(() -> {
+      try {
+        out2.write(b2, 0, 4 * HUNDRED);
+        out2.close();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }));
+
+   checkFuturesForExceptions(futures, 1);
+    // Validate that the data written in the buffer is the same as what was read
+    final byte[] readBuffer = new byte[8 * ONE_MB];
+    int result;
+    FSDataInputStream inputStream = fs.open(secondarytestfile);
+    inputStream.seek(0);
+
+    AbfsOutputStream outputStream2 = (AbfsOutputStream) out1.getWrappedStream();
+    String out1Etag = outputStream2.getIngressHandler().getETag();
+
+    AbfsOutputStream outputStream3 = (AbfsOutputStream) out2.getWrappedStream();
+    String out2Etag = outputStream3.getIngressHandler().getETag();
+
+    if (!fileETag.equals(out1Etag)) {
+      result = inputStream.read(readBuffer, 0, 4 * ONE_MB);
+      assertEquals(result, 2 * HUNDRED); // Verify that the number of bytes read matches the number of bytes written
+      assertArrayEquals(
+          Arrays.copyOfRange(readBuffer, 0, result), Arrays.copyOfRange(b1, 0,
+              result)); // Verify that the data read matches the original data written
+    } else if (!fileETag.equals(out2Etag)) {
+      result = inputStream.read(readBuffer, 0, 4 * ONE_MB);
+      assertEquals(result, 4 * HUNDRED); // Verify that the number of bytes read matches the number of bytes written
+      assertArrayEquals(Arrays.copyOfRange(readBuffer, 0, result),
+          Arrays.copyOfRange(b2, 0,
+              result)); // Verify that the data read matches the original data written
+    } else {
+      fail("Neither out1 nor out2 was flushed successfully.");
+    }
+  }
+
+  /**
+   * Verify that once flushed etag changes.
+   **/
+  @Test
+  public void testEtagMismatch() throws Exception {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    AzureBlobFileSystem fs = getFileSystem();
+    final Path filePath = path(TEST_FILE_PATH);
+    FSDataOutputStream out1 = fs.create(filePath);
+    FSDataOutputStream out2 = fs.create(filePath);
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB);
+    out2.write(TEN);
+    out2.hsync();
+    out1.write(TEN);
+    intercept(IOException.class, () -> out1.hsync());
+  }
+
+  @Test
+  public void testAppendWithLease() throws Exception {
+    final Path testFilePath = new Path(path(methodName.getMethodName()),
+        TEST_FILE_PATH);
+    final AzureBlobFileSystem fs = Mockito.spy(
+        getCustomFileSystem(testFilePath.getParent(), 1));
+    FsPermission permission = new FsPermission(FsAction.ALL, FsAction.ALL,
+        FsAction.ALL);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    AbfsOutputStream outputStream = (AbfsOutputStream) fs.getAbfsStore()
+        .createFile(testFilePath, null, true,
+            permission, umask, getTestTracingContext(fs, true));
+    outputStream.write(TEN);
+    outputStream.close();
+    assertNotNull(outputStream.getLeaseId());
+  }
+
+  private AzureBlobFileSystem getCustomFileSystem(Path infiniteLeaseDirs,
+      int numLeaseThreads) throws Exception {
+    Configuration conf = getRawConfiguration();
+    conf.setBoolean(String.format("fs.%s.impl.disable.cache", getAbfsScheme()),
+        true);
+    conf.set(FS_AZURE_INFINITE_LEASE_KEY, infiniteLeaseDirs.toUri().getPath());
+    conf.setInt(FS_AZURE_LEASE_THREADS, numLeaseThreads);
+    FileSystem fileSystem = FileSystem.newInstance(conf);
+    return (AzureBlobFileSystem) fileSystem;
+  }
+
+  @Test
+  public void testAppendImplicitDirectoryAzcopy() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    createAzCopyFolder(new Path("/src"));
+    createAzCopyFile(new Path("/src/file"));
+    intercept(FileNotFoundException.class, () -> fs.append(new Path("/src")));
+  }
+
+  /**
+   * If a write operation fails asynchronously, when the next write comes once failure is
+   * registered, that operation would fail with the exception caught on previous
+   * write operation.
+   * The next close, hsync, hflush would also fail for the last caught exception.
+   */
+  @Test
+  public void testIntermittentAppendFailureToBeReported() throws Exception {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    try (AzureBlobFileSystem fs = Mockito.spy(
+        (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) {
+      Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+      AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+      assumeBlobServiceType();
+
+      AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+      AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+      Mockito.doReturn(clientHandler).when(store).getClientHandler();
+      Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+      Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+
+      Mockito.doThrow(
+              new AbfsRestOperationException(HTTP_UNAVAILABLE, "", "", new Exception()))
+          .when(blobClient)
+          .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any(
+                  AppendRequestParameters.class), Mockito.any(), Mockito.any(),
+              Mockito.any(TracingContext.class));
+
+      byte[] bytes = new byte[1024 * 1024 * 8];
+      new Random().nextBytes(bytes);
+
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        try (FSDataOutputStream os = createMockedOutputStream(fs,
+            new Path("/test/file"), blobClient)) {
+          os.write(bytes);
+        }
+      });
+
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        FSDataOutputStream os = createMockedOutputStream(fs,
+            new Path("/test/file/file1"), blobClient);
+        os.write(bytes);
         os.close();
         os.close();
-        Mockito.verify(dataBlock[0], Mockito.times(1)).close();
-        Assertions.assertThat(dataBlock[0].getState())
-            .describedAs("On close of outputStream, state should become Closed")
-            .isEqualTo(Closed);
+      });
+
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        FSDataOutputStream os = createMockedOutputStream(fs,
+            new Path("/test/file/file2"), blobClient);
+        os.write(bytes);
+        os.hsync();
+      });
+
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        FSDataOutputStream os = createMockedOutputStream(fs,
+            new Path("/test/file/file3"), blobClient);
+        os.write(bytes);
+        os.hflush();
+      });
+
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        AbfsOutputStream os = (AbfsOutputStream) createMockedOutputStream(fs,
+            new Path("/test/file/file4"), blobClient).getWrappedStream();
+        os.write(bytes);
+        while (!os.areWriteOperationsTasksDone()) {
+          // No operation inside the loop
+        }
+        os.write(bytes);
+      });
+    }
+  }
+
+  /**
+   * Creates a mocked FSDataOutputStream for testing purposes.
+   *
+   * This method creates a mocked FSDataOutputStream by wrapping an AbfsOutputStream
+   * and its associated AzureIngressHandler. The method uses Mockito to create spies
+   * for the AbfsOutputStream and AzureIngressHandler, and sets up the necessary
+   * interactions between them.
+   *
+   * @param fs The AzureBlobFileSystem instance used to create the output stream.
+   * @param path The Path where the output stream will be created.
+   * @param client The AbfsClient instance to be used by the AzureIngressHandler.
+   * @return A mocked FSDataOutputStream instance.
+   * @throws IOException If an I/O error occurs while creating the output stream.
+   */
+  private FSDataOutputStream createMockedOutputStream(AzureBlobFileSystem fs,
+      Path path,
+      AbfsClient client) throws IOException {
+    AbfsOutputStream abfsOutputStream = Mockito.spy(
+        (AbfsOutputStream) fs.create(path).getWrappedStream());
+    AzureIngressHandler ingressHandler = Mockito.spy(
+        abfsOutputStream.getIngressHandler());
+    Mockito.doReturn(ingressHandler).when(abfsOutputStream).getIngressHandler();
+    Mockito.doReturn(client).when(ingressHandler).getClient();
+
+    FSDataOutputStream fsDataOutputStream = Mockito.spy(
+        new FSDataOutputStream(abfsOutputStream, null));
+    return fsDataOutputStream;
+  }
+
+  /**
+   * Test to check when async write takes time, the close, hsync, hflush method
+   * wait to get async ops completed and then flush. If async ops fail, the methods
+   * will throw exception.
+   */
+  @Test
+  public void testWriteAsyncOpFailedAfterCloseCalled() throws Exception {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    try (AzureBlobFileSystem fs = Mockito.spy(
+        (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) {
+      AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+      AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+      AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+      AbfsDfsClient dfsClient = Mockito.spy(clientHandler.getDfsClient());
+
+      AbfsClient client = clientHandler.getIngressClient();
+      if (clientHandler.getIngressClient() instanceof AbfsBlobClient) {
+        Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+        Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+      } else {
+        Mockito.doReturn(dfsClient).when(clientHandler).getDfsClient();
+        Mockito.doReturn(dfsClient).when(clientHandler).getIngressClient();
       }
       }
+      Mockito.doReturn(clientHandler).when(store).getClientHandler();
+
+      byte[] bytes = new byte[1024 * 1024 * 8];
+      new Random().nextBytes(bytes);
+
+      AtomicInteger count = new AtomicInteger(0);
+
+      Mockito.doAnswer(answer -> {
+            count.incrementAndGet();
+            while (count.get() < 2) {
+              // No operation inside the loop
+            }
+            Thread.sleep(10 * HUNDRED);
+            throw new AbfsRestOperationException(HTTP_UNAVAILABLE, "", "",
+                new Exception());
+          })
+          .when(client instanceof AbfsBlobClient ? blobClient : dfsClient)
+          .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any(
+                  AppendRequestParameters.class), Mockito.any(), Mockito.any(),
+              Mockito.any(TracingContext.class));
+
+      Mockito.doAnswer(answer -> {
+            count.incrementAndGet();
+            while (count.get() < 2) {
+              // No operation inside the loop
+            }
+            Thread.sleep(10 * HUNDRED);
+            throw new AbfsRestOperationException(HTTP_UNAVAILABLE, "", "",
+                new Exception());
+          })
+          .when(client instanceof AbfsBlobClient ? blobClient : dfsClient)
+          .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any(
+                  AppendRequestParameters.class), Mockito.any(), Mockito.any(),
+              Mockito.any(TracingContext.class));
+
+      FSDataOutputStream os = createMockedOutputStream(fs,
+          new Path("/test/file"),
+          client instanceof AbfsBlobClient ? blobClient : dfsClient);
+      os.write(bytes);
+      os.write(bytes);
+      LambdaTestUtils.intercept(IOException.class, os::close);
+
+      count.set(0);
+      FSDataOutputStream os1 = createMockedOutputStream(fs,
+          new Path("/test/file1"),
+          client instanceof AbfsBlobClient ? blobClient : dfsClient);
+      os1.write(bytes);
+      os1.write(bytes);
+      LambdaTestUtils.intercept(IOException.class, os1::hsync);
+
+      count.set(0);
+      FSDataOutputStream os2 = createMockedOutputStream(fs,
+          new Path("/test/file2"),
+          client instanceof AbfsBlobClient ? blobClient : dfsClient);
+      os2.write(bytes);
+      os2.write(bytes);
+      LambdaTestUtils.intercept(IOException.class, os2::hflush);
+    }
+  }
+
+  /**
+   * Helper method that generates blockId.
+   * @param position The offset needed to generate blockId.
+   * @return String representing the block ID generated.
+   */
+  private String generateBlockId(AbfsOutputStream os, long position) {
+    String streamId = os.getStreamID();
+    String streamIdHash = Integer.toString(streamId.hashCode());
+    String blockId = String.format("%d_%s", position, streamIdHash);
+    byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH];
+    System.arraycopy(blockId.getBytes(), 0, blockIdByteArray, 0, Math.min(BLOCK_ID_LENGTH, blockId.length()));
+    return new String(Base64.encodeBase64(blockIdByteArray), StandardCharsets.UTF_8);
+  }
+
+  /**
+   * Test to simulate a successful flush operation followed by a connection reset
+   * on the response, triggering a retry.
+   *
+   * This test verifies that the flush operation is retried in the event of a
+   * connection reset during the response phase. The test creates a mock
+   * AzureBlobFileSystem and its associated components to simulate the flush
+   * operation and the connection reset. It then verifies that the flush
+   * operation is retried once before succeeding if the md5hash matches.
+   *
+   * @throws Exception if an error occurs during the test execution.
+   */
+  @Test
+  public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Exception {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    // Create a spy of AzureBlobFileSystem
+    try (AzureBlobFileSystem fs = Mockito.spy(
+        (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) {
+      Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+
+      // Create a spy of AzureBlobFileSystemStore
+      AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+      assumeBlobServiceType();
+
+      // Create spies for the client handler and blob client
+      AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+      AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+      // Set up the spies to return the mocked objects
+      Mockito.doReturn(clientHandler).when(store).getClientHandler();
+      Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+      Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+      AtomicInteger flushCount = new AtomicInteger(0);
+      FSDataOutputStream os = createMockedOutputStream(fs,
+          new Path("/test/file"), blobClient);
+      AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
+      String eTag = out.getIngressHandler().getETag();
+      byte[] bytes = new byte[1024 * 1024 * 8];
+      new Random().nextBytes(bytes);
+      // Write some bytes and attempt to flush, which should retry
+      out.write(bytes);
+      List<String> list = new ArrayList<>();
+      list.add(generateBlockId(out, 0));
+      String blockListXml = generateBlockListXml(list);
+
+      Mockito.doAnswer(answer -> {
+        // Set up the mock for the flush operation
+        AbfsClientTestUtil.setMockAbfsRestOperationForFlushOperation(blobClient,
+            eTag, blockListXml,
+            (httpOperation) -> {
+              Mockito.doAnswer(invocation -> {
+                // Call the real processResponse method
+                invocation.callRealMethod();
+
+                int currentCount = flushCount.incrementAndGet();
+                if (currentCount == 1) {
+                  Mockito.when(httpOperation.getStatusCode())
+                      .thenReturn(
+                          HTTP_INTERNAL_ERROR); // Status code 500 for Internal Server Error
+                  Mockito.when(httpOperation.getStorageErrorMessage())
+                      .thenReturn("CONNECTION_RESET"); // Error message
+                  throw new IOException("Connection Reset");
+                }
+                return null;
+              }).when(httpOperation).processResponse(
+                  Mockito.nullable(byte[].class),
+                  Mockito.anyInt(),
+                  Mockito.anyInt()
+              );
+
+              return httpOperation;
+            });
+        return answer.callRealMethod();
+      }).when(blobClient).flush(
+          Mockito.any(byte[].class),
+          Mockito.anyString(),
+          Mockito.anyBoolean(),
+          Mockito.nullable(String.class),
+          Mockito.nullable(String.class),
+          Mockito.anyString(),
+          Mockito.nullable(ContextEncryptionAdapter.class),
+          Mockito.any(TracingContext.class)
+      );
+
+      out.hsync();
+      out.close();
+      Mockito.verify(blobClient, Mockito.times(1)).flush(
+          Mockito.any(byte[].class),
+          Mockito.anyString(),
+          Mockito.anyBoolean(),
+          Mockito.nullable(String.class),
+          Mockito.nullable(String.class),
+          Mockito.anyString(),
+          Mockito.nullable(ContextEncryptionAdapter.class),
+          Mockito.any(TracingContext.class));
+    }
+  }
+
+  /**
+   * Test to simulate a successful flush operation followed by a connection reset
+   * on the response, triggering a retry.
+   *
+   * This test verifies that the flush operation is retried in the event of a
+   * connection reset during the response phase. The test creates a mock
+   * AzureBlobFileSystem and its associated components to simulate the flush
+   * operation and the connection reset. It then verifies that the flush
+   * operation is retried once before succeeding if the md5hash matches.
+   *
+   * @throws Exception if an error occurs during the test execution.
+   */
+  @Test
+  public void testFlushSuccessWithConnectionResetOnResponseInvalidMd5() throws Exception {
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    // Create a spy of AzureBlobFileSystem
+    try (AzureBlobFileSystem fs = Mockito.spy(
+        (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()))) {
+      Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+
+      // Create a spy of AzureBlobFileSystemStore
+      AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+      assumeBlobServiceType();
+
+      // Create spies for the client handler and blob client
+      AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+      AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+      // Set up the spies to return the mocked objects
+      Mockito.doReturn(clientHandler).when(store).getClientHandler();
+      Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+      Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+      AtomicInteger flushCount = new AtomicInteger(0);
+      FSDataOutputStream os = createMockedOutputStream(fs,
+          new Path("/test/file"), blobClient);
+      AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
+      String eTag = out.getIngressHandler().getETag();
+      byte[] bytes = new byte[1024 * 1024 * 8];
+      new Random().nextBytes(bytes);
+      // Write some bytes and attempt to flush, which should retry
+      out.write(bytes);
+      List<String> list = new ArrayList<>();
+      list.add(generateBlockId(out, 0));
+      String blockListXml = generateBlockListXml(list);
+
+      Mockito.doAnswer(answer -> {
+        // Set up the mock for the flush operation
+        AbfsClientTestUtil.setMockAbfsRestOperationForFlushOperation(blobClient,
+            eTag, blockListXml,
+            (httpOperation) -> {
+              Mockito.doAnswer(invocation -> {
+                // Call the real processResponse method
+                invocation.callRealMethod();
+
+                int currentCount = flushCount.incrementAndGet();
+                if (currentCount == 1) {
+                  Mockito.when(httpOperation.getStatusCode())
+                      .thenReturn(
+                          HTTP_INTERNAL_ERROR); // Status code 500 for Internal Server Error
+                  Mockito.when(httpOperation.getStorageErrorMessage())
+                      .thenReturn("CONNECTION_RESET"); // Error message
+                  throw new IOException("Connection Reset");
+                } else if (currentCount == 2) {
+                  Mockito.when(httpOperation.getStatusCode())
+                      .thenReturn(HTTP_OK);
+                  Mockito.when(httpOperation.getStorageErrorMessage())
+                      .thenReturn("HTTP_OK");
+                }
+                return null;
+              }).when(httpOperation).processResponse(
+                  Mockito.nullable(byte[].class),
+                  Mockito.anyInt(),
+                  Mockito.anyInt()
+              );
+
+              return httpOperation;
+            });
+        return answer.callRealMethod();
+      }).when(blobClient).flush(
+          Mockito.any(byte[].class),
+          Mockito.anyString(),
+          Mockito.anyBoolean(),
+          Mockito.nullable(String.class),
+          Mockito.nullable(String.class),
+          Mockito.anyString(),
+          Mockito.nullable(ContextEncryptionAdapter.class),
+          Mockito.any(TracingContext.class)
+      );
+
+      FSDataOutputStream os1 = createMockedOutputStream(fs,
+          new Path("/test/file"), blobClient);
+      AbfsOutputStream out1 = (AbfsOutputStream) os1.getWrappedStream();
+      byte[] bytes1 = new byte[1024 * 1024 * 8];
+      new Random().nextBytes(bytes1);
+      out1.write(bytes1);
+
+      //parallel flush call should lead to the first call failing because of md5 mismatch.
+      Thread parallelFlushThread = new Thread(() -> {
+        try {
+          out1.hsync();
+        } catch (IOException e) {
+        }
+      });
+
+      parallelFlushThread.start(); // Start the parallel flush operation
+      parallelFlushThread.join();
+      // Perform the first flush operation
+      intercept(IOException.class,
+          "The condition specified using HTTP conditional header(s) is not met.",
+          out::hsync
+      );
     }
     }
   }
   }
 }
 }

+ 37 - 11
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java

@@ -18,12 +18,15 @@
 
 
 package org.apache.hadoop.fs.azurebfs;
 package org.apache.hadoop.fs.azurebfs;
 
 
+import java.nio.charset.StandardCharsets;
 import java.security.SecureRandom;
 import java.security.SecureRandom;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.HashSet;
 
 
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
@@ -34,10 +37,13 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobAppendRequestParameters;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
 
 
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE;
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -69,16 +75,19 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
   @Test
   @Test
   public void testAppendWithChecksumAtDifferentOffsets() throws Exception {
   public void testAppendWithChecksumAtDifferentOffsets() throws Exception {
     AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true);
     AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true);
-    AbfsClient client = fs.getAbfsStore().getClient();
+    if (!getIsNamespaceEnabled(fs)) {
+      Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    }
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     Path path = path("testPath" + getMethodName());
     Path path = path("testPath" + getMethodName());
-    fs.create(path);
+    AbfsOutputStream os = (AbfsOutputStream) fs.create(path).getWrappedStream();
     byte[] data = generateRandomBytes(MB_4);
     byte[] data = generateRandomBytes(MB_4);
     int pos = 0;
     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);
+    pos += appendWithOffsetHelper(os, client, path, data, fs, pos, 0);
+    pos += appendWithOffsetHelper(os, client, path, data, fs, pos, ONE_MB);
+    pos += appendWithOffsetHelper(os, client, path, data, fs, pos, MB_2);
+    appendWithOffsetHelper(os, client, path, data, fs, pos, MB_4 - 1);
     fs.close();
     fs.close();
   }
   }
 
 
@@ -107,16 +116,16 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
   @Test
   @Test
   public void testAbfsInvalidChecksumExceptionInAppend() throws Exception {
   public void testAbfsInvalidChecksumExceptionInAppend() throws Exception {
     AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true);
     AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true);
-    AbfsClient spiedClient = Mockito.spy(fs.getAbfsStore().getClient());
+    AbfsClient spiedClient = Mockito.spy(fs.getAbfsStore().getClientHandler().getIngressClient());
     Path path = path("testPath" + getMethodName());
     Path path = path("testPath" + getMethodName());
-    fs.create(path);
+    AbfsOutputStream os = (AbfsOutputStream) fs.create(path).getWrappedStream();
     byte[] data= generateRandomBytes(MB_4);
     byte[] data= generateRandomBytes(MB_4);
     String invalidMD5Hash = spiedClient.computeMD5Hash(
     String invalidMD5Hash = spiedClient.computeMD5Hash(
             INVALID_MD5_TEXT.getBytes(), 0, INVALID_MD5_TEXT.length());
             INVALID_MD5_TEXT.getBytes(), 0, INVALID_MD5_TEXT.length());
     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, 0);
+      appendWithOffsetHelper(os, spiedClient, path, data, fs, 0, 0);
     });
     });
 
 
     Assertions.assertThat(ex.getErrorCode())
     Assertions.assertThat(ex.getErrorCode())
@@ -163,6 +172,20 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
     }
     }
   }
   }
 
 
+  /**
+   * Helper method that generates blockId.
+   * @param position The offset needed to generate blockId.
+   * @return String representing the block ID generated.
+   */
+  private String generateBlockId(AbfsOutputStream os, long position) {
+    String streamId = os.getStreamID();
+    String streamIdHash = Integer.toString(streamId.hashCode());
+    String blockId = String.format("%d_%s", position, streamIdHash);
+    byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH];
+    System.arraycopy(blockId.getBytes(), 0, blockIdByteArray, 0, Math.min(BLOCK_ID_LENGTH, blockId.length()));
+    return new String(Base64.encodeBase64(blockIdByteArray), StandardCharsets.UTF_8);
+  }
+
   /**
   /**
    * Verify that the checksum computed on client side matches with the one
    * Verify that the checksum computed on client side matches with the one
    * computed at server side. If not, request will fail with 400 Bad request.
    * computed at server side. If not, request will fail with 400 Bad request.
@@ -173,10 +196,13 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
    * @param offset
    * @param offset
    * @throws Exception
    * @throws Exception
    */
    */
-  private int appendWithOffsetHelper(AbfsClient client, Path path,
+  private int appendWithOffsetHelper(AbfsOutputStream os, AbfsClient client, Path path,
       byte[] data, AzureBlobFileSystem fs, final int pos, final int offset) throws Exception {
       byte[] data, AzureBlobFileSystem fs, final int pos, final int offset) throws Exception {
+    String blockId = generateBlockId(os, pos);
+    String eTag = os.getIngressHandler().getETag();
     AppendRequestParameters reqParams = new AppendRequestParameters(
     AppendRequestParameters reqParams = new AppendRequestParameters(
-        pos, offset, data.length - offset, APPEND_MODE, isAppendBlobEnabled(), null, true);
+        pos, offset, data.length - offset, APPEND_MODE, isAppendBlobEnabled(), null, true,
+        new BlobAppendRequestParameters(blockId, eTag));
     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();
     return reqParams.getLength();

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

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

+ 915 - 16
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java

@@ -21,21 +21,34 @@ package org.apache.hadoop.fs.azurebfs;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.FilterOutputStream;
 import java.io.FilterOutputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.UUID;
 import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 
 
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
+import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClientHandler;
+import org.apache.hadoop.fs.azurebfs.utils.DirectoryStateHelper;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -49,6 +62,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.apache.hadoop.test.LambdaTestUtils;
 
 
 import static java.net.HttpURLConnection.HTTP_CONFLICT;
 import static java.net.HttpURLConnection.HTTP_CONFLICT;
 import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
 import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
@@ -56,6 +70,7 @@ import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
 import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
 
 
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_MKDIR_OVERWRITE;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.ArgumentMatchers.eq;
@@ -208,7 +223,7 @@ public class ITestAzureBlobFileSystemCreate extends
       out.write('2');
       out.write('2');
       out.hsync();
       out.hsync();
       fail("Expected a failure");
       fail("Expected a failure");
-    } catch (FileNotFoundException fnfe) {
+    } catch (IOException fnfe) {
       //appendblob outputStream does not generate suppressed exception on close as it is
       //appendblob outputStream does not generate suppressed exception on close as it is
       //single threaded code
       //single threaded code
       if (!fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testPath).toString())) {
       if (!fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testPath).toString())) {
@@ -233,10 +248,9 @@ public class ITestAzureBlobFileSystemCreate extends
   @Test
   @Test
   public void testFilterFSWriteAfterClose() throws Throwable {
   public void testFilterFSWriteAfterClose() throws Throwable {
     final AzureBlobFileSystem fs = getFileSystem();
     final AzureBlobFileSystem fs = getFileSystem();
-    Path testFolderPath = path(TEST_FOLDER_PATH);
-    Path testPath = new Path(testFolderPath, TEST_CHILD_FILE);
+    Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
     FSDataOutputStream out = fs.create(testPath);
     FSDataOutputStream out = fs.create(testPath);
-    intercept(FileNotFoundException.class,
+    intercept(IOException.class,
         () -> {
         () -> {
           try (FilterOutputStream fos = new FilterOutputStream(out)) {
           try (FilterOutputStream fos = new FilterOutputStream(out)) {
             byte[] bytes = new byte[8*ONE_MB];
             byte[] bytes = new byte[8*ONE_MB];
@@ -246,12 +260,12 @@ public class ITestAzureBlobFileSystemCreate extends
             out.hsync();
             out.hsync();
             fs.delete(testPath, false);
             fs.delete(testPath, false);
             // trigger the first failure
             // trigger the first failure
-            throw intercept(FileNotFoundException.class,
+            throw intercept(IOException.class,
                 () -> {
                 () -> {
-              fos.write('b');
-              out.hsync();
-              return "hsync didn't raise an IOE";
-            });
+                  fos.write('b');
+                  out.hsync();
+                  return "hsync didn't raise an IOE";
+                });
           }
           }
         });
         });
   }
   }
@@ -279,6 +293,8 @@ public class ITestAzureBlobFileSystemCreate extends
     Configuration config = new Configuration(this.getRawConfiguration());
     Configuration config = new Configuration(this.getRawConfiguration());
     config.set("fs.azure.enable.conditional.create.overwrite",
     config.set("fs.azure.enable.conditional.create.overwrite",
         Boolean.toString(enableConditionalCreateOverwrite));
         Boolean.toString(enableConditionalCreateOverwrite));
+    AzureBlobFileSystemStore store = currentFs.getAbfsStore();
+    AbfsClient client = store.getClientHandler().getIngressClient();
 
 
     final AzureBlobFileSystem fs =
     final AzureBlobFileSystem fs =
         (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(),
         (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(),
@@ -296,7 +312,11 @@ public class ITestAzureBlobFileSystemCreate extends
     fs.create(nonOverwriteFile, false);
     fs.create(nonOverwriteFile, false);
 
 
     // One request to server to create path should be issued
     // One request to server to create path should be issued
-    createRequestCount++;
+    // two calls added for -
+    // 1. getFileStatus on DFS endpoint : 1
+    //    getFileStatus on Blob endpoint: 2 (Additional List blob call)
+    // 2. actual create call: 1
+    createRequestCount += (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs) ? 3: 1);
 
 
     assertAbfsStatistics(
     assertAbfsStatistics(
         CONNECTIONS_MADE,
         CONNECTIONS_MADE,
@@ -312,7 +332,11 @@ public class ITestAzureBlobFileSystemCreate extends
     fs.registerListener(null);
     fs.registerListener(null);
 
 
     // One request to server to create path should be issued
     // One request to server to create path should be issued
-    createRequestCount++;
+    // Only single tryGetFileStatus should happen
+    // 1. getFileStatus on DFS endpoint : 1
+    //    getFileStatus on Blob endpoint: 1 (No Additional List blob call as file exists)
+
+    createRequestCount += (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs) ? 2: 1);
 
 
     assertAbfsStatistics(
     assertAbfsStatistics(
         CONNECTIONS_MADE,
         CONNECTIONS_MADE,
@@ -326,8 +350,12 @@ public class ITestAzureBlobFileSystemCreate extends
     // create should be successful
     // create should be successful
     fs.create(overwriteFilePath, true);
     fs.create(overwriteFilePath, true);
 
 
-    // One request to server to create path should be issued
-    createRequestCount++;
+    /// One request to server to create path should be issued
+    // two calls added for -
+    // 1. getFileStatus on DFS endpoint : 1
+    //    getFileStatus on Blob endpoint: 2 (Additional List blob call for non-existing path)
+    // 2. actual create call: 1
+    createRequestCount += (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs) ? 3: 1);
 
 
     assertAbfsStatistics(
     assertAbfsStatistics(
         CONNECTIONS_MADE,
         CONNECTIONS_MADE,
@@ -341,12 +369,15 @@ public class ITestAzureBlobFileSystemCreate extends
     fs.create(overwriteFilePath, true);
     fs.create(overwriteFilePath, true);
     fs.registerListener(null);
     fs.registerListener(null);
 
 
+    createRequestCount += (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs) ? 1: 0);
+
+    // Second actual create call will hap
     if (enableConditionalCreateOverwrite) {
     if (enableConditionalCreateOverwrite) {
       // Three requests will be sent to server to create path,
       // Three requests will be sent to server to create path,
       // 1. create without overwrite
       // 1. create without overwrite
       // 2. GetFileStatus to get eTag
       // 2. GetFileStatus to get eTag
       // 3. create with overwrite
       // 3. create with overwrite
-      createRequestCount += 3;
+      createRequestCount += (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs) ? 4: 3);
     } else {
     } else {
       createRequestCount++;
       createRequestCount++;
     }
     }
@@ -393,11 +424,14 @@ public class ITestAzureBlobFileSystemCreate extends
         = ITestAbfsClient.getMockAbfsClient(
         = ITestAbfsClient.getMockAbfsClient(
         fs.getAbfsStore().getClient(),
         fs.getAbfsStore().getClient(),
         fs.getAbfsStore().getAbfsConfiguration());
         fs.getAbfsStore().getAbfsConfiguration());
+    AbfsClientHandler clientHandler = Mockito.mock(AbfsClientHandler.class);
+    when(clientHandler.getIngressClient()).thenReturn(mockClient);
+    when(clientHandler.getClient(Mockito.any())).thenReturn(mockClient);
 
 
     AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
     AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
 
 
+    ReflectionUtils.setFinalField(AzureBlobFileSystemStore.class, abfsStore, "clientHandler", clientHandler);
     ReflectionUtils.setFinalField(AzureBlobFileSystemStore.class, abfsStore, "client", mockClient);
     ReflectionUtils.setFinalField(AzureBlobFileSystemStore.class, abfsStore, "client", mockClient);
-
     boolean isNamespaceEnabled = abfsStore
     boolean isNamespaceEnabled = abfsStore
         .getIsNamespaceEnabled(getTestTracingContext(fs, false));
         .getIsNamespaceEnabled(getTestTracingContext(fs, false));
 
 
@@ -494,7 +528,7 @@ public class ITestAzureBlobFileSystemCreate extends
         FsAction.ALL);
         FsAction.ALL);
     FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
     FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
         FsAction.NONE);
         FsAction.NONE);
-    Path testPath = new Path("testFile");
+    Path testPath = new Path("/testFile");
     intercept(
     intercept(
         exceptionClass,
         exceptionClass,
         () -> abfsStore.createFile(testPath, null, true, permission, umask,
         () -> abfsStore.createFile(testPath, null, true, permission, umask,
@@ -504,4 +538,869 @@ public class ITestAzureBlobFileSystemCreate extends
   private AbfsRestOperationException getMockAbfsRestOperationException(int status) {
   private AbfsRestOperationException getMockAbfsRestOperationException(int status) {
     return new AbfsRestOperationException(status, "", "", new Exception());
     return new AbfsRestOperationException(status, "", "", new Exception());
   }
   }
+
+
+  /**
+   * Attempts to test multiple flush calls.
+   */
+  @Test
+  public void testMultipleFlush() throws Throwable {
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
+    try (FSDataOutputStream out = fs.create(testPath)) {
+      out.write('1');
+      out.hsync();
+      out.write('2');
+      out.hsync();
+    }
+  }
+
+  /**
+   * Delete the blob before flush and verify that an exception should be thrown.
+   */
+  @Test
+  public void testDeleteBeforeFlush() throws Throwable {
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
+    try (FSDataOutputStream out = fs.create(testPath)) {
+      out.write('1');
+      fs.delete(testPath, false);
+      out.hsync();
+      // this will cause the next write to failAll
+    } catch (IOException fnfe) {
+      //appendblob outputStream does not generate suppressed exception on close as it is
+      //single threaded code
+      if (!fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testPath).toString())) {
+        // the exception raised in close() must be in the caught exception's
+        // suppressed list
+        Throwable[] suppressed = fnfe.getSuppressed();
+        assertEquals("suppressed count", 1, suppressed.length);
+        Throwable inner = suppressed[0];
+        if (!(inner instanceof IOException)) {
+          throw inner;
+        }
+        GenericTestUtils.assertExceptionContains(fnfe.getMessage(), inner.getCause(), inner.getCause().getMessage());
+      }
+    }
+  }
+
+  /**
+   * Creating subdirectory on existing file path should fail.
+   * @throws Exception
+   */
+  @Test
+  public void testMkdirsFailsForSubdirectoryOfExistingFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.create(new Path("a/b/c"));
+    fs.mkdirs(new Path("a/b/d"));
+    intercept(IOException.class, () -> fs.mkdirs(new Path("a/b/c/d/e")));
+
+    Assertions.assertThat(fs.exists(new Path("a/b/c"))).isTrue();
+    Assertions.assertThat(fs.exists(new Path("a/b/d"))).isTrue();
+    // Asserting directory created still exists as explicit.
+    Assertions.assertThat(
+            DirectoryStateHelper.isExplicitDirectory(new Path("a/b/d"), fs,
+                getTestTracingContext(fs, true)))
+        .describedAs("Path is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Try creating file same as an existing directory.
+   * @throws Exception
+   */
+  @Test
+  public void testCreateDirectoryAndFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a/b/c"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c"))).isTrue();
+    intercept(IOException.class, () -> fs.create(new Path("a/b/c")));
+    // Asserting that directory still exists as explicit
+    Assertions.assertThat(
+            DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"),
+                fs, getTestTracingContext(fs, true)))
+        .describedAs("Path is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Creating same file without specifying overwrite.
+   * @throws Exception
+   */
+  @Test
+  public void testCreateSameFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.create(new Path("a/b/c"));
+    fs.create(new Path("a/b/c"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Path does not exist")
+        .isTrue();
+  }
+
+  /**
+   * Creating same file with overwrite flag set to false.
+   * @throws Exception
+   */
+  @Test
+  public void testCreateSameFileWithOverwriteFalse() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.create(new Path("a/b/c"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Path does not exist")
+        .isTrue();
+    intercept(IOException.class, () -> fs.create(new Path("a/b/c"), false));
+  }
+
+  /**
+   * Creation of already existing subpath should fail.
+   * @throws Exception
+   */
+  @Test
+  public void testCreateSubPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.create(new Path("a/b/c"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Path does not exist")
+        .isTrue();
+    intercept(IOException.class, () -> fs.create(new Path("a/b")));
+  }
+
+  /**
+   * Creating path with parent explicit.
+   */
+  @Test
+  public void testCreatePathParentExplicit() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a/b/c"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Path does not exist")
+        .isTrue();
+    fs.create(new Path("a/b/c/d"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c/d")))
+        .describedAs("Path does not exist")
+        .isTrue();
+
+    // asserting that parent stays explicit
+    Assertions.assertThat(
+            DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"),
+                fs, getTestTracingContext(fs, true)))
+        .describedAs("Path is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Test create on implicit directory with explicit parent.
+   * @throws Exception
+   */
+  @Test
+  public void testParentExplicitPathImplicit() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB);
+    fs.mkdirs(new Path("/explicitParent"));
+    String sourcePathName = "/explicitParent/implicitDir";
+    Path sourcePath = new Path(sourcePathName);
+    createAzCopyFolder(sourcePath);
+
+    intercept(IOException.class, () ->
+        fs.create(sourcePath, true));
+    intercept(IOException.class, () ->
+        fs.create(sourcePath, false));
+
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(sourcePath.getParent(), fs, getTestTracingContext(fs, true)))
+        .describedAs("Parent directory should be explicit.")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(sourcePath, fs, getTestTracingContext(fs, true)))
+        .describedAs("Path should be implicit.")
+        .isTrue();
+  }
+
+  /**
+   * Test create on implicit directory with implicit parent
+   * @throws Exception
+   */
+  @Test
+  public void testParentImplicitPathImplicit() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.BLOB);
+    String parentPathName = "/implicitParent";
+    Path parentPath = new Path(parentPathName);
+    String sourcePathName = "/implicitParent/implicitDir";
+    Path sourcePath = new Path(sourcePathName);
+
+    createAzCopyFolder(parentPath);
+    createAzCopyFolder(sourcePath);
+
+    intercept(IOException.class, () ->
+        fs.create(sourcePath, true));
+    intercept(IOException.class, () ->
+        fs.create(sourcePath, false));
+
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(parentPath, fs, getTestTracingContext(fs, true)))
+        .describedAs("Parent directory is implicit.")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(sourcePath, fs, getTestTracingContext(fs, true)))
+        .describedAs("Path should also be implicit.")
+        .isTrue();
+  }
+
+  /**
+   * Tests create file when file exists already
+   * Verifies using eTag for overwrite = true/false
+   */
+  @Test
+  public void testCreateFileExistsImplicitParent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final AzureBlobFileSystemStore store = fs.getAbfsStore();
+    String parentPathName = "/implicitParent";
+    Path parentPath = new Path(parentPathName);
+    createAzCopyFolder(parentPath);
+
+    String fileName = "/implicitParent/testFile";
+    Path filePath = new Path(fileName);
+    fs.create(filePath);
+    String eTag = extractFileEtag(fileName);
+
+    // testing createFile on already existing file path
+    fs.create(filePath, true);
+
+    String eTagAfterCreateOverwrite = extractFileEtag(fileName);
+
+    Assertions.assertThat(eTag.equals(eTagAfterCreateOverwrite))
+        .describedAs("New file eTag after create overwrite should be different from old")
+        .isFalse();
+
+    intercept(IOException.class, () ->
+        fs.create(filePath, false));
+
+    String eTagAfterCreate = extractFileEtag(fileName);
+
+    Assertions.assertThat(eTagAfterCreateOverwrite.equals(eTagAfterCreate))
+        .describedAs("File eTag should not change as creation fails")
+        .isTrue();
+
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(parentPath, fs, getTestTracingContext(fs, true)))
+        .describedAs("Parent path should also change to explicit.")
+        .isTrue();
+  }
+
+  /**
+   * Tests create file when the parent is an existing file
+   * should fail.
+   * @throws Exception FileAlreadyExists for blob and IOException for dfs.
+   */
+  @Test
+  public void testCreateFileParentFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final AzureBlobFileSystemStore store = fs.getAbfsStore();
+
+    String parentName = "/testParentFile";
+    Path parent = new Path(parentName);
+    fs.create(parent);
+
+    String childName = "/testParentFile/testChildFile";
+    Path child = new Path(childName);
+    IOException e = intercept(IOException.class, () ->
+        fs.create(child, false));
+
+    // asserting that parent stays explicit
+    FileStatus status = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path(parentName)),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path is not a file")
+        .isFalse();
+  }
+
+  /**
+   * Creating directory on existing file path should fail.
+   * @throws Exception
+   */
+  @Test
+  public void testCreateMkdirs() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.create(new Path("a/b/c"));
+    intercept(IOException.class, () -> fs.mkdirs(new Path("a/b/c/d")));
+  }
+
+  /**
+   * Test mkdirs.
+   * @throws Exception
+   */
+  @Test
+  public void testMkdirs() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a/b"));
+    fs.mkdirs(new Path("a/b/c/d"));
+    fs.mkdirs(new Path("a/b/c/e"));
+
+    Assertions.assertThat(fs.exists(new Path("a/b")))
+        .describedAs("Path a/b does not exist")
+        .isTrue();
+    Assertions.assertThat(fs.exists(new Path("a/b/c/d")))
+        .describedAs("Path a/b/c/d does not exist")
+        .isTrue();
+    Assertions.assertThat(fs.exists(new Path("a/b/c/e")))
+        .describedAs("Path a/b/c/e does not exist")
+        .isTrue();
+
+    // Asserting that directories created as explicit
+    FileStatus status = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a/b")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path a/b is not an explicit directory")
+        .isTrue();
+    FileStatus status1 = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a/b/c/d")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status1.isDirectory())
+        .describedAs("Path a/b/c/d is not an explicit directory")
+        .isTrue();
+    FileStatus status2 = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a/b/c/e")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status2.isDirectory())
+        .describedAs("Path a/b/c/e is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Creating subpath of directory path should fail.
+   * @throws Exception
+   */
+  @Test
+  public void testMkdirsCreateSubPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a/b/c"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Path a/b/c does not exist")
+        .isTrue();
+    intercept(IOException.class, () -> fs.create(new Path("a/b")));
+
+    // Asserting that directories created as explicit
+    FileStatus status2 = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a/b/c")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status2.isDirectory())
+        .describedAs("Path a/b/c is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Test creation of directory by level.
+   * @throws Exception
+   */
+  @Test
+  public void testMkdirsByLevel() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a"));
+    fs.mkdirs(new Path("a/b/c"));
+    fs.mkdirs(new Path("a/b/c/d/e"));
+
+    Assertions.assertThat(fs.exists(new Path("a")))
+        .describedAs("Path a does not exist")
+        .isTrue();
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Path a/b/c does not exist")
+        .isTrue();
+    Assertions.assertThat(fs.exists(new Path("a/b/c/d/e")))
+        .describedAs("Path a/b/c/d/e does not exist")
+        .isTrue();
+
+    // Asserting that directories created as explicit
+    FileStatus status = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a/")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path a is not an explicit directory")
+        .isTrue();
+    FileStatus status1 = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a/b/c")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status1.isDirectory())
+        .describedAs("Path a/b/c is not an explicit directory")
+        .isTrue();
+    FileStatus status2 = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a/b/c/d/e")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status2.isDirectory())
+        .describedAs("Path a/b/c/d/e is not an explicit directory")
+        .isTrue();
+  }
+
+  /*
+    Delete part of a path and validate sub path exists.
+   */
+  @Test
+  public void testMkdirsWithDelete() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a/b"));
+    fs.mkdirs(new Path("a/b/c/d"));
+    fs.delete(new Path("a/b/c/d"));
+    fs.getFileStatus(new Path("a/b/c"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Path a/b/c does not exist")
+        .isTrue();
+  }
+
+  /**
+   * Verify mkdir and rename of parent.
+   */
+  @Test
+  public void testMkdirsWithRename() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a/b/c/d"));
+    fs.create(new Path("e/file"));
+    fs.delete(new Path("a/b/c/d"));
+    Assertions.assertThat(fs.rename(new Path("e"), new Path("a/b/c/d")))
+        .describedAs("Failed to rename path e to a/b/c/d")
+        .isTrue();
+    Assertions.assertThat(fs.exists(new Path("a/b/c/d/file")))
+        .describedAs("Path a/b/c/d/file does not exist")
+        .isTrue();
+  }
+
+  /**
+   * Create a file with name /dir1 and then mkdirs for /dir1/dir2 should fail.
+   * @throws Exception
+   */
+  @Test
+  public void testFileCreateMkdirsRoot() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.setWorkingDirectory(new Path("/"));
+    final Path p1 = new Path("dir1");
+    fs.create(p1);
+    intercept(IOException.class, () -> fs.mkdirs(new Path("dir1/dir2")));
+  }
+
+  /**
+   * Create a file with name /dir1 and then mkdirs for /dir1/dir2 should fail.
+   * @throws Exception
+   */
+  @Test
+  public void testFileCreateMkdirsNonRoot() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path p1 = new Path("dir1");
+    fs.create(p1);
+    intercept(IOException.class, () -> fs.mkdirs(new Path("dir1/dir2")));
+  }
+
+  /**
+   * Creation of same directory without overwrite flag should pass.
+   * @throws Exception
+   */
+  @Test
+  public void testCreateSameDirectory() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a/b/c"));
+    fs.mkdirs(new Path("a/b/c"));
+
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Path a/b/c does not exist")
+        .isTrue();
+    // Asserting that directories created as explicit
+    FileStatus status = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a/b/c")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path a/b/c is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Creation of same directory without overwrite flag should pass.
+   * @throws Exception
+   */
+  @Test
+  public void testCreateSamePathDirectory() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.create(new Path("a"));
+    intercept(IOException.class, () -> fs.mkdirs(new Path("a")));
+  }
+
+  /**
+   * Creation of directory with root as parent
+   */
+  @Test
+  public void testMkdirOnRootAsParent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path path = new Path("a");
+    fs.setWorkingDirectory(new Path("/"));
+    fs.mkdirs(path);
+
+    // Asserting that the directory created by mkdir exists as explicit.
+    FileStatus status = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("a")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path a is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Creation of directory on root
+   */
+  @Test
+  public void testMkdirOnRoot() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path path = new Path("/");
+    fs.setWorkingDirectory(new Path("/"));
+    fs.mkdirs(path);
+
+    FileStatus status = fs.getAbfsStore().getFileStatus(fs.makeQualified(new Path("/")),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Creation of directory on path with unicode chars
+   */
+  @Test
+  public void testMkdirUnicode() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path path = new Path("/dir\u0031");
+    fs.mkdirs(path);
+
+    // Asserting that the directory created by mkdir exists as explicit.
+    FileStatus status = fs.getAbfsStore().getFileStatus(fs.makeQualified(path),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Creation of directory on same path with parallel threads.
+   */
+  @Test
+  public void testMkdirParallelRequests() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path path = new Path("/dir1");
+
+    ExecutorService es = Executors.newFixedThreadPool(3);
+
+    List<CompletableFuture<Void>> tasks = new ArrayList<>();
+
+    for (int i = 0; i < 3; i++) {
+      CompletableFuture<Void> future = CompletableFuture.runAsync(() -> {
+        try {
+          fs.mkdirs(path);
+        } catch (IOException e) {
+          throw new CompletionException(e);
+        }
+      }, es);
+      tasks.add(future);
+    }
+
+    // Wait for all the tasks to complete
+    CompletableFuture.allOf(tasks.toArray(new CompletableFuture[0])).join();
+
+    // Assert that the directory created by mkdir exists as explicit
+    FileStatus status = fs.getAbfsStore().getFileStatus(fs.makeQualified(path),
+        new TracingContext(getTestTracingContext(fs, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path is not an explicit directory")
+        .isTrue();
+  }
+
+
+  /**
+   * Creation of directory with overwrite set to false should not fail according to DFS code.
+   * @throws Exception
+   */
+  @Test
+  public void testCreateSameDirectoryOverwriteFalse() throws Exception {
+    Configuration configuration = getRawConfiguration();
+    configuration.setBoolean(FS_AZURE_ENABLE_MKDIR_OVERWRITE, false);
+    AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
+    fs1.mkdirs(new Path("a/b/c"));
+    fs1.mkdirs(new Path("a/b/c"));
+
+    // Asserting that directories created as explicit
+    FileStatus status = fs1.getAbfsStore().getFileStatus(fs1.makeQualified(new Path("a/b/c")),
+        new TracingContext(getTestTracingContext(fs1, true)));
+    Assertions.assertThat(status.isDirectory())
+        .describedAs("Path is not an explicit directory")
+        .isTrue();
+  }
+
+  /**
+   * Try creating directory same as an existing file.
+   */
+  @Test
+  public void testCreateDirectoryAndFileRecreation() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("a/b/c"));
+    fs.create(new Path("a/b/c/d"));
+    Assertions.assertThat(fs.exists(new Path("a/b/c")))
+        .describedAs("Directory a/b/c does not exist")
+        .isTrue();
+    Assertions.assertThat(fs.exists(new Path("a/b/c/d")))
+        .describedAs("File a/b/c/d does not exist")
+        .isTrue();
+    intercept(IOException.class, () -> fs.mkdirs(new Path("a/b/c/d")));
+  }
+
+  @Test
+  public void testCreateNonRecursiveForAtomicDirectoryFile() throws Exception {
+    AzureBlobFileSystem fileSystem = getFileSystem();
+    fileSystem.setWorkingDirectory(new Path("/"));
+    fileSystem.mkdirs(new Path("/hbase/dir"));
+    fileSystem.createFile(new Path("/hbase/dir/file"))
+        .overwrite(false)
+        .replication((short) 1)
+        .bufferSize(1024)
+        .blockSize(1024)
+        .build();
+    Assertions.assertThat(fileSystem.exists(new Path("/hbase/dir/file")))
+        .describedAs("File /hbase/dir/file does not exist")
+        .isTrue();
+  }
+
+  @Test
+  public void testMkdirOnNonExistingPathWithImplicitParentDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path implicitPath = new Path("dir1");
+    final Path path = new Path("dir1/dir2");
+    createAzCopyFolder(implicitPath);
+
+    // Creating a directory on non-existing path inside an implicit directory
+    fs.mkdirs(path);
+
+    // Asserting that path created by azcopy becomes explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(implicitPath,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Path created by azcopy did not become explicit")
+        .isTrue();
+
+    // Asserting that the directory created by mkdir exists as explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(path,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory created by mkdir does not exist as explicit")
+        .isTrue();
+  }
+
+  /**
+   * Creation of directory with parent directory existing as implicit.
+   * And the directory to be created existing as explicit directory
+   * @throws Exception
+   */
+  @Test
+  public void testMkdirOnExistingExplicitDirWithImplicitParentDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path implicitPath = new Path("dir1");
+    final Path path = new Path("dir1/dir2");
+
+    // Creating implicit directory to be used as parent
+    createAzCopyFolder(implicitPath);
+
+    // Creating an explicit directory on the path first
+    fs.mkdirs(path);
+
+    // Creating a directory on existing explicit directory inside an implicit directory
+    fs.mkdirs(path);
+
+    // Asserting that path created by azcopy becomes explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(implicitPath,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Path created by azcopy did not become explicit")
+        .isTrue();
+
+    // Asserting that the directory created by mkdir exists as explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(path,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory created by mkdir does not exist as explicit")
+        .isTrue();
+  }
+
+  /**
+   * Creation of directory with parent directory existing as explicit.
+   * And the directory to be created existing as implicit directory
+   * @throws Exception
+   */
+  @Test
+  public void testMkdirOnExistingImplicitDirWithExplicitParentDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path explicitPath = new Path("dir1");
+    final Path path = new Path("dir1/dir2");
+
+    // Creating an explicit directory to be used a parent
+    fs.mkdirs(explicitPath);
+
+    createAzCopyFolder(path);
+
+    // Creating a directory on existing implicit directory inside an explicit directory
+    fs.mkdirs(path);
+
+    // Asserting that the directory created by mkdir exists as explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(explicitPath,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Explicit parent directory does not exist as explicit")
+        .isTrue();
+
+    // Asserting that the directory created by mkdir exists as explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(path,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory created by mkdir does not exist as explicit")
+        .isTrue();
+  }
+
+  /**
+   * Creation of directory with parent directory existing as implicit.
+   * And the directory to be created existing as implicit directory
+   * @throws Exception
+   */
+  @Test
+  public void testMkdirOnExistingImplicitDirWithImplicitParentDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path implicitPath = new Path("dir3");
+    final Path path = new Path("dir3/dir4");
+
+    createAzCopyFolder(implicitPath);
+
+    // Creating an implicit directory on path
+    createAzCopyFolder(path);
+
+    // Creating a directory on existing implicit directory inside an implicit directory
+    fs.mkdirs(path);
+
+    // Asserting that path created by azcopy becomes explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(implicitPath,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Path created by azcopy did not become explicit")
+        .isTrue();
+
+    // Asserting that the directory created by mkdir exists as explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(path,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory created by mkdir does not exist as explicit")
+        .isTrue();
+  }
+
+  /**
+   * Creation of directory with parent directory existing as implicit.
+   * And the directory to be created existing as file
+   * @throws Exception
+   */
+  @Test
+  public void testMkdirOnExistingFileWithImplicitParentDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final Path implicitPath = new Path("dir1");
+    final Path path = new Path("dir1/dir2");
+
+    createAzCopyFolder(implicitPath);
+
+    // Creating a file on path
+    fs.create(path);
+
+    // Creating a directory on existing file inside an implicit directory
+    // Asserting that the mkdir fails
+    LambdaTestUtils.intercept(FileAlreadyExistsException.class, () -> {
+      fs.mkdirs(path);
+    });
+
+    // Asserting that path created by azcopy becomes explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(implicitPath,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Path created by azcopy did not become explicit")
+        .isTrue();
+
+    // Asserting that the file still exists at path.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(path,
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("File still exists at path")
+        .isFalse();
+  }
+
+  /**
+   * 1. a/b/c as implicit.
+   * 2. Create marker for b.
+   * 3. Do mkdir on a/b/c/d.
+   * 4. Verify all b,c,d have marker but a is implicit.
+   */
+  @Test
+  public void testImplicitExplicitFolder() throws Exception {
+    Configuration configuration = Mockito.spy(getRawConfiguration());
+    final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
+    final Path implicitPath = new Path("a/b/c");
+
+    createAzCopyFolder(implicitPath);
+
+    Path path = makeQualified(new Path("a/b"));
+    AbfsBlobClient blobClient = (AbfsBlobClient) fs.getAbfsStore().getClient(AbfsServiceType.BLOB);
+    blobClient.createPath(path.toUri().getPath(), false, true,
+        null, false, null, null, getTestTracingContext(fs, true),
+        true);
+
+    fs.mkdirs(new Path("a/b/c/d"));
+
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(new Path("a"),
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory 'a' should be implicit")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(new Path("a/b"),
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory 'a/b' should be explicit")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"),
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory 'a/b/c' should be explicit")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/d"),
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory 'a/b/c/d' should be explicit")
+        .isTrue();
+  }
+
+  /**
+   * 1. a/b/c implicit.
+   * 2. Marker for a and c.
+   * 3. mkdir on a/b/c/d.
+   * 4. Verify a,c,d are explicit but b is implicit.
+   */
+  @Test
+  public void testImplicitExplicitFolder1() throws Exception {
+    Configuration configuration = Mockito.spy(getRawConfiguration());
+    final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
+    final Path implicitPath = new Path("a/b/c");
+
+    createAzCopyFolder(implicitPath);
+
+    Path path = makeQualified(new Path("a"));
+    AbfsBlobClient blobClient = (AbfsBlobClient) fs.getAbfsStore().getClient(AbfsServiceType.BLOB);
+    blobClient.createPath(path.toUri().getPath(), false, true,
+        null, false, null, null, getTestTracingContext(fs, true), true);
+
+    Path newPath = makeQualified(new Path("a/b/c"));
+    blobClient.createPath(newPath.toUri().getPath(), false, true,
+        null, false, null, null, getTestTracingContext(fs, true), true);
+
+    fs.mkdirs(new Path("a/b/c/d"));
+
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(new Path("a/b"),
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory 'a/b' should be implicit")
+        .isTrue();
+
+    // Asserting that the directory created by mkdir exists as explicit.
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(new Path("a"),
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory 'a' should be explicit")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"),
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory 'a/b/c' should be explicit")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/d"),
+            fs, getTestTracingContext(fs, true)))
+        .describedAs("Directory 'a/b/c/d' should be explicit")
+        .isTrue();
+  }
+
+  /**
+   * Extracts the eTag for an existing file
+   * @param fileName file Path in String from container root
+   * @return String etag for the file
+   * @throws IOException
+   */
+  private String extractFileEtag(String fileName) throws IOException {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final AbfsClient client = fs.getAbfsClient();
+    final TracingContext testTracingContext = getTestTracingContext(fs, false);
+    AbfsRestOperation op;
+    op = client.getPathStatus(fileName, true, testTracingContext, null);
+    return AzureBlobFileSystemStore.extractEtagHeader(op.getResult());
+  }
 }
 }

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

@@ -24,6 +24,8 @@ import java.util.Arrays;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
 import org.junit.Test;
 import org.junit.Test;
@@ -200,6 +202,7 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
   public void testWriteWithFileNotFoundException() throws Exception {
   public void testWriteWithFileNotFoundException() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();
     final AzureBlobFileSystem fs = getFileSystem();
     final Path testFilePath = path(methodName.getMethodName());
     final Path testFilePath = path(methodName.getMethodName());
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
 
 
     try (FSDataOutputStream stream = fs.create(testFilePath)) {
     try (FSDataOutputStream stream = fs.create(testFilePath)) {
       assertPathExists(fs, "Path should exist", testFilePath);
       assertPathExists(fs, "Path should exist", testFilePath);
@@ -209,7 +212,11 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
       assertPathDoesNotExist(fs, "This path should not exist", testFilePath);
       assertPathDoesNotExist(fs, "This path should not exist", testFilePath);
 
 
       // trigger append call
       // trigger append call
-      intercept(FileNotFoundException.class, () -> stream.close());
+      if (client instanceof AbfsDfsClient) {
+        intercept(FileNotFoundException.class, stream::close);
+      } else {
+        intercept(IOException.class, stream::close);
+      }
     }
     }
   }
   }
 
 
@@ -217,6 +224,7 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
   public void testFlushWithFileNotFoundException() throws Exception {
   public void testFlushWithFileNotFoundException() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();
     final AzureBlobFileSystem fs = getFileSystem();
     final Path testFilePath = path(methodName.getMethodName());
     final Path testFilePath = path(methodName.getMethodName());
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testFilePath).toString())) {
     if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testFilePath).toString())) {
       return;
       return;
     }
     }
@@ -227,7 +235,11 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
       fs.delete(testFilePath, true);
       fs.delete(testFilePath, true);
       assertPathDoesNotExist(fs, "This path should not exist", testFilePath);
       assertPathDoesNotExist(fs, "This path should not exist", testFilePath);
 
 
-      intercept(FileNotFoundException.class, () -> stream.close());
+      if (client instanceof AbfsDfsClient) {
+        intercept(FileNotFoundException.class, stream::close);
+      } else {
+        intercept(IOException.class, stream::close);
+      }
     }
     }
   }
   }
 
 

+ 7 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java

@@ -33,10 +33,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
 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.services.AbfsOutputStream;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.hamcrest.core.IsEqual;
 import org.hamcrest.core.IsEqual;
 import org.hamcrest.core.IsNot;
 import org.hamcrest.core.IsNot;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -308,6 +310,11 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
 
 
   @Test
   @Test
   public void testTracingHeaderForAppendBlob() throws Exception {
   public void testTracingHeaderForAppendBlob() throws Exception {
+    AzureBlobFileSystem fs1 = getFileSystem();
+    // AppendBlob not supported for FNS over DFS endpoint.
+    if (!getIsNamespaceEnabled(fs1)) {
+      Assume.assumeTrue(getAbfsStore(fs1).getClient() instanceof AbfsBlobClient);
+    }
     Configuration config = new Configuration(this.getRawConfiguration());
     Configuration config = new Configuration(this.getRawConfiguration());
     config.set(FS_AZURE_APPEND_BLOB_KEY, "abfss:/");
     config.set(FS_AZURE_APPEND_BLOB_KEY, "abfss:/");
     config.set(TestConfigurationKeys.FS_AZURE_TEST_APPENDBLOB_ENABLED, "true");
     config.set(TestConfigurationKeys.FS_AZURE_TEST_APPENDBLOB_ENABLED, "true");

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

@@ -78,7 +78,7 @@ public class ITestAzureBlobFileSystemInitAndCreate extends
     AzureBlobFileSystem fs = ((AzureBlobFileSystem) FileSystem.newInstance(
     AzureBlobFileSystem fs = ((AzureBlobFileSystem) FileSystem.newInstance(
         getRawConfiguration()));
         getRawConfiguration()));
     AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
     AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
-    AbfsClient client = Mockito.spy(fs.getAbfsClient());
+    AbfsClient client = Mockito.spy(fs.getAbfsStore().getClient(AbfsServiceType.DFS));
     Mockito.doReturn(client).when(store).getClient(AbfsServiceType.DFS);
     Mockito.doReturn(client).when(store).getClient(AbfsServiceType.DFS);
 
 
     Mockito.doThrow(TrileanConversionException.class)
     Mockito.doThrow(TrileanConversionException.class)

+ 34 - 7
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsLease;
 import org.apache.hadoop.fs.azurebfs.services.AbfsLease;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
@@ -41,6 +42,9 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 
 
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.CONDITION_NOT_MET;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_EXPIRED_BLOB;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_ID_SPECIFIED_BLOB;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.anyString;
@@ -139,12 +143,15 @@ 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);
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT);
     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)) {
       LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_PARALLEL_ACCESS_DETECTED
       LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_PARALLEL_ACCESS_DETECTED
-          : ERR_NO_LEASE_ID_SPECIFIED, () -> {
+          : client instanceof AbfsBlobClient
+              ? ERR_NO_LEASE_ID_SPECIFIED_BLOB
+              : ERR_NO_LEASE_ID_SPECIFIED, () -> {
         try (FSDataOutputStream out2 = fs.create(testFilePath)) {
         try (FSDataOutputStream out2 = fs.create(testFilePath)) {
         }
         }
         return "Expected second create on infinite lease dir to fail";
         return "Expected second create on infinite lease dir to fail";
@@ -154,6 +161,7 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   }
   }
 
 
   private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expectException) throws Exception {
   private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expectException) throws Exception {
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     try (FSDataOutputStream out = fs.create(testFilePath)) {
     try (FSDataOutputStream out = fs.create(testFilePath)) {
       try (FSDataOutputStream out2 = fs.append(testFilePath)) {
       try (FSDataOutputStream out2 = fs.append(testFilePath)) {
         out2.writeInt(2);
         out2.writeInt(2);
@@ -166,7 +174,23 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
         }
         }
       }
       }
       out.writeInt(1);
       out.writeInt(1);
-      out.hsync();
+      try {
+        out.hsync();
+      } catch (IOException e) {
+        // Etag mismatch leads to condition not met error for blob endpoint.
+        if (client instanceof AbfsBlobClient) {
+          GenericTestUtils.assertExceptionContains(CONDITION_NOT_MET, e);
+        } else {
+          throw e;
+        }
+      }
+    } catch (IOException e) {
+      // Etag mismatch leads to condition not met error for blob endpoint.
+      if (client instanceof AbfsBlobClient) {
+        GenericTestUtils.assertExceptionContains(CONDITION_NOT_MET, e);
+      } else {
+        throw e;
+      }
     }
     }
 
 
     Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed());
     Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed());
@@ -213,6 +237,7 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   public void testWriteAfterBreakLease() throws Exception {
   public void testWriteAfterBreakLease() 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);
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     fs.mkdirs(testFilePath.getParent());
     fs.mkdirs(testFilePath.getParent());
 
 
     FSDataOutputStream out;
     FSDataOutputStream out;
@@ -225,14 +250,15 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
         FSOperationType.BREAK_LEASE, false, 0));
         FSOperationType.BREAK_LEASE, false, 0));
     fs.breakLease(testFilePath);
     fs.breakLease(testFilePath);
     fs.registerListener(null);
     fs.registerListener(null);
-
-    LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> {
+    LambdaTestUtils.intercept(IOException.class, client instanceof AbfsBlobClient
+        ? ERR_LEASE_EXPIRED_BLOB : ERR_LEASE_EXPIRED, () -> {
       out.write(1);
       out.write(1);
       out.hsync();
       out.hsync();
       return "Expected exception on write after lease break but got " + out;
       return "Expected exception on write after lease break but got " + out;
     });
     });
 
 
-    LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> {
+    LambdaTestUtils.intercept(IOException.class, client instanceof AbfsBlobClient
+        ? ERR_LEASE_EXPIRED_BLOB : ERR_LEASE_EXPIRED, () -> {
       out.close();
       out.close();
       return "Expected exception on close after lease break but got " + out;
       return "Expected exception on close after lease break but got " + out;
     });
     });
@@ -252,14 +278,15 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   public void testLeaseFreedAfterBreak() throws Exception {
   public void testLeaseFreedAfterBreak() 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);
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     fs.mkdirs(testFilePath.getParent());
     fs.mkdirs(testFilePath.getParent());
 
 
     FSDataOutputStream out = fs.create(testFilePath);
     FSDataOutputStream out = fs.create(testFilePath);
     out.write(0);
     out.write(0);
 
 
     fs.breakLease(testFilePath);
     fs.breakLease(testFilePath);
-
-    LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> {
+    LambdaTestUtils.intercept(IOException.class, client instanceof AbfsBlobClient
+        ? ERR_LEASE_EXPIRED_BLOB : ERR_LEASE_EXPIRED, () -> {
       out.close();
       out.close();
       return "Expected exception on close after lease break but got " + out;
       return "Expected exception on close after lease break but got " + out;
     });
     });

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

@@ -22,11 +22,14 @@ import java.util.UUID;
 
 
 import org.junit.Assume;
 import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 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.services.AbfsBlobClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 
 
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_MKDIR_OVERWRITE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_MKDIR_OVERWRITE;
@@ -123,7 +126,13 @@ public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
     fs.mkdirs(dirPath);
     fs.mkdirs(dirPath);
 
 
     // One request to server
     // One request to server
-    mkdirRequestCount++;
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
+    if (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs)) {
+      // 1 GetBlobProperties + 1 ListBlobs + 1 PutBlob call.
+      mkdirRequestCount +=3;
+    } else {
+      mkdirRequestCount++;
+    }
 
 
     assertAbfsStatistics(
     assertAbfsStatistics(
         CONNECTIONS_MADE,
         CONNECTIONS_MADE,
@@ -135,11 +144,27 @@ public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
     fs.mkdirs(dirPath);
     fs.mkdirs(dirPath);
 
 
     // One request to server
     // One request to server
-    mkdirRequestCount++;
+    if (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs)) {
+      // 1 GetBlobProperties + 1 PutBlob call.
+      mkdirRequestCount +=2;
+    } else {
+      mkdirRequestCount++;
+    }
 
 
     assertAbfsStatistics(
     assertAbfsStatistics(
         CONNECTIONS_MADE,
         CONNECTIONS_MADE,
         totalConnectionMadeBeforeTest + mkdirRequestCount,
         totalConnectionMadeBeforeTest + mkdirRequestCount,
         fs.getInstrumentationMap());
         fs.getInstrumentationMap());
   }
   }
+
+  @Test
+  public void testMkdirWithExistingFilename() throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+
+    fs.create(new Path("/testFilePath"));
+    intercept(FileAlreadyExistsException.class, () -> fs.mkdirs(new Path("/testFilePath")));
+    intercept(FileAlreadyExistsException.class, () -> fs.mkdirs(new Path("/testFilePath/newDir")));
+  }
 }
 }

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

@@ -31,6 +31,7 @@ import org.junit.runners.Parameterized;
 
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.azurebfs.utils.Parallelized;
 import org.apache.hadoop.fs.azurebfs.utils.Parallelized;
@@ -77,6 +78,7 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT
 
 
     final AzureBlobFileSystem fs = this.getFileSystem();
     final AzureBlobFileSystem fs = this.getFileSystem();
     Assume.assumeTrue(getIsNamespaceEnabled(fs));
     Assume.assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE);
     fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE);
     path = new Path(testRoot, UUID.randomUUID().toString());
     path = new Path(testRoot, UUID.randomUUID().toString());
 
 
@@ -94,6 +96,7 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT
   public void testFolderPermission() throws Exception {
   public void testFolderPermission() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     final AzureBlobFileSystem fs = this.getFileSystem();
     Assume.assumeTrue(getIsNamespaceEnabled(fs));
     Assume.assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027");
     fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027");
 
 
     path = new Path(testRoot, UUID.randomUUID().toString());
     path = new Path(testRoot, UUID.randomUUID().toString());

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

@@ -115,6 +115,7 @@ public class ITestAzureBlobFileSystemRandomRead extends
   public void testRandomRead() throws Exception {
   public void testRandomRead() throws Exception {
     Assume.assumeFalse("This test does not support namespace enabled account",
     Assume.assumeFalse("This test does not support namespace enabled account",
         getIsNamespaceEnabled(getFileSystem()));
         getIsNamespaceEnabled(getFileSystem()));
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
     Path testPath = path(TEST_FILE_PREFIX + "_testRandomRead");
     Path testPath = path(TEST_FILE_PREFIX + "_testRandomRead");
     assumeHugeFileExists(testPath);
     assumeHugeFileExists(testPath);
 
 

+ 5 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java

@@ -18,6 +18,7 @@
 
 
 package org.apache.hadoop.fs.azurebfs;
 package org.apache.hadoop.fs.azurebfs;
 
 
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Lists;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
@@ -1042,6 +1043,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
   public void testDefaultAclNewFile() throws Exception {
   public void testDefaultAclNewFile() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     final AzureBlobFileSystem fs = this.getFileSystem();
     assumeTrue(getIsNamespaceEnabled(fs));
     assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     path = new Path(testRoot, UUID.randomUUID().toString());
     path = new Path(testRoot, UUID.randomUUID().toString());
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
     List<AclEntry> aclSpec = Lists.newArrayList(
     List<AclEntry> aclSpec = Lists.newArrayList(
@@ -1124,6 +1126,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
   public void testOnlyAccessAclNewDir() throws Exception {
   public void testOnlyAccessAclNewDir() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     final AzureBlobFileSystem fs = this.getFileSystem();
     assumeTrue(getIsNamespaceEnabled(fs));
     assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     path = new Path(testRoot, UUID.randomUUID().toString());
     path = new Path(testRoot, UUID.randomUUID().toString());
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
     List<AclEntry> aclSpec = Lists.newArrayList(
     List<AclEntry> aclSpec = Lists.newArrayList(
@@ -1163,6 +1166,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
   public void testDefaultAclNewFileWithMode() throws Exception {
   public void testDefaultAclNewFileWithMode() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     final AzureBlobFileSystem fs = this.getFileSystem();
     assumeTrue(getIsNamespaceEnabled(fs));
     assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     path = new Path(testRoot, UUID.randomUUID().toString());
     path = new Path(testRoot, UUID.randomUUID().toString());
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX));
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX));
     List<AclEntry> aclSpec = Lists.newArrayList(
     List<AclEntry> aclSpec = Lists.newArrayList(
@@ -1185,6 +1189,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
   public void testDefaultAclNewDirWithMode() throws Exception {
   public void testDefaultAclNewDirWithMode() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     final AzureBlobFileSystem fs = this.getFileSystem();
     assumeTrue(getIsNamespaceEnabled(fs));
     assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     path = new Path(testRoot, UUID.randomUUID().toString());
     path = new Path(testRoot, UUID.randomUUID().toString());
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX));
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX));
     List<AclEntry> aclSpec = Lists.newArrayList(
     List<AclEntry> aclSpec = Lists.newArrayList(

+ 16 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java

@@ -68,6 +68,8 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
 
 
   private static final String TRUE_STR = "true";
   private static final String TRUE_STR = "true";
   private static final String FALSE_STR = "false";
   private static final String FALSE_STR = "false";
+  private static final String FILESYSTEM_NOT_FOUND_ERROR = "The specified filesystem does not exist.";
+  private static final String CONTAINER_NOT_FOUND_ERROR = "The specified container does not exist.";
 
 
   private boolean isUsingXNSAccount;
   private boolean isUsingXNSAccount;
   public ITestGetNameSpaceEnabled() throws Exception {
   public ITestGetNameSpaceEnabled() throws Exception {
@@ -94,6 +96,8 @@ 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);
     assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT);
+    Assume.assumeTrue("Blob Endpoint Does not Allow FS init on HNS Account",
+        getAbfsServiceType() == AbfsServiceType.DFS);
     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 "
@@ -157,11 +161,17 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
     AzureBlobFileSystem fs = this.getFileSystem(nonExistingFsUrl);
     AzureBlobFileSystem fs = this.getFileSystem(nonExistingFsUrl);
     fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
     fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
 
 
-    intercept(FileNotFoundException.class,
-            "\"The specified filesystem does not exist.\", 404",
-            ()-> {
-              fs.getFileStatus(new Path("/")); // Run a dummy FS call
-            });
+    FileNotFoundException ex = intercept(FileNotFoundException.class, ()-> {
+      fs.getFileStatus(new Path("/")); // Run a dummy FS call
+    });
+
+    String expectedExceptionMessage = getAbfsServiceType() == AbfsServiceType.DFS
+        ? FILESYSTEM_NOT_FOUND_ERROR
+        : CONTAINER_NOT_FOUND_ERROR;
+
+    Assertions.assertThat(ex.getMessage()).describedAs(
+            "Expecting FileNotFoundException with message: " + expectedExceptionMessage)
+        .contains(expectedExceptionMessage);
   }
   }
 
 
   @Test
   @Test
@@ -226,7 +236,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
       throws IOException {
       throws IOException {
     final AzureBlobFileSystem abfs = Mockito.spy(this.getFileSystem());
     final AzureBlobFileSystem abfs = Mockito.spy(this.getFileSystem());
     final AzureBlobFileSystemStore abfsStore = Mockito.spy(abfs.getAbfsStore());
     final AzureBlobFileSystemStore abfsStore = Mockito.spy(abfs.getAbfsStore());
-    final AbfsClient mockClient = mock(AbfsDfsClient.class);
+    final AbfsDfsClient mockClient = mock(AbfsDfsClient.class);
     doReturn(abfsStore).when(abfs).getAbfsStore();
     doReturn(abfsStore).when(abfs).getAbfsStore();
     doReturn(mockClient).when(abfsStore).getClient();
     doReturn(mockClient).when(abfsStore).getClient();
     doReturn(mockClient).when(abfsStore).getClient(any());
     doReturn(mockClient).when(abfsStore).getClient(any());

+ 18 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java

@@ -35,6 +35,9 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 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.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
 
 
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.BYTES_SENT;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.BYTES_SENT;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
@@ -383,6 +386,7 @@ public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest {
         .get(SEND_REQUESTS.getStatName());
         .get(SEND_REQUESTS.getStatName());
     long expectedBytesSent = fs.getInstrumentationMap()
     long expectedBytesSent = fs.getInstrumentationMap()
         .get(BYTES_SENT.getStatName());
         .get(BYTES_SENT.getStatName());
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
 
 
     while (testIteration > 0) {
     while (testIteration > 0) {
       // trigger recurringWriteSize appends over numOfWrites
       // trigger recurringWriteSize appends over numOfWrites
@@ -426,7 +430,9 @@ public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest {
           ? 1 // 1 append (with flush and close param)
           ? 1 // 1 append (with flush and close param)
           : (wasDataPendingToBeWrittenToServer)
           : (wasDataPendingToBeWrittenToServer)
               ? 2 // 1 append + 1 flush (with close)
               ? 2 // 1 append + 1 flush (with close)
-              : 1); // 1 flush (with close)
+              : (recurringWriteSize == 0 && client instanceof AbfsBlobClient)
+                  ? 0 // no flush or close on prefix mode blob
+                  : 1); //1 flush (with close) // 1 flush (with close)
 
 
       expectedTotalRequestsMade += totalAppendFlushCalls;
       expectedTotalRequestsMade += totalAppendFlushCalls;
       expectedRequestsMadeWithData += totalAppendFlushCalls;
       expectedRequestsMadeWithData += totalAppendFlushCalls;
@@ -445,10 +451,19 @@ public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest {
 
 
       testIteration--;
       testIteration--;
     }
     }
+    /**
+     * Above test iteration loop executes one  of the below two patterns
+     * 1. Append + Close (triggers flush)
+     * 2. Append + Flush
+     * For both patters PutBlockList is complete in the iteration loop itself
+     * Hence with PrefixMode Blob, below close won't trigger any network call
+     */
 
 
     opStream.close();
     opStream.close();
-    expectedTotalRequestsMade += 1;
-    expectedRequestsMadeWithData += 1;
+    if (client instanceof AbfsDfsClient) {
+      expectedTotalRequestsMade += 1;
+      expectedRequestsMadeWithData += 1;
+    }
     // no change in expectedBytesSent
     // no change in expectedBytesSent
     assertOpStats(fs.getInstrumentationMap(), expectedTotalRequestsMade, expectedRequestsMadeWithData, expectedBytesSent);
     assertOpStats(fs.getInstrumentationMap(), expectedTotalRequestsMade, expectedRequestsMadeWithData, expectedBytesSent);
 
 

+ 9 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileStatus;
 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.azure.NativeAzureFileSystem;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
@@ -59,6 +60,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
     // test only valid for non-namespace enabled account
     // test only valid for non-namespace enabled account
     Assume.assumeFalse("Namespace enabled account does not support this test,",
     Assume.assumeFalse("Namespace enabled account does not support this test,",
         getIsNamespaceEnabled(fs));
         getIsNamespaceEnabled(fs));
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
 
 
     NativeAzureFileSystem wasb = getWasbFileSystem();
     NativeAzureFileSystem wasb = getWasbFileSystem();
 
 
@@ -95,6 +97,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
     // test only valid for non-namespace enabled account
     // test only valid for non-namespace enabled account
     Assume.assumeFalse("Namespace enabled account does not support this test",
     Assume.assumeFalse("Namespace enabled account does not support this test",
         getIsNamespaceEnabled(abfs));
         getIsNamespaceEnabled(abfs));
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
 
 
     NativeAzureFileSystem wasb = getWasbFileSystem();
     NativeAzureFileSystem wasb = getWasbFileSystem();
 
 
@@ -102,6 +105,12 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
     for (int i = 0; i< 4; i++) {
     for (int i = 0; i< 4; i++) {
       Path path = new Path(testFile + "/~12/!008/testfile" + i);
       Path path = new Path(testFile + "/~12/!008/testfile" + i);
       final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb;
       final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb;
+      // Read
+      final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
+      if (createFs == abfs && readFs == wasb) {
+        //Since flush changes the md5Hash value, md5 returned by GetBlobProperties will not match the one returned by GetBlob.
+        Assume.assumeFalse(getIngressServiceType() == AbfsServiceType.BLOB);
+      }
 
 
       // Write
       // Write
       try(FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
       try(FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
@@ -113,9 +122,6 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
       // Check file status
       // Check file status
       ContractTestUtils.assertIsFile(createFs, path);
       ContractTestUtils.assertIsFile(createFs, path);
 
 
-      // Read
-      final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
-
       try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) {
       try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) {
         String line = br.readLine();
         String line = br.readLine();
         assertEquals("Wrong text from " + readFs,
         assertEquals("Wrong text from " + readFs,

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

@@ -61,6 +61,7 @@ public final class TestConfigurationKeys {
 
 
   public static final String FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT = "fs.azure.test.cpk-enabled-secondary-account";
   public static final String FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT = "fs.azure.test.cpk-enabled-secondary-account";
   public static final String FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY = "fs.azure.test.cpk-enabled-secondary-account.key";
   public static final String FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY = "fs.azure.test.cpk-enabled-secondary-account.key";
+  public static final String FS_AZURE_TEST_FIXED_SAS_TOKEN = "fs.azure.test.fixed.sas.token";
 
 
   public static final String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml";
   public static final String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml";
   public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-";
   public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-";

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

@@ -23,8 +23,10 @@ import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URL;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 import java.util.Set;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantLock;
 
 
@@ -41,7 +43,17 @@ import org.apache.hadoop.util.functional.FunctionRaisingIOE;
 
 
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
 import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_XML;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCKLIST;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_LENGTH;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_TYPE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_MATCH;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_BLOB_CONTENT_MD5;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_CLOSE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_COMP;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType.PutBlockList;
 import static org.apache.hadoop.fs.azurebfs.services.AuthType.OAuth;
 import static org.apache.hadoop.fs.azurebfs.services.AuthType.OAuth;
 import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.EXPONENTIAL_RETRY_POLICY_ABBREVIATION;
 import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.EXPONENTIAL_RETRY_POLICY_ABBREVIATION;
 import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.STATIC_RETRY_POLICY_ABBREVIATION;
 import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.STATIC_RETRY_POLICY_ABBREVIATION;
@@ -87,6 +99,53 @@ public final class AbfsClientTestUtil {
     functionRaisingIOE.apply(httpOperation);
     functionRaisingIOE.apply(httpOperation);
   }
   }
 
 
+  /**
+   * Sets up a mocked AbfsRestOperation for a flush operation in the Azure Blob File System (ABFS).
+   * This method is primarily used in testing scenarios where specific behavior needs to be simulated
+   * during a flush operation, such as returning a particular status code or triggering an exception.
+   *
+   * The method creates a mock AbfsRestOperation configured with the appropriate request headers
+   * and parameters for a "PutBlockList" operation. It then uses the provided
+   * {@code functionRaisingIOE} to modify the behavior of the mock AbfsRestOperation during execution.
+   *
+   * @param spiedClient          The spied instance of the AbfsClient used for making HTTP requests.
+   * @param eTag                 The ETag of the blob, used in the If-Match header to ensure
+   *                             conditional requests.
+   * @param blockListXml         The XML string representing the block list to be uploaded.
+   * @param functionRaisingIOE   A function that modifies the behavior of the AbfsRestOperation,
+   *                             allowing the simulation of specific outcomes, such as HTTP errors
+   *                             or connection resets.
+   * @throws Exception           If an error occurs while setting up the mock operation.
+   */
+  public static void setMockAbfsRestOperationForFlushOperation(
+      final AbfsClient spiedClient, String eTag, String blockListXml, FunctionRaisingIOE<AbfsHttpOperation, AbfsHttpOperation> functionRaisingIOE)
+      throws Exception {
+    List<AbfsHttpHeader> requestHeaders = ITestAbfsClient.getTestRequestHeaders(spiedClient);
+    byte[] buffer = blockListXml.getBytes(StandardCharsets.UTF_8);
+    requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
+    requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, APPLICATION_XML));
+    requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, spiedClient.computeMD5Hash(buffer, 0, buffer.length)));
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = spiedClient.createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(false));
+    final URL url = spiedClient.createRequestUrl("/test/file", abfsUriQueryBuilder.toString());
+    AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation(
+        PutBlockList, spiedClient, HTTP_METHOD_PUT,
+        url,
+        requestHeaders, buffer, 0, buffer.length, null,
+        spiedClient.getAbfsConfiguration()));
+
+    Mockito.doReturn(abfsRestOperation)
+        .when(spiedClient)
+        .getAbfsRestOperation(eq(AbfsRestOperationType.PutBlockList),
+            Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(),
+            Mockito.nullable(byte[].class), Mockito.anyInt(), Mockito.anyInt(),
+            Mockito.nullable(String.class));
+
+    addMockBehaviourToRestOpAndHttpOp(abfsRestOperation, functionRaisingIOE);
+  }
+
   /**
   /**
    * Adding general mock behaviour to AbfsRestOperation and AbfsHttpOperation
    * Adding general mock behaviour to AbfsRestOperation and AbfsHttpOperation
    * to avoid any NPE occurring. These will avoid any network call made and
    * to avoid any NPE occurring. These will avoid any network call made and
@@ -104,6 +163,34 @@ public final class AbfsClientTestUtil {
     Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation();
     Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation();
   }
   }
 
 
+  /**
+   * Adds custom mock behavior to an {@link AbfsRestOperation} and its associated {@link AbfsHttpOperation}.
+   * This method is primarily used in testing scenarios where specific behavior needs to be simulated
+   * during an HTTP operation, such as triggering an exception or modifying the HTTP response.
+   *
+   * The method intercepts the creation of an {@link AbfsHttpOperation} within the provided
+   * {@link AbfsRestOperation} and applies a given function to modify the behavior of the
+   * {@link AbfsHttpOperation}.
+   *
+   * @param abfsRestOperation    The spied instance of {@link AbfsRestOperation} to which the mock
+   *                             behavior is added.
+   * @param functionRaisingIOE   A function that modifies the behavior of the created
+   *                             {@link AbfsHttpOperation}, allowing the simulation of specific
+   *                             outcomes, such as HTTP errors or connection resets.
+   * @throws IOException         If an I/O error occurs while applying the function to the
+   *                             {@link AbfsHttpOperation}.
+   */
+  public static void addMockBehaviourToRestOpAndHttpOp(final AbfsRestOperation abfsRestOperation,
+      FunctionRaisingIOE<AbfsHttpOperation, AbfsHttpOperation> functionRaisingIOE)
+      throws IOException {
+    Mockito.doAnswer(answer -> {
+      AbfsHttpOperation httpOp = (AbfsHttpOperation) Mockito.spy(
+          answer.callRealMethod());
+      functionRaisingIOE.apply(httpOp);
+      return httpOp;
+    }).when(abfsRestOperation).createHttpOperation();
+  }
+
   /**
   /**
    * Adding general mock behaviour to AbfsClient to avoid any NPE occurring.
    * Adding general mock behaviour to AbfsClient to avoid any NPE occurring.
    * These will avoid any network call made and will return the relevant exception or return value directly.
    * These will avoid any network call made and will return the relevant exception or return value directly.

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

@@ -666,6 +666,7 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
       // Give user error code 404 when processResponse is called.
       // Give user error code 404 when processResponse is called.
       Mockito.doReturn(HTTP_METHOD_PUT).when(httpOperation).getMethod();
       Mockito.doReturn(HTTP_METHOD_PUT).when(httpOperation).getMethod();
       Mockito.doReturn(HTTP_NOT_FOUND).when(httpOperation).getStatusCode();
       Mockito.doReturn(HTTP_NOT_FOUND).when(httpOperation).getStatusCode();
+      Mockito.doReturn(HTTP_NOT_FOUND).when(httpOperation).getConnResponseCode();
       Mockito.doReturn("Resource Not Found")
       Mockito.doReturn("Resource Not Found")
           .when(httpOperation)
           .when(httpOperation)
           .getConnResponseMessage();
           .getConnResponseMessage();

+ 41 - 18
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpClientRequestExecutor.java

@@ -25,15 +25,19 @@ import java.io.OutputStream;
 import java.net.URL;
 import java.net.URL;
 
 
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 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.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.http.HttpClientConnection;
 import org.apache.http.HttpClientConnection;
 import org.apache.http.HttpEntityEnclosingRequest;
 import org.apache.http.HttpEntityEnclosingRequest;
@@ -42,6 +46,7 @@ import org.apache.http.HttpRequest;
 import org.apache.http.HttpResponse;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.protocol.HttpClientContext;
 import org.apache.http.client.protocol.HttpClientContext;
 
 
+import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -61,6 +66,10 @@ public class ITestAbfsHttpClientRequestExecutor extends
   public void testExpect100ContinueHandling() throws Exception {
   public void testExpect100ContinueHandling() throws Exception {
     AzureBlobFileSystem fs = getFileSystem();
     AzureBlobFileSystem fs = getFileSystem();
     Path path = new Path("/testExpect100ContinueHandling");
     Path path = new Path("/testExpect100ContinueHandling");
+    if (isAppendBlobEnabled()) {
+      Assume.assumeFalse("Not valid for AppendBlob with blob endpoint",
+          getIngressServiceType() == AbfsServiceType.BLOB);
+    }
 
 
     Configuration conf = new Configuration(fs.getConf());
     Configuration conf = new Configuration(fs.getConf());
     conf.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.toString());
     conf.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.toString());
@@ -187,24 +196,38 @@ public class ITestAbfsHttpClientRequestExecutor extends
 
 
     final OutputStream os = fs2.create(path);
     final OutputStream os = fs2.create(path);
     fs.delete(path, true);
     fs.delete(path, true);
-    intercept(FileNotFoundException.class, () -> {
-      /*
-       * This would lead to two server calls.
-       * First call would be with expect headers, and expect 100 continue
-       *  assertion has to happen which would fail with 404.
-       * Second call would be a retry from AbfsOutputStream, and would not be using expect headers.
-       */
-      os.write(1);
-      os.close();
-    });
-
-    final OutputStream os2 = fs2.create(path);
-    /*
-     * This would lead to third server call. This would be with expect headers,
-     * and the expect 100 continue assertion would pass.
-     */
-    os2.write(1);
-    os2.close();
+    AbfsOutputStream innerOs
+        = (AbfsOutputStream) ((FSDataOutputStream) os).getWrappedStream();
+    if (innerOs.getClientHandler()
+        .getIngressClient() instanceof AbfsDfsClient) {
+      intercept(FileNotFoundException.class, () -> {
+        /*
+         * This would lead to two server calls.
+         * First call would be with expect headers, and expect 100 continue
+         *  assertion has to happen which would fail with 404.
+         * Second call would be a retry from AbfsOutputStream, and would not be using expect headers.
+         */
+        os.write(1);
+        os.close();
+      });
+    } else {
+      AbfsRestOperationException ex = intercept(
+          AbfsRestOperationException.class, () -> {
+            /*
+             * This would lead to two server calls.
+             * First call would be with expect headers, and expect 100 continue
+             *  assertion has to happen which would fail with 404.
+             * Second call would be a retry from AbfsOutputStream, and would not be using expect headers.
+             */
+            try {
+              os.write(1);
+              os.close();
+            } catch (IOException e) {
+              throw (IOException) e.getCause().getCause();
+            }
+          });
+      Assertions.assertThat(ex.getStatusCode()).isEqualTo(HTTP_PRECON_FAILED);
+    }
   }
   }
 
 
   /**
   /**

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

@@ -20,12 +20,15 @@ package org.apache.hadoop.fs.azurebfs.services;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.OutputStream;
+import java.net.ProtocolException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URL;
 import java.util.Arrays;
 import java.util.Arrays;
 
 
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized;
@@ -38,11 +41,21 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
-import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
+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.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.http.HttpResponse;
 
 
+import static java.net.HttpURLConnection.HTTP_CONFLICT;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 
 /**
 /**
  * Test create operation.
  * Test create operation.
@@ -52,6 +65,7 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
 
 
   private static final int TEST_EXECUTION_TIMEOUT = 2 * 60 * 1000;
   private static final int TEST_EXECUTION_TIMEOUT = 2 * 60 * 1000;
   private static final String TEST_FILE_PATH = "testfile";
   private static final String TEST_FILE_PATH = "testfile";
+  private static final int TEN = 10;
 
 
   @Parameterized.Parameter
   @Parameterized.Parameter
   public HttpOperationType httpOperationType;
   public HttpOperationType httpOperationType;
@@ -174,28 +188,39 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
       fs.close();
       fs.close();
       // verify that output stream close after fs.close() would raise a
       // verify that output stream close after fs.close() would raise a
       // pathIOE containing the path being written to.
       // pathIOE containing the path being written to.
-      LambdaTestUtils
-          .intercept(PathIOException.class, getMethodName(), out::close);
+      intercept(PathIOException.class, getMethodName(), out::close);
     }
     }
   }
   }
 
 
   @Test
   @Test
   public void testExpect100ContinueFailureInAppend() throws Exception {
   public void testExpect100ContinueFailureInAppend() throws Exception {
+    if (!getIsNamespaceEnabled(getFileSystem())) {
+      Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    }
     Configuration configuration = new Configuration(getRawConfiguration());
     Configuration configuration = new Configuration(getRawConfiguration());
     configuration.set(FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, "true");
     configuration.set(FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, "true");
     AzureBlobFileSystem fs = getFileSystem(configuration);
     AzureBlobFileSystem fs = getFileSystem(configuration);
     Path path = new Path("/testFile");
     Path path = new Path("/testFile");
     AbfsOutputStream os = Mockito.spy(
     AbfsOutputStream os = Mockito.spy(
         (AbfsOutputStream) fs.create(path).getWrappedStream());
         (AbfsOutputStream) fs.create(path).getWrappedStream());
-    AbfsClient spiedClient = Mockito.spy(os.getClient());
+    AzureIngressHandler ingressHandler = Mockito.spy(
+        os.getIngressHandler());
+    Mockito.doReturn(ingressHandler).when(os).getIngressHandler();
+
+    AbfsClient spiedClient = Mockito.spy(ingressHandler.getClient());
+    Mockito.doReturn(spiedClient).when(ingressHandler).getClient();
     AbfsHttpOperation[] httpOpForAppendTest = new AbfsHttpOperation[2];
     AbfsHttpOperation[] httpOpForAppendTest = new AbfsHttpOperation[2];
     mockSetupForAppend(httpOpForAppendTest, spiedClient);
     mockSetupForAppend(httpOpForAppendTest, spiedClient);
     Mockito.doReturn(spiedClient).when(os).getClient();
     Mockito.doReturn(spiedClient).when(os).getClient();
     fs.delete(path, true);
     fs.delete(path, true);
     os.write(1);
     os.write(1);
-    LambdaTestUtils.intercept(FileNotFoundException.class, () -> {
-      os.close();
-    });
+    if (spiedClient instanceof AbfsDfsClient) {
+      intercept(FileNotFoundException.class, os::close);
+    } else {
+      IOException ex = intercept(IOException.class, os::close);
+      Assertions.assertThat(ex.getCause().getCause()).isInstanceOf(
+          AbfsRestOperationException.class);
+    }
     Assertions.assertThat(httpOpForAppendTest[0].getConnectionDisconnectedOnError())
     Assertions.assertThat(httpOpForAppendTest[0].getConnectionDisconnectedOnError())
         .describedAs("First try from AbfsClient will have expect-100 "
         .describedAs("First try from AbfsClient will have expect-100 "
             + "header and should fail with expect-100 error.").isTrue();
             + "header and should fail with expect-100 error.").isTrue();
@@ -225,9 +250,34 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
     Mockito.doAnswer(abfsRestOpAppendGetInvocation -> {
     Mockito.doAnswer(abfsRestOpAppendGetInvocation -> {
           AbfsRestOperation op = Mockito.spy(
           AbfsRestOperation op = Mockito.spy(
               (AbfsRestOperation) abfsRestOpAppendGetInvocation.callRealMethod());
               (AbfsRestOperation) abfsRestOpAppendGetInvocation.callRealMethod());
+          boolean[] isExpectCall = new boolean[1];
+          for (AbfsHttpHeader header : op.getRequestHeaders()) {
+            if (header.getName().equals(EXPECT)) {
+              isExpectCall[0] = true;
+            }
+          }
           Mockito.doAnswer(createHttpOpInvocation -> {
           Mockito.doAnswer(createHttpOpInvocation -> {
             httpOpForAppendTest[index[0]] = Mockito.spy(
             httpOpForAppendTest[index[0]] = Mockito.spy(
                 (AbfsHttpOperation) createHttpOpInvocation.callRealMethod());
                 (AbfsHttpOperation) createHttpOpInvocation.callRealMethod());
+            if (isExpectCall[0]) {
+              if (httpOpForAppendTest[index[0]] instanceof AbfsJdkHttpOperation) {
+                Mockito.doAnswer(invocation -> {
+                      OutputStream os = (OutputStream) invocation.callRealMethod();
+                      os.write(1);
+                      os.close();
+                      throw new ProtocolException(EXPECT_100_JDK_ERROR);
+                    })
+                    .when((AbfsJdkHttpOperation) httpOpForAppendTest[index[0]])
+                    .getConnOutputStream();
+              } else {
+                Mockito.doAnswer(invocation -> {
+                      throw new AbfsApacheHttpExpect100Exception(
+                          (HttpResponse) invocation.callRealMethod());
+                    })
+                    .when((AbfsAHCHttpOperation) httpOpForAppendTest[index[0]])
+                    .executeRequest();
+              }
+            }
             return httpOpForAppendTest[index[0]++];
             return httpOpForAppendTest[index[0]++];
           }).when(op).createHttpOperation();
           }).when(op).createHttpOperation();
           return op;
           return op;
@@ -253,4 +303,127 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
     return createAbfsOutputStreamWithFlushEnabled(fs1, pathFs1);
     return createAbfsOutputStreamWithFlushEnabled(fs1, pathFs1);
   }
   }
 
 
+  /**
+   * Verify that if getBlockList throws exception append should fail.
+   */
+  @Test
+  public void testValidateGetBlockList() throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    assumeBlobServiceType();
+
+    // Mock the clientHandler to return the blobClient when getBlobClient is called
+    AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+    AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+    Mockito.doReturn(clientHandler).when(store).getClientHandler();
+    Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+    Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    Path testFilePath = new Path("/testFile");
+    AbfsOutputStream os = Mockito.spy((AbfsOutputStream) fs.create(testFilePath).getWrappedStream());
+
+    Mockito.doReturn(clientHandler).when(os).getClientHandler();
+    Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+
+    AbfsRestOperationException exception = getMockAbfsRestOperationException(HTTP_CONFLICT);
+    // Throw exception when getBlockList is called
+    Mockito.doThrow(exception).when(blobClient).getBlockList(Mockito.anyString(), Mockito.any(TracingContext.class));
+
+    // Create a non-empty file
+    os.write(TEN);
+    os.hsync();
+    os.close();
+
+    Mockito.doCallRealMethod().when(store).openFileForWrite(Mockito.any(Path.class), Mockito.any(), Mockito.anyBoolean(), Mockito.any(TracingContext.class));
+    intercept(AzureBlobFileSystemException.class, () -> store
+        .openFileForWrite(testFilePath, null, false, getTestTracingContext(fs, true)));
+  }
+
+  /**
+   * Verify that for flush without append no network calls are made for blob endpoint.
+   **/
+  @Test
+  public void testNoNetworkCallsForFlush() throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    Assume.assumeTrue(store.getClient() instanceof AbfsBlobClient);
+
+    // Mock the clientHandler to return the blobClient when getBlobClient is called
+    AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+    AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+    Mockito.doReturn(clientHandler).when(store).getClientHandler();
+    Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+    Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    Path testFilePath = new Path("/testFile");
+    AbfsOutputStream os = Mockito.spy((AbfsOutputStream) fs.create(testFilePath).getWrappedStream());
+    AzureIngressHandler ingressHandler = Mockito.spy(os.getIngressHandler());
+    Mockito.doReturn(ingressHandler).when(os).getIngressHandler();
+    Mockito.doReturn(blobClient).when(ingressHandler).getClient();
+
+    Mockito.doReturn(clientHandler).when(os).getClientHandler();
+    Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+
+    os.hsync();
+
+    Mockito.verify(blobClient, Mockito.times(0))
+        .append(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(),
+            Mockito.any(TracingContext.class));
+    Mockito.verify(blobClient, Mockito.times(0)).
+        flush(Mockito.any(byte[].class), Mockito.anyString(), Mockito.anyBoolean(),
+            Mockito.anyString(), Mockito.anyString(), Mockito.anyString(), Mockito.any(),
+            Mockito.any(TracingContext.class));
+  }
+
+  private AbfsRestOperationException getMockAbfsRestOperationException(int status) {
+    return new AbfsRestOperationException(status, "", "", new Exception());
+  }
+
+  /**
+   * Verify that for flush without append no network calls are made for blob endpoint.
+   **/
+  @Test
+  public void testNoNetworkCallsForSecondFlush() throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    Assume.assumeTrue(store.getClient() instanceof AbfsBlobClient);
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+
+    // Step 2: Mock the clientHandler to return the blobClient when getBlobClient is called
+    AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+    AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+    Mockito.doReturn(clientHandler).when(store).getClientHandler();
+    Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+    Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    Path testFilePath = new Path("/testFile");
+    AbfsOutputStream os = Mockito.spy((AbfsOutputStream) fs.create(testFilePath).getWrappedStream());
+    AzureIngressHandler ingressHandler = Mockito.spy(os.getIngressHandler());
+    Mockito.doReturn(ingressHandler).when(os).getIngressHandler();
+    Mockito.doReturn(blobClient).when(ingressHandler).getClient();
+
+    Mockito.doReturn(clientHandler).when(os).getClientHandler();
+    Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+
+    os.write(10);
+    os.hsync();
+    os.close();
+
+    Mockito.verify(blobClient, Mockito.times(1))
+        .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any(
+                AppendRequestParameters.class), Mockito.any(), Mockito.any(),
+            Mockito.any(TracingContext.class));
+    Mockito.verify(blobClient, Mockito.times(1)).
+        flush(Mockito.any(byte[].class), Mockito.anyString(), Mockito.anyBoolean(), Mockito.any(), Mockito.any(), Mockito.anyString(), Mockito.any(),
+            Mockito.any(TracingContext.class));
+  }
 }
 }

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

@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.UUID;
 import java.util.UUID;
 
 
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
@@ -33,6 +34,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 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.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider;
 import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers;
 import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers;
@@ -142,6 +144,8 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
    */
    */
   @Test
   @Test
   public void testRecursiveDeleteWithPagination() throws Exception {
   public void testRecursiveDeleteWithPagination() throws Exception {
+    Assume.assumeTrue(
+        getFileSystem().getAbfsStore().getClient() instanceof AbfsDfsClient);
     testRecursiveDeleteWithPaginationInternal(false, true,
     testRecursiveDeleteWithPaginationInternal(false, true,
         AbfsHttpConstants.ApiVersion.DEC_12_2019);
         AbfsHttpConstants.ApiVersion.DEC_12_2019);
     testRecursiveDeleteWithPaginationInternal(false, true,
     testRecursiveDeleteWithPaginationInternal(false, true,
@@ -164,6 +168,7 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
    */
    */
   @Test
   @Test
   public void testNonRecursiveDeleteWithPagination() throws Exception {
   public void testNonRecursiveDeleteWithPagination() throws Exception {
+    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
     testNonRecursiveDeleteWithPaginationInternal(true);
     testNonRecursiveDeleteWithPaginationInternal(true);
     testNonRecursiveDeleteWithPaginationInternal(false);
     testNonRecursiveDeleteWithPaginationInternal(false);
   }
   }
@@ -174,6 +179,8 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
    */
    */
   @Test
   @Test
   public void testRecursiveDeleteWithInvalidCT() throws Exception {
   public void testRecursiveDeleteWithInvalidCT() throws Exception {
+    Assume.assumeTrue(
+        getFileSystem().getAbfsStore().getClient() instanceof AbfsDfsClient);
     testRecursiveDeleteWithInvalidCTInternal(true);
     testRecursiveDeleteWithInvalidCTInternal(true);
     testRecursiveDeleteWithInvalidCTInternal(false);
     testRecursiveDeleteWithInvalidCTInternal(false);
   }
   }

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

@@ -61,7 +61,6 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.E
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION;
-import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION;
@@ -172,8 +171,7 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     AbfsClient abfsClient = fs.getAbfsStore().getClient();
     AbfsClient abfsClient = fs.getAbfsStore().getClient();
 
 
-    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
-        configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME));
+    AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration();
 
 
     // Update the configuration with reduced retry count and reduced backoff interval.
     // Update the configuration with reduced retry count and reduced backoff interval.
     AbfsConfiguration abfsConfig
     AbfsConfiguration abfsConfig
@@ -262,6 +260,7 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
       // behaviour based on response code.
       // behaviour based on response code.
 
 
       Mockito.doReturn(responseCode).when(httpOperation).getStatusCode();
       Mockito.doReturn(responseCode).when(httpOperation).getStatusCode();
+      Mockito.doReturn(responseCode).when(httpOperation).getConnResponseCode();
       if (responseCode == HTTP_UNAVAILABLE) {
       if (responseCode == HTTP_UNAVAILABLE) {
         Mockito.doReturn(EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage())
         Mockito.doReturn(EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage())
             .when(httpOperation)
             .when(httpOperation)

+ 37 - 19
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java

@@ -73,7 +73,7 @@ public final class TestAbfsOutputStream {
       boolean disableOutputStreamFlush,
       boolean disableOutputStreamFlush,
       boolean isAppendBlob,
       boolean isAppendBlob,
       boolean isExpectHeaderEnabled,
       boolean isExpectHeaderEnabled,
-      AbfsClient client,
+      AbfsClientHandler clientHandler,
       String path,
       String path,
       TracingContext tracingContext,
       TracingContext tracingContext,
       ExecutorService executorService) throws IOException,
       ExecutorService executorService) throws IOException,
@@ -97,7 +97,7 @@ public final class TestAbfsOutputStream {
             .withAppendBlob(isAppendBlob)
             .withAppendBlob(isAppendBlob)
             .withWriteMaxConcurrentRequestCount(abfsConf.getWriteMaxConcurrentRequestCount())
             .withWriteMaxConcurrentRequestCount(abfsConf.getWriteMaxConcurrentRequestCount())
             .withMaxWriteRequestsToQueue(abfsConf.getMaxWriteRequestsToQueue())
             .withMaxWriteRequestsToQueue(abfsConf.getMaxWriteRequestsToQueue())
-            .withClient(client)
+            .withClientHandler(clientHandler)
             .withPath(path)
             .withPath(path)
             .withTracingContext(tracingContext)
             .withTracingContext(tracingContext)
             .withExecutorService(executorService)
             .withExecutorService(executorService)
@@ -111,7 +111,8 @@ public final class TestAbfsOutputStream {
   @Test
   @Test
   public void verifyShortWriteRequest() throws Exception {
   public void verifyShortWriteRequest() throws Exception {
 
 
-    AbfsClient client = mock(AbfsClient.class);
+    AbfsClientHandler clientHandler = mock(AbfsClientHandler.class);
+    AbfsDfsClient client = mock(AbfsDfsClient.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsConfiguration abfsConf;
     AbfsConfiguration abfsConf;
     final Configuration conf = new Configuration();
     final Configuration conf = new Configuration();
@@ -125,6 +126,8 @@ public final class TestAbfsOutputStream {
         .thenReturn(op);
         .thenReturn(op);
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
         isNull(), any(), any(TracingContext.class))).thenReturn(op);
         isNull(), any(), any(TracingContext.class))).thenReturn(op);
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
 
 
     AbfsOutputStream out = new AbfsOutputStream(
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
         populateAbfsOutputStreamContext(
@@ -133,7 +136,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             false,
             false,
             true,
             true,
-            client,
+            clientHandler,
             PATH,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
                 FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
@@ -173,7 +176,8 @@ public final class TestAbfsOutputStream {
   @Test
   @Test
   public void verifyWriteRequest() throws Exception {
   public void verifyWriteRequest() throws Exception {
 
 
-    AbfsClient client = mock(AbfsClient.class);
+    AbfsClientHandler clientHandler = mock(AbfsClientHandler.class);
+    AbfsDfsClient client = mock(AbfsDfsClient.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsConfiguration abfsConf;
     AbfsConfiguration abfsConf;
     final Configuration conf = new Configuration();
     final Configuration conf = new Configuration();
@@ -187,6 +191,8 @@ public final class TestAbfsOutputStream {
     when(client.getAbfsPerfTracker()).thenReturn(tracker);
     when(client.getAbfsPerfTracker()).thenReturn(tracker);
     when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))).thenReturn(op);
     when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))).thenReturn(op);
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
 
 
     AbfsOutputStream out = new AbfsOutputStream(
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
         populateAbfsOutputStreamContext(
@@ -195,7 +201,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             false,
             false,
             true,
             true,
-            client,
+            clientHandler,
             PATH,
             PATH,
             tracingContext,
             tracingContext,
             createExecutorService(abfsConf)));
             createExecutorService(abfsConf)));
@@ -243,7 +249,8 @@ public final class TestAbfsOutputStream {
   @Test
   @Test
   public void verifyWriteRequestOfBufferSizeAndClose() throws Exception {
   public void verifyWriteRequestOfBufferSizeAndClose() throws Exception {
 
 
-    AbfsClient client = mock(AbfsClient.class);
+    AbfsClientHandler clientHandler = mock(AbfsClientHandler.class);
+    AbfsDfsClient client = mock(AbfsDfsClient.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class);
     AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class);
     AbfsConfiguration abfsConf;
     AbfsConfiguration abfsConf;
@@ -260,6 +267,8 @@ public final class TestAbfsOutputStream {
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
     when(op.getSasToken()).thenReturn("testToken");
     when(op.getSasToken()).thenReturn("testToken");
     when(op.getResult()).thenReturn(httpOp);
     when(op.getResult()).thenReturn(httpOp);
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
 
 
     AbfsOutputStream out = new AbfsOutputStream(
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
         populateAbfsOutputStreamContext(
@@ -268,7 +277,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             false,
             false,
             true,
             true,
-            client,
+            clientHandler,
             PATH,
             PATH,
             tracingContext,
             tracingContext,
             createExecutorService(abfsConf)));
             createExecutorService(abfsConf)));
@@ -316,7 +325,8 @@ public final class TestAbfsOutputStream {
   @Test
   @Test
   public void verifyWriteRequestOfBufferSize() throws Exception {
   public void verifyWriteRequestOfBufferSize() throws Exception {
 
 
-    AbfsClient client = mock(AbfsClient.class);
+    AbfsClientHandler clientHandler = mock(AbfsClientHandler.class);
+    AbfsDfsClient client = mock(AbfsDfsClient.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class);
     AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class);
     AbfsConfiguration abfsConf;
     AbfsConfiguration abfsConf;
@@ -333,6 +343,8 @@ public final class TestAbfsOutputStream {
         any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
         any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
     when(op.getSasToken()).thenReturn("testToken");
     when(op.getSasToken()).thenReturn("testToken");
     when(op.getResult()).thenReturn(httpOp);
     when(op.getResult()).thenReturn(httpOp);
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
 
 
     AbfsOutputStream out = new AbfsOutputStream(
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
         populateAbfsOutputStreamContext(
@@ -341,7 +353,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             false,
             false,
             true,
             true,
-            client,
+            clientHandler,
             PATH,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
                 FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
@@ -375,7 +387,8 @@ public final class TestAbfsOutputStream {
   @Test
   @Test
   public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception {
   public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception {
 
 
-    AbfsClient client = mock(AbfsClient.class);
+    AbfsClientHandler clientHandler = mock(AbfsClientHandler.class);
+    AbfsDfsClient client = mock(AbfsDfsClient.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsConfiguration abfsConf;
     AbfsConfiguration abfsConf;
     final Configuration conf = new Configuration();
     final Configuration conf = new Configuration();
@@ -389,7 +402,8 @@ public final class TestAbfsOutputStream {
         .thenReturn(op);
         .thenReturn(op);
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
         isNull(), any(), any(TracingContext.class))).thenReturn(op);
         isNull(), any(), any(TracingContext.class))).thenReturn(op);
-
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
     AbfsOutputStream out = new AbfsOutputStream(
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
         populateAbfsOutputStreamContext(
             BUFFER_SIZE,
             BUFFER_SIZE,
@@ -397,7 +411,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             true,
             true,
             true,
             true,
-            client,
+            clientHandler,
             PATH,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
                 FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
@@ -431,7 +445,8 @@ public final class TestAbfsOutputStream {
   @Test
   @Test
   public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception {
   public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception {
 
 
-    AbfsClient client = mock(AbfsClient.class);
+    AbfsClientHandler clientHandler = mock(AbfsClientHandler.class);
+    AbfsDfsClient client = mock(AbfsDfsClient.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     when(op.getSasToken()).thenReturn("");
     when(op.getSasToken()).thenReturn("");
     AbfsConfiguration abfsConf;
     AbfsConfiguration abfsConf;
@@ -449,7 +464,8 @@ public final class TestAbfsOutputStream {
         .thenReturn(op);
         .thenReturn(op);
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
         isNull(), any(), any(TracingContext.class))).thenReturn(op);
         isNull(), any(), any(TracingContext.class))).thenReturn(op);
-
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
     AbfsOutputStream out = new AbfsOutputStream(
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
         populateAbfsOutputStreamContext(
             BUFFER_SIZE,
             BUFFER_SIZE,
@@ -457,7 +473,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             false,
             false,
             true,
             true,
-            client,
+            clientHandler,
             PATH,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
                 FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
@@ -506,7 +522,8 @@ public final class TestAbfsOutputStream {
   @Test
   @Test
   public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception {
   public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception {
 
 
-    AbfsClient client = mock(AbfsClient.class);
+    AbfsClientHandler clientHandler = mock(AbfsClientHandler.class);
+    AbfsDfsClient client = mock(AbfsDfsClient.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsRestOperation op = mock(AbfsRestOperation.class);
     AbfsConfiguration abfsConf;
     AbfsConfiguration abfsConf;
     final Configuration conf = new Configuration();
     final Configuration conf = new Configuration();
@@ -519,7 +536,8 @@ public final class TestAbfsOutputStream {
         .thenReturn(op);
         .thenReturn(op);
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
     when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
         isNull(), any(), any(TracingContext.class))).thenReturn(op);
         isNull(), any(), any(TracingContext.class))).thenReturn(op);
-
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
     AbfsOutputStream out = new AbfsOutputStream(
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
         populateAbfsOutputStreamContext(
             BUFFER_SIZE,
             BUFFER_SIZE,
@@ -527,7 +545,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             false,
             false,
             true,
             true,
-            client,
+            clientHandler,
             PATH,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
                 FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),

+ 38 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AzcopyExecutionException.java

@@ -0,0 +1,38 @@
+/**
+ * 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.utils;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown when there is an error executing the Azcopy tool.
+ * This exception is used only in test code to indicate issues with the Azcopy tool execution.
+ * It provides a suggestion to delete the specified Azcopy tool directory and rerun the tests.
+ */
+public class AzcopyExecutionException extends IOException {
+  private static final String SUGGESTION = "Try deleting the following azcopy tool directory and rerun tests: ";
+
+  public AzcopyExecutionException(String message, String azcopyPath) {
+    super(message + SUGGESTION + azcopyPath);
+  }
+
+  public AzcopyExecutionException(String message, String azcopyPath, Throwable cause) {
+    super(message + SUGGESTION + azcopyPath, cause);
+  }
+}

+ 319 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AzcopyToolHelper.java

@@ -0,0 +1,319 @@
+/**
+ * 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.utils;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DOT;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE;
+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;
+
+/**
+ * Singleton class to create a file or folder in Azure Blob Storage using Azcopy tool.
+ * <a href="https://learn.microsoft.com/en-us/azure/storage/common/storage-use-azcopy-v10">
+ * Azcopy</a> is a command-line utility tool to copy blobs or files to or from a storage account.
+ * It uses Blob Endpoint and ends up creating implicit paths in the storage account.
+ * We will leverage this tool to create implicit paths in storage account for testing purposes.
+ */
+public final class AzcopyToolHelper {
+
+  private File hadoopAzureDir;
+  private String azcopyDirPath;
+  private String azcopyExecutablePath;
+  private String fileCreationScriptPath;
+  private String folderCreationScriptPath;
+  private String fileCreationScriptContent;
+  private String folderCreationScriptContent;
+  private String sasToken;
+  private boolean initialized = false;
+
+  private static final String USER_DIR_SYSTEM_PROPERTY = "user.dir";
+  private static final String HADOOP_AZURE_DIR = "hadoop-azure";
+  private static final String AZCOPY_DIR_NAME = "azcopy";
+  private static final String AZCOPY_EXECUTABLE_NAME = "azcopy";
+  private static final String FILE_CREATION_SCRIPT_NAME = "createAzcopyFile.sh";
+  private static final String FOLDER_CREATION_SCRIPT_NAME = "createAzcopyFolder.sh";
+  private static final String DIR_NOT_FOUND_ERROR = " directory not found";
+  private static final String AZCOPY_DOWNLOADED_DIR_NAME = "/azcopy_linux_amd64_*/* ";
+  private static final String AZCOPY_DOWNLOADED_TAR_NAME = "/azcopy_linux_amd64_* azcopy.tar.gz";
+  private static final String SCRIPT_CREATION_ERROR = "Unable to create azcopy file/folder creation script. ";
+  private static final String SCRIPT_RUN_ERROR = "Unable to run azcopy file/folder creation script. ";
+  private static final String SCRIPT_NOT_FOUND_ERROR = "Azcopy file/folder creation script not found and should be regenerated. ";
+  private static final String SCRIPT_EXECUTION_FAILED = "Azcopy file/folder creation script failed with non-zero exit code. "
+      + "This can be due to corrupt azcopy executable or invalid SAS Token. Exit code: ";
+
+  private static final String AZCOPY_CMD_SHELL = "bash";
+  private static final String AZCOPY_CMD_OPTION = "-c";
+  private static final String AZCOPY_DOWNLOAD_URL = "https://aka.ms/downloadazcopy-v10-linux";
+  private static final String AZCOPY_DOWNLOAD_CMD = "wget " + AZCOPY_DOWNLOAD_URL + " -O azcopy.tar.gz" + " --no-check-certificate";
+  private static final String EXTRACT_CMD = "tar -xf azcopy.tar.gz -C ";
+  private static final String MOVE_CMD = "mv ";
+  private static final String REMOVE_CMD = "rm -rf ";
+  private static final String CHMOD_CMD = "chmod +x ";
+  private static final char QUESTION_MARK = '?';
+  private static final int WAIT_TIME = 10_000; // 10 seconds
+  private static final int MAX_WAIT_TIME = 2 * 6 * WAIT_TIME; // 2 minutes
+  private static final Logger LOG = LoggerFactory.getLogger(AzcopyToolHelper.class);
+
+  private static AzcopyToolHelper azcopyToolHelper; // singleton, initialized in static initialization block
+  private static final ReentrantLock LOCK = new ReentrantLock();
+
+  private AzcopyToolHelper() {
+
+  }
+
+  /**
+   * Constructor to initialize the AzcopyToolHelper. Each JVM running will have
+   * its own instance but will share the tool and scripts.
+   * Azcopy tool work with SAS based authentication. SAS can be configured using
+   * test configuration "fs.azure.test.fixed.sas.token".
+   * @param sasToken to be used for authentication.
+   */
+  public static AzcopyToolHelper getInstance(String sasToken)
+      throws IOException, InterruptedException {
+    if (azcopyToolHelper == null) {
+      LOCK.lock();
+      try {
+        if (azcopyToolHelper == null) {
+          azcopyToolHelper = new AzcopyToolHelper();
+          azcopyToolHelper.init(sasToken);
+        }
+      } finally {
+        LOCK.unlock();
+      }
+    }
+    return azcopyToolHelper;
+  }
+
+  /**
+   * Create a file with implicit parent in the container using Azcopy tool.
+   * @param absolutePathToBeCreated absolute path to be created.
+   * @throws Exception if file creation fails.
+   */
+  public void createFileUsingAzcopy(String absolutePathToBeCreated) throws Exception {
+    if (absolutePathToBeCreated != null) {
+      absolutePathToBeCreated = absolutePathToBeCreated.replace(
+          ABFS_DFS_DOMAIN_NAME, ABFS_BLOB_DOMAIN_NAME) + sasToken;
+      runShellScript(fileCreationScriptPath, absolutePathToBeCreated);
+    }
+  }
+
+  /**
+   * Create a implicit folder with implicit parent in the container using Azcopy tool.
+   * @param absolutePathToBeCreated absolute path to be created.
+   * @throws Exception
+   */
+  public void createFolderUsingAzcopy(String absolutePathToBeCreated) throws Exception {
+    if (absolutePathToBeCreated != null) {
+      absolutePathToBeCreated = absolutePathToBeCreated.replace(
+          ABFS_DFS_DOMAIN_NAME, ABFS_BLOB_DOMAIN_NAME) + sasToken;
+      runShellScript(folderCreationScriptPath, absolutePathToBeCreated);
+    }
+  }
+
+  private void init(String sasToken) throws IOException, InterruptedException {
+    if (initialized) {
+      return;
+    }
+    this.sasToken = sasToken.charAt(0) == QUESTION_MARK ? sasToken : QUESTION_MARK + sasToken;
+    hadoopAzureDir = findHadoopAzureDir();
+    azcopyDirPath = hadoopAzureDir.getAbsolutePath() + FORWARD_SLASH + AZCOPY_DIR_NAME;
+    azcopyExecutablePath = azcopyDirPath + FORWARD_SLASH + AZCOPY_EXECUTABLE_NAME;
+    fileCreationScriptPath = azcopyDirPath + FORWARD_SLASH + FILE_CREATION_SCRIPT_NAME;
+    folderCreationScriptPath = azcopyDirPath + FORWARD_SLASH + FOLDER_CREATION_SCRIPT_NAME;
+    fileCreationScriptContent = "blobPath=$1\n" + "echo $blobPath\n"
+        + azcopyExecutablePath + " copy \"" + azcopyDirPath
+        + "/NOTICE.txt\" $blobPath\n";
+    folderCreationScriptContent = "blobPath=$1\n" + "echo $blobPath\n"
+        + azcopyExecutablePath + " copy \"" + azcopyDirPath
+        + "\" $blobPath --recursive\n";
+
+    /*
+     * Synchronized across JVMs on directory creation. If multiple process try
+     * to create directory, only one will succeed and that process will download
+     * azcopy tool if not present and generate scripts if not present.
+     */
+    downloadAzcopyToolAndGenerateScripts();
+
+    // Change working directory to the hadoop-azure directory.
+    System.setProperty(USER_DIR_SYSTEM_PROPERTY, hadoopAzureDir.getAbsolutePath());
+
+    initialized = true;
+  }
+
+  private void downloadAzcopyToolAndGenerateScripts()
+      throws IOException, InterruptedException {
+    File azcopyDir = new File(azcopyDirPath);
+    if (!azcopyDir.exists()) {
+      if (!azcopyDir.mkdir()) {
+        LOG.info("Azcopy Directory not created by process: {}",
+            Thread.currentThread().getName());
+        return;
+      }
+      downloadAzcopyExecutable();
+      createShellScript(fileCreationScriptPath, fileCreationScriptContent);
+      createShellScript(folderCreationScriptPath, folderCreationScriptContent);
+    } else {
+      LOG.info("Azcopy directory already exists. Skipping download and script "
+          + "generation by the process: {}", Thread.currentThread().getName());
+    }
+  }
+
+  private void downloadAzcopyExecutable()
+      throws IOException, InterruptedException {
+    // Check if azcopy tool is present in the azcopy directory.
+    File azcopyFile = new File(azcopyExecutablePath);
+    if (!azcopyFile.exists()) {
+      // Download Azcopy tool from the Azure website.
+      executeCommand(AZCOPY_DOWNLOAD_CMD);
+
+      // Extract the azcopy executable from the tarball
+      String extractCmd = EXTRACT_CMD + hadoopAzureDir.getAbsolutePath();
+      executeCommand(extractCmd);
+
+      // Rename the azcopy_linux_amd64_* directory to 'azcopy'
+      String renameCmd = MOVE_CMD + hadoopAzureDir.getAbsolutePath()
+          + AZCOPY_DOWNLOADED_DIR_NAME + azcopyDirPath;
+      executeCommand(renameCmd);
+
+      // Remove the downloaded tarball and azcopy folder
+      String cleanupCmd = REMOVE_CMD + hadoopAzureDir.getAbsolutePath()
+          + AZCOPY_DOWNLOADED_TAR_NAME;
+      executeCommand(cleanupCmd);
+
+      // Set the execute permission on the azcopy executable
+      String chmodCmd = CHMOD_CMD + azcopyDirPath;
+      executeCommand(chmodCmd);
+    } else {
+      LOG.info("Azcopy executable already exists. Skipping download by process: {}",
+          Thread.currentThread().getName());
+    }
+  }
+
+  private void executeCommand(String command) throws IOException, InterruptedException {
+    String[] commandArray = {AZCOPY_CMD_SHELL, AZCOPY_CMD_OPTION, command};
+    Process process = Runtime.getRuntime().exec(commandArray);
+    process.waitFor();
+  }
+
+  /**
+   * Create a shell script if not already created.
+   * @param scriptPath to be created
+   * @param scriptContent to be written in the script.
+   */
+  private void createShellScript(String scriptPath, String scriptContent) throws IOException {
+    File scriptFile = new File(scriptPath);
+    if (!scriptFile.exists()) {
+      try {
+        FileWriter writer = new FileWriter(scriptFile);
+        writer.write(scriptContent);
+        writer.close();
+        scriptFile.setExecutable(true); // make the script executable
+      } catch (IOException e) {
+        LOG.error("Error creating shell script: {} by process {}",
+            e.getMessage(), Thread.currentThread().getName());
+        throw new AzcopyExecutionException(SCRIPT_CREATION_ERROR, azcopyDirPath, e);
+      }
+    }
+  }
+
+  private void runShellScript(String scriptPath, String argument) throws IOException {
+    // Check if script exists, otherwise wait for parallel JVM process to create the script.
+    checkAndWaitOnScriptCreation(scriptPath);
+    try {
+      ProcessBuilder pb = new ProcessBuilder(scriptPath, argument);
+      Process p = pb.start();
+      // wait for the process to finish
+      int exitCode = p.waitFor();
+      if (exitCode != 0) {
+        throw new AzcopyExecutionException(SCRIPT_EXECUTION_FAILED + exitCode
+            + DOT + SINGLE_WHITE_SPACE, azcopyDirPath);
+      }
+    } catch (AzcopyExecutionException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AzcopyExecutionException(SCRIPT_RUN_ERROR, azcopyDirPath, e);
+    }
+  }
+
+  private void checkAndWaitOnScriptCreation(String scriptPath) throws IOException {
+    File scriptFile = new File(scriptPath);
+    int totalWaitTime = 0;
+    while (!(scriptFile.exists() && scriptFile.canExecute())) {
+      try {
+        Thread.sleep(WAIT_TIME);
+        totalWaitTime += WAIT_TIME;
+        if (totalWaitTime > MAX_WAIT_TIME) {
+          LOG.error("Timeout waiting for script creation: {} by process {}",
+              scriptPath, Thread.currentThread().getName());
+          throw new AzcopyExecutionException(SCRIPT_NOT_FOUND_ERROR, azcopyDirPath);
+        }
+      } catch (InterruptedException e) {
+        LOG.error("Error waiting for script creation: {} by process {}",
+            scriptPath, Thread.currentThread().getName());
+        throw new AzcopyExecutionException(SCRIPT_NOT_FOUND_ERROR, azcopyDirPath);
+      }
+    }
+  }
+
+  private File findHadoopAzureDir() throws FileNotFoundException {
+    // Find the hadoop-azure directory from the current working directory.
+    File hadoopAzureDir;
+    File currentDir = new File(System.getProperty(USER_DIR_SYSTEM_PROPERTY));
+    if (!currentDir.isDirectory() && !currentDir.getName().equals(HADOOP_AZURE_DIR)) {
+      hadoopAzureDir = findHadoopAzureDir(currentDir);
+      if (hadoopAzureDir == null) {
+        throw new FileNotFoundException(HADOOP_AZURE_DIR + DIR_NOT_FOUND_ERROR);
+      }
+    } else {
+      hadoopAzureDir = currentDir;
+    }
+    return hadoopAzureDir;
+  }
+
+  private File findHadoopAzureDir(File dir) {
+    if (dir == null) {
+      return null;
+    }
+    File[] files = dir.listFiles();
+    if (files == null) {
+      return null;
+    }
+    for (File file : files) {
+      if (file.isDirectory() && file.getName().equals(HADOOP_AZURE_DIR)) {
+        return file;
+      } else {
+        File hadoopAzureDir = findHadoopAzureDir(file);
+        if (hadoopAzureDir != null) {
+          return hadoopAzureDir;
+        }
+      }
+    }
+    return null;
+  }
+}

+ 118 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DirectoryStateHelper.java

@@ -0,0 +1,118 @@
+/**
+ * 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.utils;
+
+import org.junit.Assume;
+
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+
+import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
+
+/**
+ * Helper class to check the state of a directory as implicit or explicit.
+ * With Blob Endpoint support, driver need to handle implicit paths in store at client side.
+ * On DFS Endpoint, this handling is done on server side and driver works seamlessly.
+ * This toll will be used by tests classes to assert that HDFS APIs work
+ * seamlessly on implicit paths even with Blob Endpoint.
+ */
+public final class DirectoryStateHelper {
+
+  private DirectoryStateHelper() {
+  }
+
+  /**
+   * DFS Endpoint abstracts nature of directory from user and hence there is no
+   * way to detect implicit directory using DFS Endpoint APIs.
+   * Similarly implicit paths cannot exist on HNS Enabled Accounts.
+   * To assert that a path exists as implicit directory we need two things to assert.
+   * 1. Blob Endpoint Listing on the path should return some entries.
+   * 2. GetBlobProperties on path should fail on Blob Endpoint.
+   * @param path to be checked. Can be relative or absolute.
+   * @param fs AzureBlobFileSystem for API calls
+   * @return boolean whether the path exists as Implicit directory or not
+   */
+  public static boolean isImplicitDirectory(Path path, AzureBlobFileSystem fs,
+      TracingContext testTracingContext) throws Exception {
+    Assume.assumeFalse(fs.getAbfsStore().getIsNamespaceEnabled(testTracingContext));
+    path = new Path(fs.makeQualified(path).toUri().getPath());
+    String relativePath = fs.getAbfsStore().getRelativePath(path);
+
+    // Implicit nature can be checked only on Blob Endpoint.
+    AbfsBlobClient client = fs.getAbfsStore().getClientHandler().getBlobClient();
+
+    // 1st condition: getPathStatus should fail with HTTP_NOT_FOUND.
+    try {
+      client.getPathStatus(relativePath, testTracingContext, null, false);
+      return false;
+    } catch (AbfsRestOperationException ex) {
+      if (ex.getStatusCode() != HTTP_NOT_FOUND) {
+        return false;
+      }
+    }
+
+    // 2nd condition: listPaths should return some entries.
+    AbfsRestOperation op = client.listPath(
+        relativePath, false, 1, null, testTracingContext, false);
+    if (op != null && op.getResult() != null) {
+      int listSize = op.getResult().getListResultSchema().paths().size();
+      if (listSize > 0) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Every directory in HNS account is explicit directory. For FNS account,
+   * to assert that a path exists as explicit directory, we need to assert that
+   * marker blob exists on the path for both DFS and Blob Endpoint.
+   * @param path to be checked
+   * @param fs AzureBlobFileSystem for API calls
+   * @return boolean whether the path exists as Implicit directory or not
+   */
+  public static boolean isExplicitDirectory(Path path, AzureBlobFileSystem fs,
+      TracingContext testTracingContext) throws Exception {
+    if (path.isRoot()) {
+      return true;
+    }
+    if (fs.getAbfsStore().getIsNamespaceEnabled(testTracingContext)) {
+      try {
+        return fs.getFileStatus(path).isDirectory();
+      } catch (Exception ex) {
+        return false;
+      }
+    }
+    path = new Path(fs.makeQualified(path).toUri().getPath());
+    AbfsBlobClient client = fs.getAbfsStore().getClientHandler().getBlobClient();
+    AbfsRestOperation op = null;
+    try {
+      op = client.getPathStatus(fs.getAbfsStore().getRelativePath(path), testTracingContext, null, false);
+      return client.checkIsDir(op.getResult());
+    } catch (Exception ex) {
+      return false;
+    }
+  }
+}
+

+ 78 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestAzcopyToolHelper.java

@@ -0,0 +1,78 @@
+/**
+ * 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.utils;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+public class TestAzcopyToolHelper extends AbstractAbfsIntegrationTest {
+
+  public TestAzcopyToolHelper() throws Exception {
+
+  }
+
+  @Test
+  public void testGetAzcopyToolCommand() throws Exception {
+    AzureBlobFileSystem fs = this.getFileSystem();
+    Path filePath = path("dir/file.txt");
+    Path implicitDirPath = path("dir1");
+    Path explicitDirPath = path("dir2/dir3");
+    Path nonExistentPath = path("dir/nonexistent");
+    this.createAzCopyFile(filePath);
+    this.createAzCopyFolder(implicitDirPath);
+    fs.mkdirs(explicitDirPath);
+
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(
+            filePath.getParent(), fs, getTestTracingContext(fs, false)))
+        .describedAs("File created by azcopy should have implicit parent")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(
+            implicitDirPath, fs, getTestTracingContext(fs, false)))
+        .describedAs("Folder created by azcopy should be implicit")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(
+            explicitDirPath, fs, getTestTracingContext(fs, false)))
+        .describedAs("Folder created by azcopy should be implicit")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(
+            explicitDirPath.getParent(), fs, getTestTracingContext(fs, false)))
+        .describedAs("Folder created by azcopy should be implicit")
+        .isTrue();
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(
+            filePath, fs, getTestTracingContext(fs, false)))
+        .describedAs("Folder created by azcopy should be implicit")
+        .isFalse();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(
+            filePath, fs, getTestTracingContext(fs, false)))
+        .describedAs("Folder created by azcopy should be implicit")
+        .isFalse();
+    Assertions.assertThat(DirectoryStateHelper.isImplicitDirectory(
+            nonExistentPath, fs, getTestTracingContext(fs, false)))
+        .describedAs("Folder created by azcopy should be implicit")
+        .isFalse();
+    Assertions.assertThat(DirectoryStateHelper.isExplicitDirectory(
+            nonExistentPath, fs, getTestTracingContext(fs, false)))
+        .describedAs("Folder created by azcopy should be implicit")
+        .isFalse();
+  }
+}

+ 20 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java

@@ -38,6 +38,8 @@ public class TracingHeaderValidator implements Listener {
   private TracingHeaderFormat format;
   private TracingHeaderFormat format;
 
 
   private static final String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$";
   private static final String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$";
+  private String ingressHandler = null;
+  private String position = null;
 
 
   @Override
   @Override
   public void callTracingHeaderValidator(String tracingContextHeader,
   public void callTracingHeaderValidator(String tracingContextHeader,
@@ -52,6 +54,8 @@ public class TracingHeaderValidator implements Listener {
         clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId,
         clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId,
         retryNum, streamID);
         retryNum, streamID);
     tracingHeaderValidator.primaryRequestId = primaryRequestId;
     tracingHeaderValidator.primaryRequestId = primaryRequestId;
+    tracingHeaderValidator.ingressHandler = ingressHandler;
+    tracingHeaderValidator.position = position;
     return tracingHeaderValidator;
     return tracingHeaderValidator;
   }
   }
 
 
@@ -92,8 +96,13 @@ public class TracingHeaderValidator implements Listener {
 
 
   private void validateBasicFormat(String[] idList) {
   private void validateBasicFormat(String[] idList) {
     if (format == TracingHeaderFormat.ALL_ID_FORMAT) {
     if (format == TracingHeaderFormat.ALL_ID_FORMAT) {
+      int expectedSize = 8;
+      if (ingressHandler != null) {
+        expectedSize += 2;
+      }
       Assertions.assertThat(idList)
       Assertions.assertThat(idList)
-          .describedAs("header should have 8 elements").hasSize(8);
+          .describedAs("header should have " + expectedSize + " elements")
+          .hasSize(expectedSize);
     } else if (format == TracingHeaderFormat.TWO_ID_FORMAT) {
     } else if (format == TracingHeaderFormat.TWO_ID_FORMAT) {
       Assertions.assertThat(idList)
       Assertions.assertThat(idList)
           .describedAs("header should have 2 elements").hasSize(2);
           .describedAs("header should have 2 elements").hasSize(2);
@@ -152,4 +161,14 @@ public class TracingHeaderValidator implements Listener {
   public void updatePrimaryRequestID(String primaryRequestId) {
   public void updatePrimaryRequestID(String primaryRequestId) {
     this.primaryRequestId = primaryRequestId;
     this.primaryRequestId = primaryRequestId;
   }
   }
+
+  @Override
+  public void updateIngressHandler(String ingressHandler) {
+    this.ingressHandler = ingressHandler;
+  }
+
+  @Override
+  public void updatePosition(String position) {
+    this.position = position;
+  }
 }
 }