Преглед изворни кода

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;
   }
 
+  /**
+   * 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 {
     String value = getPasswordString(key);
     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.hadoop.classification.VisibleForTesting;
 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.security.ProviderUtils;
 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_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_SAS_FIXED_TOKEN;
 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.DATA_BLOCKS_BUFFER_DEFAULT;
 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.UNAUTHORIZED_SAS;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
 import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel;
 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);
     }
 
+    /*
+     * 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).
      * 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");
     if (UserGroupInformation.isSecurityEnabled()) {
@@ -797,7 +824,7 @@ public class AzureBlobFileSystem extends FileSystem
     Path qualifiedPath = makeQualified(path);
 
     try {
-      return abfsStore.getFileStatus(qualifiedPath, tracingContext);
+      return getAbfsStore().getFileStatus(qualifiedPath, tracingContext);
     } catch (AzureBlobFileSystemException ex) {
       checkException(path, ex);
       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");
   }
 
+  /**
+   * 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
    * blob instead of block blob.
@@ -635,14 +647,15 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       final FsPermission permission, final FsPermission umask,
       TracingContext tracingContext) throws IOException {
     try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) {
+      AbfsClient createClient = getClientHandler().getIngressClient();
       boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
       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);
       boolean isAppendBlob = false;
@@ -660,9 +673,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       }
 
       final ContextEncryptionAdapter contextEncryptionAdapter;
-      if (getClient().getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) {
+      if (createClient.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) {
         contextEncryptionAdapter = new ContextProviderEncryptionAdapter(
-            getClient().getEncryptionContextProvider(), getRelativePath(path));
+            createClient.getEncryptionContextProvider(), getRelativePath(path));
       } else {
         contextEncryptionAdapter = NoContextEncryptionAdapter.getInstance();
       }
@@ -677,7 +690,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
         );
 
       } else {
-        op = getClient().createPath(relativePath, true,
+        op = createClient.createPath(relativePath, true,
             overwrite,
             new Permissions(isNamespaceEnabled, permission, umask),
             isAppendBlob,
@@ -689,15 +702,16 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       perfInfo.registerResult(op.getResult()).registerSuccess(true);
 
       AbfsLease lease = maybeCreateLease(relativePath, tracingContext);
-
+      String eTag = extractEtagHeader(op.getResult());
       return new AbfsOutputStream(
           populateAbfsOutputStreamContext(
               isAppendBlob,
               lease,
-              getClient(),
+              getClientHandler(),
               statistics,
               relativePath,
               0,
+              eTag,
               contextEncryptionAdapter,
               tracingContext));
     }
@@ -720,12 +734,12 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       final ContextEncryptionAdapter contextEncryptionAdapter,
       final TracingContext tracingContext) throws IOException {
     AbfsRestOperation op;
-
+    AbfsClient createClient = getClientHandler().getIngressClient();
     try {
       // 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
       // 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);
 
     } 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 {
           // 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);
         } catch (AbfsRestOperationException ex) {
           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 lease          instance of AbfsLease for this AbfsOutputStream.
-   * @param client         AbfsClient.
+   * @param clientHandler  AbfsClientHandler.
    * @param statistics     FileSystem statistics.
    * @param path           Path for AbfsOutputStream.
    * @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
    *                       calls on the same file.
+   * @param eTag           eTag of the file.
    * @param tracingContext instance of TracingContext for this AbfsOutputStream.
    * @return AbfsOutputStreamContext instance with the desired parameters.
    */
   private AbfsOutputStreamContext populateAbfsOutputStreamContext(
       boolean isAppendBlob,
       AbfsLease lease,
-      AbfsClient client,
+      AbfsClientHandler clientHandler,
       FileSystem.Statistics statistics,
       String path,
       long position,
+      String eTag,
       ContextEncryptionAdapter contextEncryptionAdapter,
       TracingContext tracingContext) {
     int bufferSize = abfsConfiguration.getWriteBufferSize();
@@ -814,7 +829,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
             .withEncryptionAdapter(contextEncryptionAdapter)
             .withBlockFactory(getBlockFactory())
             .withBlockOutputActiveBlocks(blockOutputActiveBlocks)
-            .withClient(client)
+            .withClientHandler(clientHandler)
             .withPosition(position)
             .withFsStatistics(statistics)
             .withPath(path)
@@ -822,16 +837,30 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
                 blockOutputActiveBlocks, true))
             .withTracingContext(tracingContext)
             .withAbfsBackRef(fsBackRef)
+            .withIngressServiceType(abfsConfiguration.getIngressServiceType())
+            .withDFSToBlobFallbackEnabled(abfsConfiguration.isDfsToBlobFallbackEnabled())
+            .withETag(eTag)
             .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,
       final FsPermission umask, TracingContext tracingContext)
       throws IOException {
     try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) {
+      AbfsClient createClient = getClientHandler().getIngressClient();
       boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
       LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}",
-              getClient().getFileSystem(),
+              createClient.getFileSystem(),
               path,
               permission,
               umask,
@@ -841,7 +870,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
           !isNamespaceEnabled || abfsConfiguration.isEnabledMkdirOverwrite();
       Permissions permissions = new Permissions(isNamespaceEnabled,
           permission, umask);
-      final AbfsRestOperation op = getClient().createPath(getRelativePath(path),
+      final AbfsRestOperation op = createClient.createPath(getRelativePath(path),
           false, overwrite, permissions, false, null, null, tracingContext);
       perfInfo.registerResult(op.getResult()).registerSuccess(true);
     }
@@ -976,6 +1005,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
               overwrite);
 
       String relativePath = getRelativePath(path);
+      AbfsClient writeClient = getClientHandler().getIngressClient();
 
       final AbfsRestOperation op = getClient()
           .getPathStatus(relativePath, false, tracingContext, null);
@@ -1000,8 +1030,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       }
 
       AbfsLease lease = maybeCreateLease(relativePath, tracingContext);
+      final String eTag = extractEtagHeader(op.getResult());
       final ContextEncryptionAdapter contextEncryptionAdapter;
-      if (getClient().getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) {
+      if (writeClient.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) {
         final String encryptionContext = op.getResult()
             .getResponseHeader(
                 HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT);
@@ -1010,7 +1041,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
               "File doesn't have encryptionContext.");
         }
         contextEncryptionAdapter = new ContextProviderEncryptionAdapter(
-            getClient().getEncryptionContextProvider(), getRelativePath(path),
+            writeClient.getEncryptionContextProvider(), getRelativePath(path),
             encryptionContext.getBytes(StandardCharsets.UTF_8));
       } else {
         contextEncryptionAdapter = NoContextEncryptionAdapter.getInstance();
@@ -1020,10 +1051,11 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
           populateAbfsOutputStreamContext(
               isAppendBlob,
               lease,
-              getClient(),
+              getClientHandler(),
               statistics,
               relativePath,
               offset,
+              eTag,
               contextEncryptionAdapter,
               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 APPEND_BLOB_TYPE = "appendblob";
   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.
 
@@ -238,7 +240,7 @@ public final class AbfsHttpConstants {
   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>
    * 100 - 199 : Informational responses
    * 200 - 299 : Successful responses
@@ -249,6 +251,28 @@ public final class AbfsHttpConstants {
    */
   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.
    * <ol>
@@ -289,6 +313,12 @@ public final class AbfsHttpConstants {
   public static final String APACHE_IMPL = "Apache";
   public static final String JDK_FALLBACK = "JDK_fallback";
   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() {}
 }

+ 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;
 
+  /**
+   * Length of the block ID used for appends.
+   */
+  public static final int BLOCK_ID_LENGTH = 60;
+
   /**
    * Buffer blocks to disk.
    * 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";
 
+  /**
+   * Http Request Header for denoting blob type.
+   * {@value}
+   */
+  public static final String X_MS_BLOB_TYPE = "x-ms-blob-type";
+
   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 boolean isExpectHeaderEnabled;
   private boolean isRetryDueToExpect;
-  private final BlobAppendRequestParameters blobParams;
+  private BlobAppendRequestParameters blobParams;
 
 
   /**
@@ -129,27 +129,65 @@ public class AppendRequestParameters {
     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.
    * @return blockId
    */
   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) {
     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) {
     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),
   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."),
+  BLOB_OPERATION_NOT_SUPPORTED("BlobOperationNotSupported", HttpURLConnection.HTTP_CONFLICT, null),
+  INVALID_APPEND_OPERATION("InvalidAppendOperation", HttpURLConnection.HTTP_CONFLICT, null),
   UNKNOWN(null, -1, null);
 
   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() {
     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_PROPERTIES_OPERATION = "set-properties";
   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.

+ 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.azurebfs.AbfsConfiguration;
 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.HttpHeaderConfigurations;
 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.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.utils.TracingContext;
 
+import static java.net.HttpURLConnection.HTTP_CONFLICT;
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_OK;
 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.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_OCTET_STREAM;
 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.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.BREAK_LEASE_ACTION;
 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_PUT;
 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.LIST;
 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_HDI_ISFOLDER;
 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_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.CONTENT_LENGTH;
 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.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_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_LEASE_ACTION;
 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_PREFIX;
 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.
@@ -328,7 +341,7 @@ public class AbfsBlobClient extends AbfsClient {
       abfsUriQueryBuilder.addQuery(QUERY_PARAM_DELIMITER, FORWARD_SLASH);
     }
     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 AbfsRestOperation op = getAbfsRestOperation(
@@ -377,8 +390,181 @@ public class AbfsBlobClient extends AbfsClient {
       final String eTag,
       final ContextEncryptionAdapter contextEncryptionAdapter,
       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();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -427,6 +614,7 @@ public class AbfsBlobClient extends AbfsClient {
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -454,6 +642,7 @@ public class AbfsBlobClient extends AbfsClient {
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -480,6 +669,7 @@ public class AbfsBlobClient extends AbfsClient {
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
     final AbfsRestOperation op = getAbfsRestOperation(
@@ -540,7 +730,6 @@ public class AbfsBlobClient extends AbfsClient {
     addEncryptionKeyRequestHeaders(path, requestHeaders, false,
         contextEncryptionAdapter, tracingContext);
     requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
-    requestHeaders.add(new AbfsHttpHeader(IF_MATCH, reqParams.getETag()));
     if (reqParams.getLeaseId() != null) {
       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_BLOCKID, reqParams.getBlockId());
 
-    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
+    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION,
         abfsUriQueryBuilder, cachedSasToken);
 
     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,
          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()) {
         LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path);
         reqParams.setExpectHeaderEnabled(false);
@@ -611,6 +800,55 @@ public class AbfsBlobClient extends AbfsClient {
     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.
    * This method is not supported on Blob Endpoint.
@@ -677,7 +915,7 @@ public class AbfsBlobClient extends AbfsClient {
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
     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);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
@@ -739,13 +977,12 @@ public class AbfsBlobClient extends AbfsClient {
 
     AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     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 AbfsRestOperation op = getAbfsRestOperation(
         AbfsRestOperationType.SetPathProperties,
         HTTP_METHOD_PUT, url, requestHeaders);
-    op.execute(tracingContext);
     try {
       op.execute(tracingContext);
     } catch (AbfsRestOperationException ex) {
@@ -756,7 +993,8 @@ public class AbfsBlobClient extends AbfsClient {
       // This path could be present as an implicit directory in FNS.
       if (op.getResult().getStatusCode() == HTTP_NOT_FOUND && isNonEmptyListing(path, tracingContext)) {
         // 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.
         boolean hdiIsFolderExists = properties.containsKey(XML_TAG_HDI_ISFOLDER);
         if (!hdiIsFolderExists) {
@@ -812,7 +1050,7 @@ public class AbfsBlobClient extends AbfsClient {
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN,
         String.valueOf(getAbfsConfiguration().isUpnUsed()));
-    appendSASTokenToQuery(path, SASTokenProvider.GET_PROPERTIES_OPERATION,
+    appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION,
         abfsUriQueryBuilder);
 
     final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
@@ -889,7 +1127,7 @@ public class AbfsBlobClient extends AbfsClient {
     }
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION,
+    String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.FIXED_SAS_STORE_OPERATION,
         abfsUriQueryBuilder, cachedSasToken);
 
     URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
@@ -1031,7 +1269,7 @@ public class AbfsBlobClient extends AbfsClient {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
-    String operation = SASTokenProvider.READ_OPERATION;
+    String operation = SASTokenProvider.FIXED_SAS_STORE_OPERATION;
     appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
 
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
@@ -1069,9 +1307,9 @@ public class AbfsBlobClient extends AbfsClient {
     String dstBlobRelativePath = destinationBlobPath.toUri().getPath();
     String srcBlobRelativePath = sourceBlobPath.toUri().getPath();
     appendSASTokenToQuery(dstBlobRelativePath,
-        SASTokenProvider.WRITE_OPERATION, abfsUriQueryBuilderDst);
+        SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilderDst);
     appendSASTokenToQuery(srcBlobRelativePath,
-        SASTokenProvider.READ_OPERATION, abfsUriQueryBuilderSrc);
+        SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilderSrc);
     final URL url = createRequestUrl(dstBlobRelativePath,
         abfsUriQueryBuilderDst.toString());
     final String sourcePathUrl = createRequestUrl(srcBlobRelativePath,
@@ -1105,7 +1343,7 @@ public class AbfsBlobClient extends AbfsClient {
     AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     String blobRelativePath = blobPath.toUri().getPath();
     appendSASTokenToQuery(blobRelativePath,
-        SASTokenProvider.DELETE_OPERATION, abfsUriQueryBuilder);
+        SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
     final URL url = createRequestUrl(blobRelativePath, abfsUriQueryBuilder.toString());
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     if (leaseId != null) {
@@ -1469,4 +1707,21 @@ public class AbfsBlobClient extends AbfsClient {
     }
     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 AbfsApacheHttpClient abfsApacheHttpClient;
+  private static boolean isNamespaceEnabled = false;
 
   /**
    * logging the rename failure if metadata is in an incomplete state.
@@ -1618,6 +1619,24 @@ public abstract class AbfsClient implements Closeable {
     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() {
     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);
 
   private AbfsServiceType defaultServiceType;
+  private AbfsServiceType ingressServiceType;
   private final AbfsDfsClient dfsAbfsClient;
   private final AbfsBlobClient blobAbfsClient;
 
@@ -80,6 +81,7 @@ public class AbfsClientHandler {
    */
   private void initServiceType(final AbfsConfiguration abfsConfiguration) {
     this.defaultServiceType = abfsConfiguration.getFsConfiguredServiceType();
+    this.ingressServiceType = abfsConfiguration.getIngressServiceType();
   }
 
   /**
@@ -90,6 +92,15 @@ public class AbfsClientHandler {
     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.
    * @param serviceType AbfsServiceType.
@@ -99,6 +110,24 @@ public class AbfsClientHandler {
     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.
    * 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
 public final class AbfsErrors {
   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 "
       + "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 "
       + "detected. Failing request to honor single writer semantics";
   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 "
       + "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 "
-    + "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 "
       + "operation";
   public static final String ERR_NO_LEASE_THREADS = "Lease desired but no 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 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() {}
 }

+ 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
       // error, since the response may have been handled by the HTTP driver
       // 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.Future;
 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.classification.VisibleForTesting;
 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.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.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE;
 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,
     StreamCapabilities, IOStatisticsSource {
 
-  private final AbfsClient client;
+  private volatile AbfsClient client;
   private final String path;
   /** The position in the file being uploaded, where the next block would be
    * uploaded.
@@ -117,24 +123,35 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
   /** Factory for blocks. */
   private final DataBlocks.BlockFactory blockFactory;
 
-  /** Current data block. Null means none currently active. */
-  private DataBlocks.DataBlock activeBlock;
-
   /** Count of blocks uploaded. */
   private long blockCount = 0;
 
-  /** The size of a single block. */
-  private final int blockSize;
-
   /** Executor service to carry out the parallel upload requests. */
   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. */
   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)
       throws IOException {
-    this.client = abfsOutputStreamContext.getClient();
     this.statistics = abfsOutputStreamContext.getStatistics();
     this.path = abfsOutputStreamContext.getPath();
     this.position = abfsOutputStreamContext.getPosition();
@@ -144,7 +161,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     this.disableOutputStreamFlush = abfsOutputStreamContext
             .isDisableOutputStreamFlush();
     this.enableSmallWriteOptimization
-        = abfsOutputStreamContext.isEnableSmallWriteOptimization();
+        = abfsOutputStreamContext.isSmallWriteSupported();
     this.isAppendBlob = abfsOutputStreamContext.isAppendBlob();
     this.lastError = null;
     this.lastFlushOffset = 0;
@@ -155,6 +172,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     this.outputStreamStatistics = abfsOutputStreamContext.getStreamStatistics();
     this.fsBackRef = abfsOutputStreamContext.getFsBackRef();
     this.contextEncryptionAdapter = abfsOutputStreamContext.getEncryptionAdapter();
+    this.eTag = abfsOutputStreamContext.getETag();
 
     if (this.isAppendBlob) {
       this.maxConcurrentRequestCount = 1;
@@ -177,12 +195,185 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     this.tracingContext.setOperation(FSOperationType.WRITE);
     this.ioStatistics = outputStreamStatistics.getIOStatistics();
     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() {
     return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN);
   }
@@ -226,6 +417,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
   @Override
   public synchronized void write(final byte[] data, final int off, final int length)
       throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
     // validate if data is not null and index out of bounds.
     DataBlocks.validateWriteArgs(data, off, length);
     maybeThrowLastError();
@@ -237,8 +431,13 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     if (hasLease() && isLeaseFreed()) {
       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();
 
     if (written < length) {
@@ -264,14 +463,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
    * @return the active block; null if there isn't one.
    * @throws IOException on any failure to create
    */
-  private synchronized DataBlocks.DataBlock createBlockIfNeeded()
+  private synchronized AbfsBlock createBlockIfNeeded(long position)
       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.
    */
   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 {
-      uploadBlockAsync(getActiveBlock(), false, false);
+      uploadBlockAsync(getBlockManager().getActiveBlock(),
+          false, false);
     } 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.
    * @throws IOException     upload failure
    */
-  private void uploadBlockAsync(DataBlocks.DataBlock blockToUpload,
+  private void uploadBlockAsync(AbfsBlock blockToUpload,
       boolean isFlush, boolean isClose)
       throws IOException {
     if (this.isAppendBlob) {
-      writeAppendBlobCurrentBufferToService();
+      getIngressHandler().writeAppendBlobCurrentBufferToService();
       return;
     }
     if (!blockToUpload.hasData()) {
@@ -319,9 +517,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     final Future<Void> job =
         executorService.submit(() -> {
           AbfsPerfTracker tracker =
-              client.getAbfsPerfTracker();
+              getClient().getAbfsPerfTracker();
           try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
-              "writeCurrentBufferToService", "append")) {
+              "writeCurrentBufferToService", APPEND_ACTION)) {
             AppendRequestParameters.Mode
                 mode = APPEND_MODE;
             if (isFlush & isClose) {
@@ -338,9 +536,15 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
              */
             AppendRequestParameters reqParams = new AppendRequestParameters(
                 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());
             perfInfo.registerResult(op.getResult());
             perfInfo.registerSuccess(true);
@@ -361,7 +565,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
    * @param ex Exception caught.
    * @throws IOException Throws the lastError.
    */
-  private void failureWhileSubmit(Exception ex) throws IOException {
+  void failureWhileSubmit(Exception ex) throws IOException {
     if (ex instanceof AbfsRestOperationException) {
       if (((AbfsRestOperationException) ex).getStatusCode()
           == HttpURLConnection.HTTP_NOT_FOUND) {
@@ -376,43 +580,15 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     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?
    *
    * @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() {
     return outputStreamId;
   }
 
+  /**
+   * Registers a listener for this output stream.
+   *
+   * @param listener1 the listener to register.
+   */
   public void registerListener(Listener listener1) {
     listener = listener1;
     tracingContext.setListener(listener);
@@ -522,13 +708,23 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
       bufferIndex = 0;
       closed = true;
       writeOperations.clear();
-      if (hasActiveBlock()) {
-        clearActiveBlock();
+      if (getBlockManager().hasActiveBlock()) {
+        getBlockManager().clearActiveBlock();
       }
     }
     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 {
     maybeThrowLastError();
 
@@ -550,17 +746,35 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
     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 {
     // writeCurrentBufferToService will increment numOfAppendsToServerSinceLastFlush
-    uploadBlockAsync(getActiveBlock(), true, isClose);
+    uploadBlockAsync(getBlockManager().getActiveBlock(),
+        true, isClose);
     waitForAppendsToComplete();
     shrinkWriteOperationQueue();
     maybeThrowLastError();
     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 {
     maybeThrowLastError();
+    // Upload the current block if there is active block data.
     if (hasActiveBlockDataToUpload()) {
       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 {
     for (WriteOperation writeOperation : writeOperations) {
       try {
+        // Wait for the write operation task to complete.
         writeOperation.task.get();
       } catch (Exception ex) {
         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 {
+    // Ensure all appends are completed before flushing.
     waitForAppendsToComplete();
+    // Flush the written bytes to the service.
     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 {
+    // Manage the write operation queue to ensure efficient writes
     shrinkWriteOperationQueue();
 
+    // Only flush if there are uncommitted data beyond the last flush offset
     if (this.lastTotalAppendOffset > this.lastFlushOffset) {
       this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true,
         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,
       final boolean retainUncommitedData, final boolean isClose) throws IOException {
     // flush is called for appendblob only on close
@@ -652,24 +858,43 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
       return;
     }
 
+    // Tracker to monitor performance metrics
     AbfsPerfTracker tracker = client.getAbfsPerfTracker();
     try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
             "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());
         }
+        // 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();
   }
 
+  /**
+   * Gets the reference to the file system back.
+   *
+   * @return The back reference to the file system.
+   */
   @VisibleForTesting
   BackReference getFsBackRef() {
     return fsBackRef;
   }
 
+  /**
+   * Gets the executor service used for asynchronous operations.
+   *
+   * @return The executor service.
+   */
   @VisibleForTesting
   ListeningExecutorService getExecutorService() {
     return executorService;
   }
 
+  /**
+   * Gets the Azure Blob Storage client.
+   *
+   * @return The Azure Blob Storage client.
+   */
   @VisibleForTesting
   AbfsClient getClient() {
     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 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.utils.TracingContext;
 import org.apache.hadoop.fs.impl.BackReference;
@@ -57,8 +58,6 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
 
   private int blockOutputActiveBlocks;
 
-  private AbfsClient client;
-
   private long position;
 
   private FileSystem.Statistics statistics;
@@ -72,6 +71,14 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
   /** A BackReference to the FS instance that created this OutputStream. */
   private BackReference fsBackRef;
 
+  private AbfsServiceType ingressServiceType;
+
+  private boolean isDFSToBlobFallbackEnabled;
+
+  private String eTag;
+
+  private AbfsClientHandler clientHandler;
+
   public AbfsOutputStreamContext(final long 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;
   }
 
@@ -164,12 +171,31 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
     return this;
   }
 
+  public AbfsOutputStreamContext withETag(
+      final String eTag) {
+    this.eTag = eTag;
+    return this;
+  }
+
   public AbfsOutputStreamContext withAbfsBackRef(
       final BackReference fsBackRef) {
     this.fsBackRef = fsBackRef;
     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() {
     // Validation of parameters to be done here.
     if (streamStatistics == null) {
@@ -261,9 +287,6 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
     return blockOutputActiveBlocks;
   }
 
-  public AbfsClient getClient() {
-    return client;
-  }
 
   public FileSystem.Statistics getStatistics() {
     return statistics;
@@ -288,4 +311,32 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
   public BackReference getFsBackRef() {
     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,
     SetBlobMetadata,
     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);
   Listener getClone();
   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
   //final concatenated ID list set into x-ms-client-request-id header
   private String header = EMPTY_STRING;
+  private String ingressHandler = EMPTY_STRING;
+  private String position = EMPTY_STRING;
   private String metricResults = EMPTY_STRING;
   private String metricHeader = EMPTY_STRING;
 
@@ -131,6 +133,8 @@ public class TracingContext {
     this.retryCount = 0;
     this.primaryRequestId = originalTracingContext.primaryRequestId;
     this.format = originalTracingContext.format;
+    this.position = originalTracingContext.getPosition();
+    this.ingressHandler = originalTracingContext.getIngressHandler();
     if (originalTracingContext.listener != null) {
       this.listener = originalTracingContext.listener.getClone();
     }
@@ -193,6 +197,12 @@ public class TracingContext {
               + ":" + opType + ":" + retryCount;
       header = addFailureReasons(header, previousFailure, retryPolicyAbbreviation);
       header += (":" + httpOperation.getTracingContextSuffix());
+      if (!(ingressHandler.equals(EMPTY_STRING))) {
+        header += ":" + ingressHandler;
+      }
+      if (!(position.equals(EMPTY_STRING))) {
+        header += ":" + position;
+      }
       metricHeader += !(metricResults.trim().isEmpty()) ? metricResults  : "";
       break;
     case TWO_ID_FORMAT:
@@ -257,4 +267,45 @@ public class TracingContext {
     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.                                                         |
 | [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.                                  |
+| [Append Block](#append-block)                         | Path          | Commits a new block of data to the end of an existing append blob.                          |
 
 ## Create Container
 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
 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.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 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.AbfsHttpConstants;
 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.TracingHeaderFormat;
 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 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.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.constants.TestConfigurationKeys.*;
 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,
             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);
     assumeValidAuthConfigsPresent();
@@ -111,9 +120,6 @@ public abstract class AbstractAbfsIntegrationTest extends
     abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME
             : FileSystemUriSchemes.ABFS_SECURE_SCHEME;
 
-    final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
-    URI defaultUri = null;
-
     try {
       defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
     } catch (Exception ex) {
@@ -436,6 +442,13 @@ public abstract class AbstractAbfsIntegrationTest extends
         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(
       final String url,
       final String fromNonSecureScheme,
@@ -575,4 +588,52 @@ public abstract class AbstractAbfsIntegrationTest extends
   protected boolean isAppendBlobEnabled() {
     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.PathIOException;
 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.services.AbfsClientUtils;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 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.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.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_ENCODED_CLIENT_PROVIDED_KEY;
 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 String cpkSHAEncoded;
+  private static final String BLOCK_ID = "MF8tNDE1MjkzOTE4AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA";
 
   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)
       throws Exception {
     Path testPath = path("/testFile");
@@ -244,7 +256,9 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
       Path testPath, EncryptionContextProvider ecp)
       throws Exception {
     AbfsClient client = fs.getAbfsClient();
+    AbfsClient ingressClient = fs.getAbfsStore().getClientHandler().getIngressClient();
     AbfsClientUtils.setEncryptionContextProvider(client, ecp);
+    AbfsClientUtils.setEncryptionContextProvider(ingressClient, ecp);
     if (isExceptionCase) {
       LambdaTestUtils.intercept(IOException.class, () -> {
         switch (operation) {
@@ -310,12 +324,26 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
           }
         }
       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:
-        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:
         return client.setAcl(path, AclEntry.aclSpecToString(
           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.FSDataOutputStream;
 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.fs.Path;
 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.GET_RESPONSES;
 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 {
 
@@ -47,6 +51,21 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
   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
    * {@link AbfsRestOperation}.
@@ -59,12 +78,15 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
     AzureBlobFileSystem fs = getFileSystem();
     Map<String, Long> metricMap;
     Path sendRequestPath = path(getMethodName());
+    String path = sendRequestPath.toString();
+    int directory = countDirectory(path);
     String testNetworkStatsString = "http_send";
 
-    metricMap = fs.getInstrumentationMap();
+    metricMap = getInstrumentationMap(fs);
     long expectedConnectionsMade = metricMap.get(CONNECTIONS_MADE.getStatName());
     long expectedRequestsSent = metricMap.get(SEND_REQUESTS.getStatName());
     long expectedBytesSent = 0;
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
 
     // --------------------------------------------------------------------
      // Operation: Creating AbfsOutputStream
@@ -72,8 +94,14 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
         sendRequestPath)) {
        // Network stats calculation: For Creating AbfsOutputStream:
        // 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
@@ -103,7 +131,7 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
        *   1 append = 1 connection and 1 send request
        */
       if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(sendRequestPath).toString())
-          || (this.getConfiguration().isSmallWriteOptimizationEnabled())) {
+          || (fs.getAbfsStore().getAbfsConfiguration().isSmallWriteOptimizationEnabled())) {
         expectedConnectionsMade++;
         expectedRequestsSent++;
       } else {
@@ -114,7 +142,7 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
       // --------------------------------------------------------------------
 
       // Assertions
-      metricMap = fs.getInstrumentationMap();
+      metricMap = getInstrumentationMap(fs);
       assertAbfsStatistics(CONNECTIONS_MADE,
           expectedConnectionsMade, metricMap);
       assertAbfsStatistics(SEND_REQUESTS, expectedRequestsSent,
@@ -127,8 +155,11 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
     // Operation: AbfsOutputStream close.
     // Network Stats calculation: 1 flush (with close) is send.
     // 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
@@ -144,11 +175,17 @@ public class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
        *    create overwrite=false (will fail in this case as file is indeed present)
        *    + getFileStatus to fetch the file ETag
        *    + 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 {
         expectedConnectionsMade += 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.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.services.AzureServiceErrorCode;
 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)
       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);
       // 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())
           .describedAs("Status Code Field in exception message "
               + "should be \"404\"")
@@ -91,46 +101,85 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest
     try {
       fs.listFiles(nonExistedFilePath2, false);
     } catch (Exception ex) {
-      // verify its format
       String errorMessage = ex.getLocalizedMessage();
       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.IOException;
 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.List;
 import java.util.Random;
 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.junit.Assume;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.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.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.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.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_DISK;
 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.Writing;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.mockito.ArgumentMatchers.anyString;
 
 /**
  * Test append operations.
  */
 public class ITestAzureBlobFileSystemAppend extends
     AbstractAbfsIntegrationTest {
+
   private static final String TEST_FILE_PATH = "testfile";
+
   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 {
     super();
   }
@@ -69,7 +119,7 @@ public class ITestAzureBlobFileSystemAppend extends
   @Test
   public void testAppendWithLength0() throws Exception {
     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];
       new Random().nextBytes(b);
       stream.write(b, 1000, 0);
@@ -104,7 +154,7 @@ public class ITestAzureBlobFileSystemAppend extends
     fs.registerListener(new TracingHeaderValidator(
         fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
         fs.getFileSystemId(), FSOperationType.APPEND, false, 0));
-    fs.append(testPath, 10);
+    fs.append(testPath, TEN);
   }
 
   @Test
@@ -116,37 +166,1013 @@ public class ITestAzureBlobFileSystemAppend extends
     for (String blockBufferType : blockBufferTypes) {
       Configuration configuration = new Configuration(getRawConfiguration());
       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());
-      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();
-        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;
 
+import java.nio.charset.StandardCharsets;
 import java.security.SecureRandom;
 import java.util.Arrays;
 import java.util.HashSet;
 
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -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.AbfsRestOperationException;
 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.AbfsOutputStream;
 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.FileSystemConfigurations.BLOCK_ID_LENGTH;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -69,16 +75,19 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
   @Test
   public void testAppendWithChecksumAtDifferentOffsets() throws Exception {
     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());
-    fs.create(path);
+    AbfsOutputStream os = (AbfsOutputStream) fs.create(path).getWrappedStream();
     byte[] data = generateRandomBytes(MB_4);
     int pos = 0;
 
-    pos += appendWithOffsetHelper(client, path, data, fs, pos, 0);
-    pos += appendWithOffsetHelper(client, path, data, fs, pos, ONE_MB);
-    pos += appendWithOffsetHelper(client, path, data, fs, pos, MB_2);
-    appendWithOffsetHelper(client, path, data, fs, pos, MB_4 - 1);
+    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();
   }
 
@@ -107,16 +116,16 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
   @Test
   public void testAbfsInvalidChecksumExceptionInAppend() throws Exception {
     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());
-    fs.create(path);
+    AbfsOutputStream os = (AbfsOutputStream) fs.create(path).getWrappedStream();
     byte[] data= generateRandomBytes(MB_4);
     String invalidMD5Hash = spiedClient.computeMD5Hash(
             INVALID_MD5_TEXT.getBytes(), 0, INVALID_MD5_TEXT.length());
     Mockito.doReturn(invalidMD5Hash).when(spiedClient).computeMD5Hash(any(),
         any(Integer.class), any(Integer.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())
@@ -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
    * computed at server side. If not, request will fail with 400 Bad request.
@@ -173,10 +196,13 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
    * @param offset
    * @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 {
+    String blockId = generateBlockId(os, pos);
+    String eTag = os.getIngressHandler().getETag();
     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,
         getTestTracingContext(fs, false));
     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.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException;
 import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider;
@@ -92,6 +93,7 @@ public class ITestAzureBlobFileSystemChooseSAS extends AbstractAbfsIntegrationTe
    */
   @Test
   public void testBothProviderFixedTokenConfigured() throws Exception {
+    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS && getIsNamespaceEnabled(getFileSystem()));
     AbfsConfiguration testAbfsConfig = new AbfsConfiguration(
         getRawConfiguration(), this.getAccountName());
     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.FilterOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.EnumSet;
+import java.util.List;
 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.junit.Assume;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 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.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.FsPermission;
 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.utils.TracingContext;
 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_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_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.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
@@ -208,7 +223,7 @@ public class ITestAzureBlobFileSystemCreate extends
       out.write('2');
       out.hsync();
       fail("Expected a failure");
-    } catch (FileNotFoundException fnfe) {
+    } 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())) {
@@ -233,10 +248,9 @@ public class ITestAzureBlobFileSystemCreate extends
   @Test
   public void testFilterFSWriteAfterClose() throws Throwable {
     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);
-    intercept(FileNotFoundException.class,
+    intercept(IOException.class,
         () -> {
           try (FilterOutputStream fos = new FilterOutputStream(out)) {
             byte[] bytes = new byte[8*ONE_MB];
@@ -246,12 +260,12 @@ public class ITestAzureBlobFileSystemCreate extends
             out.hsync();
             fs.delete(testPath, false);
             // 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());
     config.set("fs.azure.enable.conditional.create.overwrite",
         Boolean.toString(enableConditionalCreateOverwrite));
+    AzureBlobFileSystemStore store = currentFs.getAbfsStore();
+    AbfsClient client = store.getClientHandler().getIngressClient();
 
     final AzureBlobFileSystem fs =
         (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(),
@@ -296,7 +312,11 @@ public class ITestAzureBlobFileSystemCreate extends
     fs.create(nonOverwriteFile, false);
 
     // 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(
         CONNECTIONS_MADE,
@@ -312,7 +332,11 @@ public class ITestAzureBlobFileSystemCreate extends
     fs.registerListener(null);
 
     // 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(
         CONNECTIONS_MADE,
@@ -326,8 +350,12 @@ public class ITestAzureBlobFileSystemCreate extends
     // create should be successful
     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(
         CONNECTIONS_MADE,
@@ -341,12 +369,15 @@ public class ITestAzureBlobFileSystemCreate extends
     fs.create(overwriteFilePath, true);
     fs.registerListener(null);
 
+    createRequestCount += (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs) ? 1: 0);
+
+    // Second actual create call will hap
     if (enableConditionalCreateOverwrite) {
       // Three requests will be sent to server to create path,
       // 1. create without overwrite
       // 2. GetFileStatus to get eTag
       // 3. create with overwrite
-      createRequestCount += 3;
+      createRequestCount += (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs) ? 4: 3);
     } else {
       createRequestCount++;
     }
@@ -393,11 +424,14 @@ public class ITestAzureBlobFileSystemCreate extends
         = ITestAbfsClient.getMockAbfsClient(
         fs.getAbfsStore().getClient(),
         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();
 
+    ReflectionUtils.setFinalField(AzureBlobFileSystemStore.class, abfsStore, "clientHandler", clientHandler);
     ReflectionUtils.setFinalField(AzureBlobFileSystemStore.class, abfsStore, "client", mockClient);
-
     boolean isNamespaceEnabled = abfsStore
         .getIsNamespaceEnabled(getTestTracingContext(fs, false));
 
@@ -494,7 +528,7 @@ public class ITestAzureBlobFileSystemCreate extends
         FsAction.ALL);
     FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
         FsAction.NONE);
-    Path testPath = new Path("testFile");
+    Path testPath = new Path("/testFile");
     intercept(
         exceptionClass,
         () -> abfsStore.createFile(testPath, null, true, permission, umask,
@@ -504,4 +538,869 @@ public class ITestAzureBlobFileSystemCreate extends
   private AbfsRestOperationException getMockAbfsRestOperationException(int status) {
     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 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.assertj.core.api.Assertions;
 import org.junit.Test;
@@ -200,6 +202,7 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
   public void testWriteWithFileNotFoundException() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();
     final Path testFilePath = path(methodName.getMethodName());
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
 
     try (FSDataOutputStream stream = fs.create(testFilePath)) {
       assertPathExists(fs, "Path should exist", testFilePath);
@@ -209,7 +212,11 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
       assertPathDoesNotExist(fs, "This path should not exist", testFilePath);
 
       // 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 {
     final AzureBlobFileSystem fs = getFileSystem();
     final Path testFilePath = path(methodName.getMethodName());
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testFilePath).toString())) {
       return;
     }
@@ -227,7 +235,11 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
       fs.delete(testFilePath, true);
       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.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.hamcrest.core.IsEqual;
 import org.hamcrest.core.IsNot;
+import org.junit.Assume;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -308,6 +310,11 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
 
   @Test
   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());
     config.set(FS_AZURE_APPEND_BLOB_KEY, "abfss:/");
     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(
         getRawConfiguration()));
     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.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.contracts.exceptions.AbfsDriverException;
 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.AbfsLease;
 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.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.anyInt;
 import static org.mockito.ArgumentMatchers.anyString;
@@ -139,12 +143,15 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   public void testTwoCreate() throws Exception {
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1);
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT);
     fs.mkdirs(testFilePath.getParent());
 
     try (FSDataOutputStream out = fs.create(testFilePath)) {
       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)) {
         }
         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 {
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     try (FSDataOutputStream out = fs.create(testFilePath)) {
       try (FSDataOutputStream out2 = fs.append(testFilePath)) {
         out2.writeInt(2);
@@ -166,7 +174,23 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
         }
       }
       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());
@@ -213,6 +237,7 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   public void testWriteAfterBreakLease() throws Exception {
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1);
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     fs.mkdirs(testFilePath.getParent());
 
     FSDataOutputStream out;
@@ -225,14 +250,15 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
         FSOperationType.BREAK_LEASE, false, 0));
     fs.breakLease(testFilePath);
     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.hsync();
       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();
       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 {
     final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE);
     final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1);
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
     fs.mkdirs(testFilePath.getParent());
 
     FSDataOutputStream out = fs.create(testFilePath);
     out.write(0);
 
     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();
       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.Test;
+import org.mockito.Mockito;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
 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.constants.ConfigurationKeys.FS_AZURE_ENABLE_MKDIR_OVERWRITE;
@@ -123,7 +126,13 @@ public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
     fs.mkdirs(dirPath);
 
     // 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(
         CONNECTIONS_MADE,
@@ -135,11 +144,27 @@ public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
     fs.mkdirs(dirPath);
 
     // One request to server
-    mkdirRequestCount++;
+    if (client instanceof AbfsBlobClient && !getIsNamespaceEnabled(fs)) {
+      // 1 GetBlobProperties + 1 PutBlob call.
+      mkdirRequestCount +=2;
+    } else {
+      mkdirRequestCount++;
+    }
 
     assertAbfsStatistics(
         CONNECTIONS_MADE,
         totalConnectionMadeBeforeTest + mkdirRequestCount,
         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.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.azurebfs.utils.Parallelized;
@@ -77,6 +78,7 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT
 
     final AzureBlobFileSystem fs = this.getFileSystem();
     Assume.assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE);
     path = new Path(testRoot, UUID.randomUUID().toString());
 
@@ -94,6 +96,7 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT
   public void testFolderPermission() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     Assume.assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027");
 
     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 {
     Assume.assumeFalse("This test does not support namespace enabled account",
         getIsNamespaceEnabled(getFileSystem()));
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
     Path testPath = path(TEST_FILE_PREFIX + "_testRandomRead");
     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;
 
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.util.Lists;
 
 import java.io.FileNotFoundException;
@@ -1042,6 +1043,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
   public void testDefaultAclNewFile() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     path = new Path(testRoot, UUID.randomUUID().toString());
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
     List<AclEntry> aclSpec = Lists.newArrayList(
@@ -1124,6 +1126,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
   public void testOnlyAccessAclNewDir() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     path = new Path(testRoot, UUID.randomUUID().toString());
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
     List<AclEntry> aclSpec = Lists.newArrayList(
@@ -1163,6 +1166,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
   public void testDefaultAclNewFileWithMode() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     path = new Path(testRoot, UUID.randomUUID().toString());
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX));
     List<AclEntry> aclSpec = Lists.newArrayList(
@@ -1185,6 +1189,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
   public void testDefaultAclNewDirWithMode() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();
     assumeTrue(getIsNamespaceEnabled(fs));
+    Assume.assumeTrue(getIngressServiceType() == AbfsServiceType.DFS);
     path = new Path(testRoot, UUID.randomUUID().toString());
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX));
     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 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;
   public ITestGetNameSpaceEnabled() throws Exception {
@@ -94,6 +96,8 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
   @Test
   public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception {
     assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT);
+    Assume.assumeTrue("Blob Endpoint Does not Allow FS init on HNS Account",
+        getAbfsServiceType() == AbfsServiceType.DFS);
     AzureBlobFileSystem fs = getNewFSWithHnsConf(TRUE_STR);
     Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs(
         "getIsNamespaceEnabled should return true when the "
@@ -157,11 +161,17 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
     AzureBlobFileSystem fs = this.getFileSystem(nonExistingFsUrl);
     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
@@ -226,7 +236,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
       throws IOException {
     final AzureBlobFileSystem abfs = Mockito.spy(this.getFileSystem());
     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(mockClient).when(abfsStore).getClient();
     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.Path;
 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.CONNECTIONS_MADE;
@@ -383,6 +386,7 @@ public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest {
         .get(SEND_REQUESTS.getStatName());
     long expectedBytesSent = fs.getInstrumentationMap()
         .get(BYTES_SENT.getStatName());
+    AbfsClient client = fs.getAbfsStore().getClientHandler().getIngressClient();
 
     while (testIteration > 0) {
       // trigger recurringWriteSize appends over numOfWrites
@@ -426,7 +430,9 @@ public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest {
           ? 1 // 1 append (with flush and close param)
           : (wasDataPendingToBeWrittenToServer)
               ? 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;
       expectedRequestsMadeWithData += totalAppendFlushCalls;
@@ -445,10 +451,19 @@ public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest {
 
       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();
-    expectedTotalRequestsMade += 1;
-    expectedRequestsMadeWithData += 1;
+    if (client instanceof AbfsDfsClient) {
+      expectedTotalRequestsMade += 1;
+      expectedRequestsMadeWithData += 1;
+    }
     // no change in 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.Path;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 
 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
     Assume.assumeFalse("Namespace enabled account does not support this test,",
         getIsNamespaceEnabled(fs));
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
 
     NativeAzureFileSystem wasb = getWasbFileSystem();
 
@@ -95,6 +97,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
     // test only valid for non-namespace enabled account
     Assume.assumeFalse("Namespace enabled account does not support this test",
         getIsNamespaceEnabled(abfs));
+    Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
 
     NativeAzureFileSystem wasb = getWasbFileSystem();
 
@@ -102,6 +105,12 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
     for (int i = 0; i< 4; i++) {
       Path path = new Path(testFile + "/~12/!008/testfile" + i);
       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
       try(FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
@@ -113,9 +122,6 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
       // Check file status
       ContractTestUtils.assertIsFile(createFs, path);
 
-      // Read
-      final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
-
       try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) {
         String line = br.readLine();
         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_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_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.URISyntaxException;
 import java.net.URL;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 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_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_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.RetryPolicyConstants.EXPONENTIAL_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);
   }
 
+  /**
+   * 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
    * 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();
   }
 
+  /**
+   * 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.
    * 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.
       Mockito.doReturn(HTTP_METHOD_PUT).when(httpOperation).getMethod();
       Mockito.doReturn(HTTP_NOT_FOUND).when(httpOperation).getStatusCode();
+      Mockito.doReturn(HTTP_NOT_FOUND).when(httpOperation).getConnResponseCode();
       Mockito.doReturn("Resource Not Found")
           .when(httpOperation)
           .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 org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.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.http.HttpClientConnection;
 import org.apache.http.HttpEntityEnclosingRequest;
@@ -42,6 +46,7 @@ import org.apache.http.HttpRequest;
 import org.apache.http.HttpResponse;
 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.HttpOperationType.APACHE_HTTP_CLIENT;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -61,6 +66,10 @@ public class ITestAbfsHttpClientRequestExecutor extends
   public void testExpect100ContinueHandling() throws Exception {
     AzureBlobFileSystem fs = getFileSystem();
     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());
     conf.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.toString());
@@ -187,24 +196,38 @@ public class ITestAbfsHttpClientRequestExecutor extends
 
     final OutputStream os = fs2.create(path);
     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.IOException;
+import java.io.OutputStream;
+import java.net.ProtocolException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.Arrays;
 
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 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.azurebfs.AbstractAbfsIntegrationTest;
 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.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.HttpHeaderConfigurations.EXPECT;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * 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 String TEST_FILE_PATH = "testfile";
+  private static final int TEN = 10;
 
   @Parameterized.Parameter
   public HttpOperationType httpOperationType;
@@ -174,28 +188,39 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
       fs.close();
       // verify that output stream close after fs.close() would raise a
       // pathIOE containing the path being written to.
-      LambdaTestUtils
-          .intercept(PathIOException.class, getMethodName(), out::close);
+      intercept(PathIOException.class, getMethodName(), out::close);
     }
   }
 
   @Test
   public void testExpect100ContinueFailureInAppend() throws Exception {
+    if (!getIsNamespaceEnabled(getFileSystem())) {
+      Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+    }
     Configuration configuration = new Configuration(getRawConfiguration());
     configuration.set(FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, "true");
     AzureBlobFileSystem fs = getFileSystem(configuration);
     Path path = new Path("/testFile");
     AbfsOutputStream os = Mockito.spy(
         (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];
     mockSetupForAppend(httpOpForAppendTest, spiedClient);
     Mockito.doReturn(spiedClient).when(os).getClient();
     fs.delete(path, true);
     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())
         .describedAs("First try from AbfsClient will have expect-100 "
             + "header and should fail with expect-100 error.").isTrue();
@@ -225,9 +250,34 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
     Mockito.doAnswer(abfsRestOpAppendGetInvocation -> {
           AbfsRestOperation op = Mockito.spy(
               (AbfsRestOperation) abfsRestOpAppendGetInvocation.callRealMethod());
+          boolean[] isExpectCall = new boolean[1];
+          for (AbfsHttpHeader header : op.getRequestHeaders()) {
+            if (header.getName().equals(EXPECT)) {
+              isExpectCall[0] = true;
+            }
+          }
           Mockito.doAnswer(createHttpOpInvocation -> {
             httpOpForAppendTest[index[0]] = Mockito.spy(
                 (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]++];
           }).when(op).createHttpOperation();
           return op;
@@ -253,4 +303,127 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
     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 org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
 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.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider;
 import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers;
@@ -142,6 +144,8 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
    */
   @Test
   public void testRecursiveDeleteWithPagination() throws Exception {
+    Assume.assumeTrue(
+        getFileSystem().getAbfsStore().getClient() instanceof AbfsDfsClient);
     testRecursiveDeleteWithPaginationInternal(false, true,
         AbfsHttpConstants.ApiVersion.DEC_12_2019);
     testRecursiveDeleteWithPaginationInternal(false, true,
@@ -164,6 +168,7 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
    */
   @Test
   public void testNonRecursiveDeleteWithPagination() throws Exception {
+    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
     testNonRecursiveDeleteWithPaginationInternal(true);
     testNonRecursiveDeleteWithPaginationInternal(false);
   }
@@ -174,6 +179,8 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
    */
   @Test
   public void testRecursiveDeleteWithInvalidCT() throws Exception {
+    Assume.assumeTrue(
+        getFileSystem().getAbfsStore().getClient() instanceof AbfsDfsClient);
     testRecursiveDeleteWithInvalidCTInternal(true);
     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.HttpQueryParams.QUERY_PARAM_ACTION;
 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.HttpOperationType.APACHE_HTTP_CLIENT;
 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);
     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.
     AbfsConfiguration abfsConfig
@@ -262,6 +260,7 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
       // behaviour based on response code.
 
       Mockito.doReturn(responseCode).when(httpOperation).getStatusCode();
+      Mockito.doReturn(responseCode).when(httpOperation).getConnResponseCode();
       if (responseCode == HTTP_UNAVAILABLE) {
         Mockito.doReturn(EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage())
             .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 isAppendBlob,
       boolean isExpectHeaderEnabled,
-      AbfsClient client,
+      AbfsClientHandler clientHandler,
       String path,
       TracingContext tracingContext,
       ExecutorService executorService) throws IOException,
@@ -97,7 +97,7 @@ public final class TestAbfsOutputStream {
             .withAppendBlob(isAppendBlob)
             .withWriteMaxConcurrentRequestCount(abfsConf.getWriteMaxConcurrentRequestCount())
             .withMaxWriteRequestsToQueue(abfsConf.getMaxWriteRequestsToQueue())
-            .withClient(client)
+            .withClientHandler(clientHandler)
             .withPath(path)
             .withTracingContext(tracingContext)
             .withExecutorService(executorService)
@@ -111,7 +111,8 @@ public final class TestAbfsOutputStream {
   @Test
   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);
     AbfsConfiguration abfsConf;
     final Configuration conf = new Configuration();
@@ -125,6 +126,8 @@ public final class TestAbfsOutputStream {
         .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(
         populateAbfsOutputStreamContext(
@@ -133,7 +136,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             true,
-            client,
+            clientHandler,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
@@ -173,7 +176,8 @@ public final class TestAbfsOutputStream {
   @Test
   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);
     AbfsConfiguration abfsConf;
     final Configuration conf = new Configuration();
@@ -187,6 +191,8 @@ public final class TestAbfsOutputStream {
     when(client.getAbfsPerfTracker()).thenReturn(tracker);
     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(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
 
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
@@ -195,7 +201,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             true,
-            client,
+            clientHandler,
             PATH,
             tracingContext,
             createExecutorService(abfsConf)));
@@ -243,7 +249,8 @@ public final class TestAbfsOutputStream {
   @Test
   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);
     AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class);
     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(op.getSasToken()).thenReturn("testToken");
     when(op.getResult()).thenReturn(httpOp);
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
 
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
@@ -268,7 +277,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             true,
-            client,
+            clientHandler,
             PATH,
             tracingContext,
             createExecutorService(abfsConf)));
@@ -316,7 +325,8 @@ public final class TestAbfsOutputStream {
   @Test
   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);
     AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class);
     AbfsConfiguration abfsConf;
@@ -333,6 +343,8 @@ public final class TestAbfsOutputStream {
         any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
     when(op.getSasToken()).thenReturn("testToken");
     when(op.getResult()).thenReturn(httpOp);
+    when(clientHandler.getClient(any())).thenReturn(client);
+    when(clientHandler.getDfsClient()).thenReturn(client);
 
     AbfsOutputStream out = new AbfsOutputStream(
         populateAbfsOutputStreamContext(
@@ -341,7 +353,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             true,
-            client,
+            clientHandler,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
@@ -375,7 +387,8 @@ public final class TestAbfsOutputStream {
   @Test
   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);
     AbfsConfiguration abfsConf;
     final Configuration conf = new Configuration();
@@ -389,7 +402,8 @@ public final class TestAbfsOutputStream {
         .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(
         populateAbfsOutputStreamContext(
             BUFFER_SIZE,
@@ -397,7 +411,7 @@ public final class TestAbfsOutputStream {
             false,
             true,
             true,
-            client,
+            clientHandler,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
@@ -431,7 +445,8 @@ public final class TestAbfsOutputStream {
   @Test
   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);
     when(op.getSasToken()).thenReturn("");
     AbfsConfiguration abfsConf;
@@ -449,7 +464,8 @@ public final class TestAbfsOutputStream {
         .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(
         populateAbfsOutputStreamContext(
             BUFFER_SIZE,
@@ -457,7 +473,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             true,
-            client,
+            clientHandler,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
@@ -506,7 +522,8 @@ public final class TestAbfsOutputStream {
   @Test
   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);
     AbfsConfiguration abfsConf;
     final Configuration conf = new Configuration();
@@ -519,7 +536,8 @@ public final class TestAbfsOutputStream {
         .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(
         populateAbfsOutputStreamContext(
             BUFFER_SIZE,
@@ -527,7 +545,7 @@ public final class TestAbfsOutputStream {
             false,
             false,
             true,
-            client,
+            clientHandler,
             PATH,
             new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
                 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 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
   public void callTracingHeaderValidator(String tracingContextHeader,
@@ -52,6 +54,8 @@ public class TracingHeaderValidator implements Listener {
         clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId,
         retryNum, streamID);
     tracingHeaderValidator.primaryRequestId = primaryRequestId;
+    tracingHeaderValidator.ingressHandler = ingressHandler;
+    tracingHeaderValidator.position = position;
     return tracingHeaderValidator;
   }
 
@@ -92,8 +96,13 @@ public class TracingHeaderValidator implements Listener {
 
   private void validateBasicFormat(String[] idList) {
     if (format == TracingHeaderFormat.ALL_ID_FORMAT) {
+      int expectedSize = 8;
+      if (ingressHandler != null) {
+        expectedSize += 2;
+      }
       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) {
       Assertions.assertThat(idList)
           .describedAs("header should have 2 elements").hasSize(2);
@@ -152,4 +161,14 @@ public class TracingHeaderValidator implements Listener {
   public void updatePrimaryRequestID(String primaryRequestId) {
     this.primaryRequestId = primaryRequestId;
   }
+
+  @Override
+  public void updateIngressHandler(String ingressHandler) {
+    this.ingressHandler = ingressHandler;
+  }
+
+  @Override
+  public void updatePosition(String position) {
+    this.position = position;
+  }
 }