Explorar el Código

HADOOP-19233: [ABFS][FnsOverBlob][Backport 3.4] Implementing Rename and Delete APIs over Blob Endpoint (#7392)

Contributed by Manish Bhatt.
Signed off by Anuj Modi
Manish Bhatt hace 4 meses
padre
commit
302cf3661f
Se han modificado 46 ficheros con 5037 adiciones y 351 borrados
  1. 67 12
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
  2. 70 58
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
  3. 75 78
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
  4. 5 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
  5. 25 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
  6. 16 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
  7. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
  8. 5 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
  9. 41 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java
  10. 304 22
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
  11. 44 30
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
  12. 33 14
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
  13. 6 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java
  14. 1 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
  15. 145 20
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java
  16. 3 2
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsPermission.java
  17. 1 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
  18. 201 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobDeleteHandler.java
  19. 634 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobRenameHandler.java
  20. 270 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListActionTaker.java
  21. 197 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobQueue.java
  22. 108 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PathInformation.java
  23. 336 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
  24. 58 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenamePendingJsonFormat.java
  25. 15 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java
  26. 38 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java
  27. 13 21
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java
  28. 0 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java
  29. 162 18
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
  30. 5 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java
  31. 313 16
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
  32. 2 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
  33. 2 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
  34. 14 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java
  35. 3 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java
  36. 1448 5
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
  37. 21 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java
  38. 6 9
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java
  39. 49 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
  40. 0 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClientHandler.java
  41. 3 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsPaginatedDelete.java
  42. 2 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestApacheClientConnectionPool.java
  43. 79 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityTestUtils.java
  44. 30 22
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java
  45. 142 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestListActionTaker.java
  46. 23 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java

+ 67 - 12
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java

@@ -21,25 +21,24 @@ package org.apache.hadoop.fs.azurebfs;
 import java.io.IOException;
 import java.lang.reflect.Field;
 
-import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
-import org.apache.hadoop.fs.azurebfs.services.FixedSASTokenProvider;
-import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
-import org.apache.hadoop.fs.azurebfs.utils.MetricFormat;
-import org.apache.hadoop.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.AuthConfigurations;
+import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
 import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
 import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerWithOutlierConfigurationValidatorAnnotation;
 import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
 import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
@@ -65,16 +64,16 @@ import org.apache.hadoop.fs.azurebfs.oauth2.WorkloadIdentityTokenProvider;
 import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy;
+import org.apache.hadoop.fs.azurebfs.services.FixedSASTokenProvider;
 import org.apache.hadoop.fs.azurebfs.services.KeyProvider;
 import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider;
+import org.apache.hadoop.fs.azurebfs.utils.MetricFormat;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
-import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
 import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*;
@@ -399,6 +398,34 @@ public class AbfsConfiguration{
       FS_AZURE_ENABLE_PAGINATED_DELETE, DefaultValue = DEFAULT_ENABLE_PAGINATED_DELETE)
   private boolean isPaginatedDeleteEnabled;
 
+  @LongConfigurationValidatorAnnotation(ConfigurationKey =
+      FS_AZURE_BLOB_COPY_PROGRESS_WAIT_MILLIS, DefaultValue = DEFAULT_AZURE_BLOB_COPY_PROGRESS_WAIT_MILLIS)
+  private long blobCopyProgressPollWaitMillis;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey =
+          FS_AZURE_BLOB_COPY_MAX_WAIT_MILLIS, DefaultValue = DEFAULT_AZURE_BLOB_COPY_MAX_WAIT_MILLIS)
+  private long blobCopyProgressMaxWaitMillis;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey =
+      FS_AZURE_BLOB_ATOMIC_RENAME_LEASE_REFRESH_DURATION, DefaultValue = DEFAULT_AZURE_BLOB_ATOMIC_RENAME_LEASE_REFRESH_DURATION)
+  private long blobAtomicRenameLeaseRefreshDuration;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey =
+      FS_AZURE_PRODUCER_QUEUE_MAX_SIZE, DefaultValue = DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE)
+  private int producerQueueMaxSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey =
+          FS_AZURE_CONSUMER_MAX_LAG, DefaultValue = DEFAULT_FS_AZURE_CONSUMER_MAX_LAG)
+  private int listingMaxConsumptionLag;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey =
+      FS_AZURE_BLOB_DIR_RENAME_MAX_THREAD, DefaultValue = DEFAULT_FS_AZURE_BLOB_RENAME_THREAD)
+  private int blobRenameDirConsumptionParallelism;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey =
+      FS_AZURE_BLOB_DIR_DELETE_MAX_THREAD, DefaultValue = DEFAULT_FS_AZURE_BLOB_DELETE_THREAD)
+  private int blobDeleteDirConsumptionParallelism;
+
   @IntegerConfigurationValidatorAnnotation(ConfigurationKey =
       FS_AZURE_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES, DefaultValue = DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES)
   private int maxApacheHttpClientIoExceptionsRetries;
@@ -1515,4 +1542,32 @@ public class AbfsConfiguration{
   public void setIsChecksumValidationEnabled(boolean isChecksumValidationEnabled) {
     this.isChecksumValidationEnabled = isChecksumValidationEnabled;
   }
+
+  public long getBlobCopyProgressPollWaitMillis() {
+    return blobCopyProgressPollWaitMillis;
+  }
+
+  public long getBlobCopyProgressMaxWaitMillis() {
+    return blobCopyProgressMaxWaitMillis;
+  }
+
+  public long getAtomicRenameLeaseRefreshDuration() {
+    return blobAtomicRenameLeaseRefreshDuration;
+  }
+
+  public int getProducerQueueMaxSize() {
+    return producerQueueMaxSize;
+  }
+
+  public int getListingMaxConsumptionLag() {
+    return listingMaxConsumptionLag;
+  }
+
+  public int getBlobRenameDirConsumptionParallelism() {
+    return blobRenameDirConsumptionParallelism;
+  }
+
+  public int getBlobDeleteDirConsumptionParallelism() {
+    return blobDeleteDirConsumptionParallelism;
+  }
 }

+ 70 - 58
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

@@ -28,10 +28,10 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.nio.file.AccessDeniedException;
 import java.time.Duration;
-import java.util.Hashtable;
-import java.util.List;
 import java.util.ArrayList;
 import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.UUID;
@@ -43,23 +43,15 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 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;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename;
-import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
-import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonPathCapabilities;
@@ -71,27 +63,34 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 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.FileSystemOperationUnhandledException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsCounters;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
 import org.apache.hadoop.fs.azurebfs.services.AbfsLocatedFileStatus;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.azurebfs.utils.Listener;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
 import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
+import org.apache.hadoop.fs.impl.BackReference;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -102,14 +101,16 @@ import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.fs.store.DataBlocks;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.RateLimiting;
-import org.apache.hadoop.util.RateLimitingFactory;
-import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.LambdaUtils;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.RateLimiting;
+import org.apache.hadoop.util.RateLimitingFactory;
+import org.apache.hadoop.util.functional.RemoteIterators;
 
 import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
 import static java.net.HttpURLConnection.HTTP_CONFLICT;
@@ -431,8 +432,12 @@ public class AzureBlobFileSystem extends FileSystem
 
   @Override
   @SuppressWarnings("deprecation")
-  public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission,
-      final boolean overwrite, final int bufferSize, final short replication, final long blockSize,
+  public FSDataOutputStream createNonRecursive(final Path f,
+      final FsPermission permission,
+      final boolean overwrite,
+      final int bufferSize,
+      final short replication,
+      final long blockSize,
       final Progressable progress) throws IOException {
 
     statIncrement(CALL_CREATE_NON_RECURSIVE);
@@ -442,18 +447,21 @@ public class AzureBlobFileSystem extends FileSystem
           ERR_CREATE_ON_ROOT,
           null);
     }
-    final Path parent = f.getParent();
-    TracingContext tracingContext = new TracingContext(clientCorrelationId,
-        fileSystemId, FSOperationType.CREATE_NON_RECURSIVE, tracingHeaderFormat,
-        listener);
-    final FileStatus parentFileStatus = tryGetFileStatus(parent, tracingContext);
-
-    if (parentFileStatus == null) {
-      throw new FileNotFoundException("Cannot create file "
-          + f.getName() + " because parent folder does not exist.");
+    Path qualifiedPath = makeQualified(f);
+    try {
+      TracingContext tracingContext = new TracingContext(clientCorrelationId,
+          fileSystemId, FSOperationType.CREATE_NON_RECURSIVE, tracingHeaderFormat,
+          listener);
+      OutputStream outputStream =  getAbfsStore().createNonRecursive(qualifiedPath, statistics,
+          overwrite,
+          permission == null ? FsPermission.getFileDefault() : permission,
+          FsPermission.getUMask(getConf()), tracingContext);
+      statIncrement(FILES_CREATED);
+      return new FSDataOutputStream(outputStream, statistics);
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
     }
-
-    return create(f, permission, overwrite, bufferSize, replication, blockSize, progress);
   }
 
   @Override
@@ -480,7 +488,10 @@ public class AzureBlobFileSystem extends FileSystem
   @Override
   @SuppressWarnings("deprecation")
   public FSDataOutputStream createNonRecursive(final Path f,
-      final boolean overwrite, final int bufferSize, final short replication, final long blockSize,
+      final boolean overwrite,
+      final int bufferSize,
+      final short replication,
+      final long blockSize,
       final Progressable progress) throws IOException {
     return this.createNonRecursive(f, FsPermission.getFileDefault(),
         overwrite, bufferSize, replication, blockSize, progress);
@@ -530,38 +541,32 @@ public class AzureBlobFileSystem extends FileSystem
       return tryGetFileStatus(qualifiedSrcPath, tracingContext) != null;
     }
 
-    FileStatus dstFileStatus = null;
+    FileStatus dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext);
+    Path adjustedDst = dst;
     if (qualifiedSrcPath.equals(qualifiedDstPath)) {
       // rename to itself
       // - if it doesn't exist, return false
       // - if it is file, return true
       // - if it is dir, return false.
-      dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext);
       if (dstFileStatus == null) {
         return false;
       }
-      return dstFileStatus.isDirectory() ? false : true;
+      return !dstFileStatus.isDirectory();
     }
-
-    // Non-HNS account need to check dst status on driver side.
-    if (!getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) {
-      dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext);
+    // adjust the destination path in case of FNS account.
+    if (!getIsNamespaceEnabled(tracingContext) && dstFileStatus != null) {
+      // return false if the destination is a file.
+      if (!dstFileStatus.isDirectory()) {
+        return false;
+      }
+      String sourceFileName = src.getName();
+      adjustedDst = new Path(dst, sourceFileName);
     }
 
     try {
-      String sourceFileName = src.getName();
-      Path adjustedDst = dst;
-
-      if (dstFileStatus != null) {
-        if (!dstFileStatus.isDirectory()) {
-          return qualifiedSrcPath.equals(qualifiedDstPath);
-        }
-        adjustedDst = new Path(dst, sourceFileName);
-      }
-
       qualifiedDstPath = makeQualified(adjustedDst);
-
-      abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, tracingContext, null);
+      getAbfsStore().rename(qualifiedSrcPath, qualifiedDstPath, tracingContext,
+          null);
       return true;
     } catch (AzureBlobFileSystemException ex) {
       LOG.debug("Rename operation failed. ", ex);
@@ -569,6 +574,8 @@ public class AzureBlobFileSystem extends FileSystem
           src,
           ex,
           AzureServiceErrorCode.PATH_ALREADY_EXISTS,
+          AzureServiceErrorCode.BLOB_ALREADY_EXISTS,
+          AzureServiceErrorCode.BLOB_PATH_NOT_FOUND,
           AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH,
           AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND,
           AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE,
@@ -641,7 +648,7 @@ public class AzureBlobFileSystem extends FileSystem
       final Duration waitTime = rateLimiting.acquire(1);
 
       try {
-        final boolean recovered = abfsStore.rename(qualifiedSrcPath,
+        final boolean recovered = getAbfsStore().rename(qualifiedSrcPath,
             qualifiedDstPath, tracingContext, sourceEtag);
         return Pair.of(recovered, waitTime);
       } catch (AzureBlobFileSystemException ex) {
@@ -655,9 +662,11 @@ public class AzureBlobFileSystem extends FileSystem
   }
 
   @Override
-  public boolean delete(final Path f, final boolean recursive) throws IOException {
+  public boolean delete(final Path f, final boolean recursive)
+      throws IOException {
     LOG.debug(
-        "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive);
+        "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(),
+        recursive);
     statIncrement(CALL_DELETE);
     Path qualifiedPath = makeQualified(f);
 
@@ -673,10 +682,13 @@ public class AzureBlobFileSystem extends FileSystem
       TracingContext tracingContext = new TracingContext(clientCorrelationId,
           fileSystemId, FSOperationType.DELETE, tracingHeaderFormat,
           listener);
-      abfsStore.delete(qualifiedPath, recursive, tracingContext);
+      getAbfsStore().delete(qualifiedPath, recursive, tracingContext);
       return true;
     } catch (AzureBlobFileSystemException ex) {
-      checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND);
+      checkException(f,
+          ex,
+          AzureServiceErrorCode.PATH_NOT_FOUND,
+          AzureServiceErrorCode.BLOB_PATH_NOT_FOUND);
       return false;
     }
 
@@ -693,7 +705,7 @@ public class AzureBlobFileSystem extends FileSystem
       TracingContext tracingContext = new TracingContext(clientCorrelationId,
           fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat,
           listener);
-      FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext);
+      FileStatus[] result = getAbfsStore().listStatus(qualifiedPath, tracingContext);
       return result;
     } catch (AzureBlobFileSystemException ex) {
       checkException(f, ex);

+ 75 - 78
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java

@@ -21,12 +21,11 @@ import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.lang.reflect.InvocationTargetException;
 import java.io.UnsupportedEncodingException;
+import java.lang.reflect.InvocationTargetException;
 import java.net.HttpURLConnection;
 import java.net.MalformedURLException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
@@ -52,35 +51,22 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
-import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
-import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
-import org.apache.hadoop.fs.azurebfs.security.ContextProviderEncryptionAdapter;
-import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
-import org.apache.hadoop.fs.azurebfs.security.NoContextEncryptionAdapter;
-import org.apache.hadoop.fs.azurebfs.services.AbfsClientHandler;
-import org.apache.hadoop.fs.azurebfs.utils.EncryptionType;
-import org.apache.hadoop.fs.impl.BackReference;
-import org.apache.hadoop.fs.PathIOException;
-
-import org.apache.hadoop.util.Preconditions;
-import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
-import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
-import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.EtagSource;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 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;
@@ -90,44 +76,52 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperati
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException;
 import org.apache.hadoop.fs.azurebfs.enums.Trilean;
-import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
+import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
 import org.apache.hadoop.fs.azurebfs.extensions.ExtensionHelper;
+import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.AzureADAuthenticator;
 import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
 import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
+import org.apache.hadoop.fs.azurebfs.security.ContextProviderEncryptionAdapter;
+import org.apache.hadoop.fs.azurebfs.security.NoContextEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientContext;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientContextBuilder;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClientHandler;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientRenameResult;
 import org.apache.hadoop.fs.azurebfs.services.AbfsCounters;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
 import org.apache.hadoop.fs.azurebfs.services.AbfsInputStreamContext;
 import org.apache.hadoop.fs.azurebfs.services.AbfsInputStreamStatisticsImpl;
+import org.apache.hadoop.fs.azurebfs.services.AbfsLease;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamContext;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.azurebfs.services.AbfsPerfInfo;
+import org.apache.hadoop.fs.azurebfs.services.AbfsPerfTracker;
 import org.apache.hadoop.fs.azurebfs.services.AbfsPermission;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy;
-import org.apache.hadoop.fs.azurebfs.services.StaticRetryPolicy;
-import org.apache.hadoop.fs.azurebfs.services.AbfsLease;
-import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials;
-import org.apache.hadoop.fs.azurebfs.services.AbfsPerfTracker;
-import org.apache.hadoop.fs.azurebfs.services.AbfsPerfInfo;
 import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials;
+import org.apache.hadoop.fs.azurebfs.services.StaticRetryPolicy;
 import org.apache.hadoop.fs.azurebfs.utils.Base64;
 import org.apache.hadoop.fs.azurebfs.utils.CRC64;
 import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils;
+import org.apache.hadoop.fs.azurebfs.utils.EncryptionType;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
+import org.apache.hadoop.fs.impl.BackReference;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -136,7 +130,11 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.store.DataBlocks;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture;
 import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.apache.http.client.utils.URIBuilder;
@@ -159,8 +157,10 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_AB
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT;
+import static org.apache.hadoop.fs.azurebfs.utils.UriUtils.isKeyForDirectorySet;
 
 /**
  * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage.
@@ -188,7 +188,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
   private final Map<AbfsLease, Object> leaseRefs;
 
   private final AbfsConfiguration abfsConfiguration;
-  private final Set<String> azureAtomicRenameDirSet;
   private Set<String> azureInfiniteLeaseDirSet;
   private volatile Trilean isNamespaceEnabled;
   private final AuthType authType;
@@ -256,8 +255,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     }
     LOG.trace("primaryUserGroup is {}", this.primaryUserGroup);
 
-    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(
-        abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
     updateInfiniteLeaseDirs();
     this.authType = abfsConfiguration.getAuthType(accountName);
     boolean usingOauth = (authType == AuthType.OAuth);
@@ -642,6 +639,36 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     }
   }
 
+  /**
+   * Checks existence of parent of the given path.
+   *
+   * @param path Path to check.
+   * @param statistics FileSystem statistics.
+   * @param overwrite Overwrite flag.
+   * @param permission Permission of tha path.
+   * @param umask Umask of the path.
+   * @param tracingContext tracing context
+   *
+   * @return OutputStream output stream of the created file.
+   * @throws IOException if there is an issue with the operation.
+   */
+  public OutputStream createNonRecursive(final Path path,
+      final FileSystem.Statistics statistics, final boolean overwrite,
+      final FsPermission permission, final FsPermission umask,
+      TracingContext tracingContext)
+      throws IOException {
+    LOG.debug("CreateNonRecursive for filesystem: {} path: {} overwrite: {} permission: {} umask: {}",
+            getClient().getFileSystem(),
+            path,
+            overwrite,
+            permission,
+            umask);
+    getClient().createNonRecursivePreCheck(path.getParent(),
+        tracingContext);
+   return createFile(path, statistics, overwrite, permission,
+       umask, tracingContext);
+  }
+
   public OutputStream createFile(final Path path,
       final FileSystem.Statistics statistics, final boolean overwrite,
       final FsPermission permission, final FsPermission umask,
@@ -1013,10 +1040,10 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
 
       if (getClient().checkIsDir(op.getResult())) {
         throw new AbfsRestOperationException(
-                AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
-                AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
-                "openFileForWrite must be used with files and not directories",
-                null);
+              AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+              AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+              "openFileForWrite must be used with files and not directories",
+              null);
       }
 
       final long contentLength = extractContentLength(op.getResult());
@@ -1089,21 +1116,15 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
   public boolean rename(final Path source,
       final Path destination,
       final TracingContext tracingContext,
-      final String sourceEtag) throws
-    IOException {
+      final String sourceEtag) throws IOException {
     final Instant startAggregate = abfsPerfTracker.getLatencyInstant();
     long countAggregate = 0;
     boolean shouldContinue;
 
-    if (isAtomicRenameKey(source.getName())) {
-      LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
-              +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
-    }
-
     LOG.debug("renameAsync filesystem: {} source: {} destination: {}",
-            getClient().getFileSystem(),
-            source,
-            destination);
+        getClient().getFileSystem(),
+        source,
+        destination);
 
     String continuation = null;
 
@@ -1114,15 +1135,14 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
 
     do {
       try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) {
-        boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
         final AbfsClientRenameResult abfsClientRenameResult =
             getClient().renamePath(sourceRelativePath, destinationRelativePath,
-                continuation, tracingContext, sourceEtag, false,
-                  isNamespaceEnabled);
+                continuation, tracingContext, sourceEtag, false);
 
         AbfsRestOperation op = abfsClientRenameResult.getOp();
         perfInfo.registerResult(op.getResult());
-        continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+        continuation = op.getResult()
+            .getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
         perfInfo.registerSuccess(true);
         countAggregate++;
         shouldContinue = continuation != null && !continuation.isEmpty();
@@ -1144,9 +1164,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     boolean shouldContinue = true;
 
     LOG.debug("delete filesystem: {} path: {} recursive: {}",
-            getClient().getFileSystem(),
-            path,
-            String.valueOf(recursive));
+        getClient().getFileSystem(),
+        path,
+        String.valueOf(recursive));
 
     String continuation = null;
 
@@ -1155,9 +1175,10 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     do {
       try (AbfsPerfInfo perfInfo = startTracking("delete", "deletePath")) {
         AbfsRestOperation op = getClient().deletePath(relativePath, recursive,
-            continuation, tracingContext, getIsNamespaceEnabled(tracingContext));
+            continuation, tracingContext);
         perfInfo.registerResult(op.getResult());
-        continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+        continuation = op.getResult()
+            .getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
         perfInfo.registerSuccess(true);
         countAggregate++;
         shouldContinue = continuation != null && !continuation.isEmpty();
@@ -1754,10 +1775,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     }
   }
 
-  public boolean isAtomicRenameKey(String key) {
-    return isKeyForDirectorySet(key, azureAtomicRenameDirSet);
-  }
-
   public boolean isInfiniteLeaseKey(String key) {
     if (azureInfiniteLeaseDirSet.isEmpty()) {
       return false;
@@ -1956,27 +1973,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     return properties;
   }
 
-  private boolean isKeyForDirectorySet(String key, Set<String> dirSet) {
-    for (String dir : dirSet) {
-      if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) {
-        return true;
-      }
-
-      try {
-        URI uri = new URI(dir);
-        if (null == uri.getAuthority()) {
-          if (key.startsWith(dir + "/")){
-            return true;
-          }
-        }
-      } catch (URISyntaxException e) {
-        LOG.info("URI syntax error creating URI for {}", dir);
-      }
-    }
-
-    return false;
-  }
-
   private AbfsPerfInfo startTracking(String callerName, String calleeName) {
     return new AbfsPerfInfo(abfsPerfTracker, callerName, calleeName);
   }
@@ -2235,7 +2231,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     if (!enableInfiniteLease) {
       return null;
     }
-    AbfsLease lease = new AbfsLease(getClient(), relativePath, tracingContext);
+    AbfsLease lease = new AbfsLease(getClient(), relativePath, true,
+            INFINITE_LEASE_DURATION, null, tracingContext);
     leaseRefs.put(lease, null);
     return lease;
   }

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

@@ -320,5 +320,10 @@ public final class AbfsHttpConstants {
   public static final String FALLBACK_FLUSH = "FB";
   public static final String FALLBACK_APPEND = "FB";
 
+  public static final String COPY_STATUS_SUCCESS = "success";
+  public static final String COPY_STATUS_PENDING = "pending";
+  public static final String COPY_STATUS_ABORTED = "aborted";
+  public static final String COPY_STATUS_FAILED = "failed";
+
   private AbfsHttpConstants() {}
 }

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

@@ -359,5 +359,30 @@ public final class ConfigurationKeys {
   public static final String FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE = "fs.azure.apache.http.client.max.cache.connection.size";
   /**Maximum idle time for a ApacheHttpClient-connection: {@value}*/
   public static final String FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL = "fs.azure.apache.http.client.idle.connection.ttl";
+  /**
+   * Blob copy API is an async API, this configuration defines polling duration
+   * for checking copy status: {@value}
+   */
+  public static final String FS_AZURE_BLOB_COPY_PROGRESS_WAIT_MILLIS = "fs.azure.blob.copy.progress.wait.millis";
+  /**
+   * Maximum time to wait for a blob copy operation to complete: {@value}
+   */
+  public static final String FS_AZURE_BLOB_COPY_MAX_WAIT_MILLIS = "fs.azure.blob.copy.max.wait.millis";
+  /**Blob rename lease refresh duration: {@value}*/
+  public static final String FS_AZURE_BLOB_ATOMIC_RENAME_LEASE_REFRESH_DURATION
+          = "fs.azure.blob.atomic.rename.lease.refresh.duration";
+  /**Maximum number of blob information enqueued in memory for rename or delete orchestration: {@value}*/
+  public static final String FS_AZURE_PRODUCER_QUEUE_MAX_SIZE = "fs.azure.blob.dir.list.producer.queue.max.size";
+  /**
+   * Maximum consumer lag (count of blob information which is yet to be taken for operation)
+   * in blob listing which can be tolerated before making producer to wait for
+   * consumer lag to become tolerable: {@value}.
+   */
+  public static final String FS_AZURE_CONSUMER_MAX_LAG = "fs.azure.blob.dir.list.consumer.max.lag";
+  /**Maximum number of thread per blob-rename orchestration: {@value}*/
+  public static final String FS_AZURE_BLOB_DIR_RENAME_MAX_THREAD = "fs.azure.blob.dir.rename.max.thread";
+  /**Maximum number of thread per blob-delete orchestration: {@value}*/
+  public static final String FS_AZURE_BLOB_DIR_DELETE_MAX_THREAD = "fs.azure.blob.dir.delete.max.thread";
+
   private ConfigurationKeys() {}
 }

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

@@ -182,5 +182,21 @@ public final class FileSystemConfigurations {
 
   public static final int DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS = 5;
 
+  public static final long DEFAULT_AZURE_BLOB_COPY_PROGRESS_WAIT_MILLIS = 1_000L;
+
+  public static final long DEFAULT_AZURE_BLOB_COPY_MAX_WAIT_MILLIS = 300_000L;
+
+  public static final long DEFAULT_AZURE_BLOB_ATOMIC_RENAME_LEASE_REFRESH_DURATION = 60_000L;
+
+  public static final int DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE = 2 * DEFAULT_AZURE_LIST_MAX_RESULTS;
+
+  public static final int DEFAULT_FS_AZURE_CONSUMER_MAX_LAG = DEFAULT_AZURE_LIST_MAX_RESULTS;
+
+  public static final int DEFAULT_FS_AZURE_LISTING_ACTION_THREADS = 5;
+
+  public static final int DEFAULT_FS_AZURE_BLOB_RENAME_THREAD = DEFAULT_FS_AZURE_LISTING_ACTION_THREADS;
+
+  public static final int DEFAULT_FS_AZURE_BLOB_DELETE_THREAD = DEFAULT_FS_AZURE_LISTING_ACTION_THREADS;
+
   private FileSystemConfigurations() {}
 }

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

@@ -110,5 +110,27 @@ public final class HttpHeaderConfigurations {
    */
   public static final String X_MS_BLOB_TYPE = "x-ms-blob-type";
 
+  /**
+   * Http Request Header for copy id.
+   * {@value}
+   */
+  public static final String X_MS_COPY_ID = "x-ms-copy-id";
+
+    /**
+     * Http Response Header for denoting the copy status description of the operation.
+     * {@value}
+     */
+  public static final String X_MS_COPY_STATUS_DESCRIPTION = "x-ms-copy-status-description";
+
+    /**
+     * Http Response Header for denoting the copy status of the operation.
+     * Expected Values:
+     * {@link AbfsHttpConstants#COPY_STATUS_SUCCESS}
+     * {@link AbfsHttpConstants#COPY_STATUS_FAILED}
+     * {@link AbfsHttpConstants#COPY_STATUS_PENDING}
+     * {@link AbfsHttpConstants#COPY_STATUS_ABORTED}
+     */
+  public static final String X_MS_COPY_STATUS = "x-ms-copy-status";
+
   private HttpHeaderConfigurations() {}
 }

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

@@ -45,6 +45,8 @@ public enum AzureServiceErrorCode {
   INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE("InvalidSourceOrDestinationResourceType", HttpURLConnection.HTTP_CONFLICT, null),
   RENAME_DESTINATION_PARENT_PATH_NOT_FOUND("RenameDestinationParentPathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
   INVALID_RENAME_SOURCE_PATH("InvalidRenameSourcePath", HttpURLConnection.HTTP_CONFLICT, null),
+  NON_EMPTY_DIRECTORY_DELETE("DirectoryNotEmpty", HttpURLConnection.HTTP_CONFLICT,
+          "The recursive query parameter value must be true to delete a non-empty directory"),
   INGRESS_OVER_ACCOUNT_LIMIT("ServerBusy", HttpURLConnection.HTTP_UNAVAILABLE,
           "Ingress is over the account limit."),
   EGRESS_OVER_ACCOUNT_LIMIT("ServerBusy", HttpURLConnection.HTTP_UNAVAILABLE,
@@ -54,10 +56,13 @@ public enum AzureServiceErrorCode {
   OTHER_SERVER_THROTTLING("ServerBusy", HttpURLConnection.HTTP_UNAVAILABLE,
           "The server is currently unable to receive requests. Please retry your request."),
   INVALID_QUERY_PARAMETER_VALUE("InvalidQueryParameterValue", HttpURLConnection.HTTP_BAD_REQUEST, null),
+  INVALID_RENAME_DESTINATION("InvalidRenameDestinationPath", HttpURLConnection.HTTP_BAD_REQUEST, null),
   AUTHORIZATION_PERMISSION_MISS_MATCH("AuthorizationPermissionMismatch", HttpURLConnection.HTTP_FORBIDDEN, null),
   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."),
+  COPY_BLOB_FAILED("CopyBlobFailed", HttpURLConnection.HTTP_INTERNAL_ERROR, null),
+  COPY_BLOB_ABORTED("CopyBlobAborted", HttpURLConnection.HTTP_INTERNAL_ERROR, null),
   BLOB_OPERATION_NOT_SUPPORTED("BlobOperationNotSupported", HttpURLConnection.HTTP_CONFLICT, null),
   INVALID_APPEND_OPERATION("InvalidAppendOperation", HttpURLConnection.HTTP_CONFLICT, null),
   UNKNOWN(null, -1, null);

+ 41 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java

@@ -0,0 +1,41 @@
+/**
+ * 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.enums;
+
+/**
+ * Enum for BlobCopyProgress.
+ */
+public enum BlobCopyProgress {
+  /**
+   * Copy is successful.
+   */
+  SUCCESS,
+  /**
+   * Copy is failed.
+   */
+  FAILURE,
+  /**
+   * Copy is aborted.
+   */
+  ABORTED,
+  /**
+   * Copy is pending.
+   */
+  PENDING;
+}

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

@@ -22,6 +22,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.parsers.SAXParser;
 import javax.xml.parsers.SAXParserFactory;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.UnsupportedEncodingException;
@@ -33,6 +34,8 @@ import java.nio.charset.Charset;
 import java.nio.charset.CharsetEncoder;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.Hashtable;
 import java.util.List;
 import java.util.Map;
@@ -45,13 +48,14 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.NotImplementedException;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
 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.FSOperationType;
 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;
@@ -72,10 +76,13 @@ 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_INTERNAL_ERROR;
 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.AbfsStatistic.CALL_GET_FILE_STATUS;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ACQUIRE_LEASE_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.AND_MARK;
 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;
@@ -153,12 +160,18 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARA
 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;
+import static org.apache.hadoop.fs.azurebfs.utils.UriUtils.isKeyForDirectorySet;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ATOMIC_DIR_RENAME_RECOVERY_ON_GET_PATH_EXCEPTION;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_DELETE_BLOB;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_RENAME_BLOB;
 
 /**
  * AbfsClient interacting with Blob endpoint.
  */
 public class AbfsBlobClient extends AbfsClient {
 
+  private final HashSet<String> azureAtomicRenameDirSet;
+
   public AbfsBlobClient(final URL baseUrl,
       final SharedKeyCredentials sharedKeyCredentials,
       final AbfsConfiguration abfsConfiguration,
@@ -167,6 +180,9 @@ public class AbfsBlobClient extends AbfsClient {
       final AbfsClientContext abfsClientContext) throws IOException {
     super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider,
         encryptionContextProvider, abfsClientContext);
+    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(
+        abfsConfiguration.getAzureAtomicRenameDirs()
+            .split(AbfsHttpConstants.COMMA)));
   }
 
   public AbfsBlobClient(final URL baseUrl,
@@ -177,6 +193,9 @@ public class AbfsBlobClient extends AbfsClient {
       final AbfsClientContext abfsClientContext) throws IOException {
     super(baseUrl, sharedKeyCredentials, abfsConfiguration, sasTokenProvider,
         encryptionContextProvider, abfsClientContext);
+    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(
+        abfsConfiguration.getAzureAtomicRenameDirs()
+            .split(AbfsHttpConstants.COMMA)));
   }
 
   /**
@@ -351,6 +370,8 @@ public class AbfsBlobClient extends AbfsClient {
         requestHeaders);
 
     op.execute(tracingContext);
+    // Filter the paths for which no rename redo operation is performed.
+    fixAtomicEntriesInListResults(op, tracingContext);
     if (isEmptyListResults(op.getResult()) && is404CheckRequired) {
       // If the list operation returns no paths, we need to check if the path is a file.
       // If it is a file, we need to return the file in the list.
@@ -372,6 +393,65 @@ public class AbfsBlobClient extends AbfsClient {
     return op;
   }
 
+  /**
+   * Filter the paths for which no rename redo operation is performed.
+   * Update BlobListResultSchema path with filtered entries.
+   *
+   * @param op blob list operation
+   * @param tracingContext tracing context
+   * @throws AzureBlobFileSystemException if rest operation or response parsing fails.
+   */
+  private void fixAtomicEntriesInListResults(final AbfsRestOperation op,
+                                             final TracingContext tracingContext) throws AzureBlobFileSystemException {
+    /*
+     * Crashed HBase log rename recovery is done by Filesystem.getFileStatus and
+     * Filesystem.listStatus.
+     */
+    if (tracingContext == null
+        || tracingContext.getOpType() != FSOperationType.LISTSTATUS
+        || op == null || op.getResult() == null
+        || op.getResult().getStatusCode() != HTTP_OK) {
+      return;
+    }
+    BlobListResultSchema listResultSchema
+        = (BlobListResultSchema) op.getResult().getListResultSchema();
+    if (listResultSchema == null) {
+      return;
+    }
+    List<BlobListResultEntrySchema> filteredEntries = new ArrayList<>();
+    for (BlobListResultEntrySchema entry : listResultSchema.paths()) {
+      if (!takeListPathAtomicRenameKeyAction(entry.path(),
+          entry.contentLength().intValue(), tracingContext)) {
+        filteredEntries.add(entry);
+      }
+    }
+
+    listResultSchema.withPaths(filteredEntries);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void createNonRecursivePreCheck(Path parentPath,
+      TracingContext tracingContext)
+      throws IOException {
+    try {
+      if (isAtomicRenameKey(parentPath.toUri().getPath())) {
+        takeGetPathStatusAtomicRenameKeyAction(parentPath, tracingContext);
+      }
+      getPathStatus(parentPath.toUri().getPath(), false,
+          tracingContext, null);
+    } catch (AbfsRestOperationException ex) {
+      if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
+        throw new FileNotFoundException("Cannot create file "
+            + parentPath.toUri().getPath()
+            + " because parent folder does not exist.");
+      }
+      throw ex;
+    } finally {
+      getAbfsCounters().incrementCounter(CALL_GET_FILE_STATUS, 1);
+    }
+  }
+
   /**
    * Get Rest Operation for API
    * <a href="../../../../site/markdown/blobEndpoint.md#put-blob">Put Blob</a>.
@@ -577,12 +657,17 @@ public class AbfsBlobClient extends AbfsClient {
    * @throws AzureBlobFileSystemException if rest operation fails.
    */
   @Override
-  public AbfsRestOperation acquireLease(final String path, final int duration,
+  public AbfsRestOperation acquireLease(final String path,
+      final int duration,
+      final String eTag,
       TracingContext tracingContext) throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION));
     requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_DURATION, Integer.toString(duration)));
     requestHeaders.add(new AbfsHttpHeader(X_MS_PROPOSED_LEASE_ID, UUID.randomUUID().toString()));
+    if (StringUtils.isNotEmpty(eTag)) {
+      requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
+    }
 
     final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
     abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE);
@@ -680,7 +765,15 @@ public class AbfsBlobClient extends AbfsClient {
   }
 
   /**
-   * Get results for the rename operation.
+   * Rename a file or directory.
+   * If a source etag is passed in, the operation will attempt to recover
+   * from a missing source file by probing the destination for
+   * existence and comparing etags.
+   * The second value in the result will be true to indicate that this
+   * took place.
+   * As rename recovery is only attempted if the source etag is non-empty,
+   * in normal rename operations rename recovery will never happen.
+   *
    * @param source                    path to source file
    * @param destination               destination of rename.
    * @param continuation              continuation.
@@ -688,22 +781,52 @@ public class AbfsBlobClient extends AbfsClient {
    * @param sourceEtag                etag of source file. may be null or empty
    * @param isMetadataIncompleteState was there a rename failure due to
    *                                  incomplete metadata state?
-   * @param isNamespaceEnabled        whether namespace enabled account or not
-   * @return result of rename operation
-   * @throws IOException if rename operation fails.
+   *
+   * @return AbfsClientRenameResult result of rename operation indicating the
+   * AbfsRest operation, rename recovery and incomplete metadata state failure.
+   *
+   * @throws IOException failure, excluding any recovery from overload failures.
    */
   @Override
   public AbfsClientRenameResult renamePath(final String source,
       final String destination,
       final String continuation,
       final TracingContext tracingContext,
+      String sourceEtag,
+      boolean isMetadataIncompleteState)
+      throws IOException {
+    BlobRenameHandler blobRenameHandler = getBlobRenameHandler(source,
+        destination, sourceEtag, isAtomicRenameKey(source), tracingContext
+    );
+    incrementAbfsRenamePath();
+    if (blobRenameHandler.execute()) {
+      final AbfsUriQueryBuilder abfsUriQueryBuilder
+          = createDefaultUriQueryBuilder();
+      final URL url = createRequestUrl(destination,
+          abfsUriQueryBuilder.toString());
+      final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+      final AbfsRestOperation successOp = getAbfsRestOperation(
+          AbfsRestOperationType.RenamePath, HTTP_METHOD_PUT,
+          url, requestHeaders);
+      successOp.hardSetResult(HttpURLConnection.HTTP_OK);
+      return new AbfsClientRenameResult(successOp, true, false);
+    } else {
+      throw new AbfsRestOperationException(HTTP_INTERNAL_ERROR,
+          AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+          ERR_RENAME_BLOB + source + SINGLE_WHITE_SPACE + AND_MARK
+              + SINGLE_WHITE_SPACE + destination,
+          null);
+    }
+  }
+
+  @VisibleForTesting
+  BlobRenameHandler getBlobRenameHandler(final String source,
+      final String destination,
       final String sourceEtag,
-      final boolean isMetadataIncompleteState,
-      final boolean isNamespaceEnabled) throws IOException {
-    /**
-     * TODO: [FnsOverBlob] To be implemented as part of rename-delete over blob endpoint work. <a href="https://issues.apache.org/jira/browse/HADOOP-19233">HADOOP-19233</a>.
-     */
-    throw new NotImplementedException("Rename operation on Blob endpoint yet to be implemented.");
+      final boolean isAtomicRename,
+      final TracingContext tracingContext) {
+    return new BlobRenameHandler(source,
+        destination, this, sourceEtag, isAtomicRename, false, tracingContext);
   }
 
   /**
@@ -1024,9 +1147,17 @@ public class AbfsBlobClient extends AbfsClient {
       final TracingContext tracingContext,
       final ContextEncryptionAdapter contextEncryptionAdapter)
       throws AzureBlobFileSystemException {
-    return this.getPathStatus(path, tracingContext,
+    AbfsRestOperation op = this.getPathStatus(path, tracingContext,
         contextEncryptionAdapter, true);
-
+    /*
+     * Crashed HBase log-folder rename can be recovered by FileSystem#getFileStatus
+     * and FileSystem#listStatus calls.
+     */
+    if (tracingContext.getOpType() == FSOperationType.GET_FILESTATUS
+        && op.getResult() != null && checkIsDir(op.getResult())) {
+      takeGetPathStatusAtomicRenameKeyAction(new Path(path), tracingContext);
+    }
+    return op;
   }
 
   /**
@@ -1147,12 +1278,12 @@ public class AbfsBlobClient extends AbfsClient {
   }
 
   /**
-   * Orchestration for delete operation to be implemented.
+   * Orchestration of delete path over Blob endpoint.
+   * Delete the file or directory at specified path.
    * @param path to be deleted.
    * @param recursive if the path is a directory, delete recursively.
    * @param continuation to specify continuation token.
-   * @param tracingContext for tracing the server calls.
-   * @param isNamespaceEnabled specify if the namespace is enabled.
+   * @param tracingContext TracingContext instance to track identifiers
    * @return executed rest operation containing response from server.
    * @throws AzureBlobFileSystemException if rest operation fails.
    */
@@ -1160,10 +1291,33 @@ public class AbfsBlobClient extends AbfsClient {
   public AbfsRestOperation deletePath(final String path,
       final boolean recursive,
       final String continuation,
-      TracingContext tracingContext,
-      final boolean isNamespaceEnabled) throws AzureBlobFileSystemException {
-    // TODO: [FnsOverBlob][HADOOP-19233] To be implemented as part of rename-delete over blob endpoint work.
-    throw new NotImplementedException("Delete operation on Blob endpoint will be implemented in future.");
+      final TracingContext tracingContext) throws AzureBlobFileSystemException {
+    BlobDeleteHandler blobDeleteHandler = getBlobDeleteHandler(path, recursive,
+        tracingContext);
+    if (blobDeleteHandler.execute()) {
+      final AbfsUriQueryBuilder abfsUriQueryBuilder
+          = createDefaultUriQueryBuilder();
+      final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+      final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+      final AbfsRestOperation successOp = getAbfsRestOperation(
+          AbfsRestOperationType.DeletePath, HTTP_METHOD_DELETE,
+          url, requestHeaders);
+      successOp.hardSetResult(HttpURLConnection.HTTP_OK);
+      return successOp;
+    } else {
+      throw new AbfsRestOperationException(HTTP_INTERNAL_ERROR,
+          AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+          ERR_DELETE_BLOB + path,
+          null);
+    }
+  }
+
+  @VisibleForTesting
+  public BlobDeleteHandler getBlobDeleteHandler(final String path,
+      final boolean recursive,
+      final TracingContext tracingContext) {
+    return new BlobDeleteHandler(new Path(path), recursive, this,
+        tracingContext);
   }
 
   /**
@@ -1344,7 +1498,8 @@ public class AbfsBlobClient extends AbfsClient {
     String blobRelativePath = blobPath.toUri().getPath();
     appendSASTokenToQuery(blobRelativePath,
         SASTokenProvider.FIXED_SAS_STORE_OPERATION, abfsUriQueryBuilder);
-    final URL url = createRequestUrl(blobRelativePath, abfsUriQueryBuilder.toString());
+    final URL url = createRequestUrl(blobRelativePath,
+        abfsUriQueryBuilder.toString());
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     if (leaseId != null) {
       requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
@@ -1556,6 +1711,133 @@ public class AbfsBlobClient extends AbfsClient {
     return directory;
   }
 
+  /**
+   * Check if the path is present in the set of atomic rename keys.
+   * @param key path to be checked.
+   * @return true if path is present in the set else false.
+   */
+  public boolean isAtomicRenameKey(String key) {
+    return isKeyForDirectorySet(key, azureAtomicRenameDirSet);
+  }
+
+  /**
+   * Action to be taken when atomic-key is present on a getPathStatus path.
+   *
+   * @param path path of the pendingJson for the atomic path.
+   * @param tracingContext tracing context.
+   *
+   * @throws AzureBlobFileSystemException server error or the path is renamePending json file and action is taken.
+   */
+  public void takeGetPathStatusAtomicRenameKeyAction(final Path path,
+      final TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    if (path == null || path.isRoot() || !isAtomicRenameKey(
+        path.toUri().getPath())) {
+      return;
+    }
+    AbfsRestOperation pendingJsonFileStatus;
+    Path pendingJsonPath = new Path(path.getParent(),
+        path.toUri().getPath() + RenameAtomicity.SUFFIX);
+    try {
+      pendingJsonFileStatus = getPathStatus(
+          pendingJsonPath.toUri().getPath(), tracingContext,
+          null, false);
+      if (checkIsDir(pendingJsonFileStatus.getResult())) {
+        return;
+      }
+    } catch (AbfsRestOperationException ex) {
+      if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
+        return;
+      }
+      throw ex;
+    }
+
+    boolean renameSrcHasChanged;
+    try {
+      RenameAtomicity renameAtomicity = getRedoRenameAtomicity(
+          pendingJsonPath, Integer.parseInt(pendingJsonFileStatus.getResult()
+              .getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)),
+          tracingContext);
+      renameAtomicity.redo();
+      renameSrcHasChanged = false;
+    } catch (AbfsRestOperationException ex) {
+      /*
+       * At this point, the source marked by the renamePending json file, might have
+       * already got renamed by some parallel thread, or at this point, the path
+       * would have got modified which would result in eTag change, which would lead
+       * to a HTTP_CONFLICT. In this case, no more operation needs to be taken, and
+       * the calling getPathStatus can return this source path as result.
+       */
+      if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND
+          || ex.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) {
+        renameSrcHasChanged = true;
+      } else {
+        throw ex;
+      }
+    }
+    if (!renameSrcHasChanged) {
+      throw new AbfsRestOperationException(
+          AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+          AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+          ATOMIC_DIR_RENAME_RECOVERY_ON_GET_PATH_EXCEPTION,
+          null);
+    }
+  }
+
+  /**
+   * Redo the rename operation when path is present in atomic directory list
+   * or when path has {@link RenameAtomicity#SUFFIX} suffix.
+   *
+   * @param path path of the pendingJson for the atomic path.
+   * @param renamePendingJsonLen length of the pendingJson file.
+   * @param tracingContext tracing context.
+   *
+   * @return true if action is taken.
+   * @throws AzureBlobFileSystemException server error
+   */
+  private boolean takeListPathAtomicRenameKeyAction(final Path path,
+      final int renamePendingJsonLen,
+      final TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    if (path == null || path.isRoot() || !isAtomicRenameKey(
+        path.toUri().getPath()) || !path.toUri()
+        .getPath()
+        .endsWith(RenameAtomicity.SUFFIX)) {
+      return false;
+    }
+    try {
+      RenameAtomicity renameAtomicity
+          = getRedoRenameAtomicity(path, renamePendingJsonLen,
+          tracingContext);
+      renameAtomicity.redo();
+    } catch (AbfsRestOperationException ex) {
+      /*
+       * At this point, the source marked by the renamePending json file, might have
+       * already got renamed by some parallel thread, or at this point, the path
+       * would have got modified which would result in eTag change, which would lead
+       * to a HTTP_CONFLICT. In this case, no more operation needs to be taken, but
+       * since this is a renamePendingJson file and would be deleted by the redo operation,
+       * the calling listPath should not return this json path as result.
+       */
+      if (ex.getStatusCode() != HttpURLConnection.HTTP_NOT_FOUND
+          && ex.getStatusCode() != HttpURLConnection.HTTP_CONFLICT) {
+        throw ex;
+      }
+    }
+    return true;
+  }
+
+  @VisibleForTesting
+  RenameAtomicity getRedoRenameAtomicity(final Path renamePendingJsonPath,
+      int fileLen,
+      final TracingContext tracingContext) {
+    return new RenameAtomicity(renamePendingJsonPath,
+        fileLen,
+        tracingContext,
+        null,
+        this);
+  }
+
   /**
    * Checks if the value contains pure ASCII characters or not.
    * @param value to be checked.

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

@@ -42,25 +42,42 @@ import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
-import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ApiVersion;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+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.AbfsInvalidChecksumException;
+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.InvalidAbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
 import org.apache.hadoop.fs.azurebfs.contracts.services.StorageErrorResponseSchema;
-import org.apache.hadoop.fs.azurebfs.utils.MetricFormat;
-import org.apache.hadoop.fs.store.LogExactlyOnce;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions;
 import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
+import org.apache.hadoop.fs.azurebfs.extensions.ExtensionHelper;
+import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
+import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.utils.EncryptionType;
+import org.apache.hadoop.fs.azurebfs.utils.MetricFormat;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
 import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
@@ -70,23 +87,6 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningS
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.util.Preconditions;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ApiVersion;
-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.InvalidUriException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException;
-import org.apache.hadoop.fs.azurebfs.extensions.ExtensionHelper;
-import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
-import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
-import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
-import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
-import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
-import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
@@ -203,7 +203,9 @@ public abstract class AbfsClient implements Closeable {
 
     if (encryptionContextProvider != null) {
       this.encryptionContextProvider = encryptionContextProvider;
-      xMsVersion = ApiVersion.APR_10_2021; // will be default once server change deployed
+      // Version update needed to support x-ms-encryption-context header
+      // @link https://learn.microsoft.com/en-us/rest/api/storageservices/put-block?tabs=microsoft-entra-id}
+      xMsVersion = ApiVersion.AUG_03_2023; // will be default once server change deployed
       encryptionType = EncryptionType.ENCRYPTION_CONTEXT;
     } else if (abfsConfiguration.getEncodedClientProvidedEncryptionKey() != null) {
       clientProvidedEncryptionKey =
@@ -554,15 +556,31 @@ public abstract class AbfsClient implements Closeable {
       ContextEncryptionAdapter contextEncryptionAdapter,
       TracingContext tracingContext) throws AzureBlobFileSystemException;
 
+  /**
+   * Performs a pre-check for a createNonRecursivePreCheck operation. Checks if parentPath
+   * exists or not.
+   *
+   * @param parentPath parent path of the file to be created.
+   * @param tracingContext trace context
+   *
+   * @throws IOException if parentPath does not exist or server error.
+   */
+  public abstract void createNonRecursivePreCheck(Path parentPath,
+      TracingContext tracingContext)
+      throws IOException;
+
   /**
    * Acquire lease on specified path.
    * @param path on which lease has to be acquired.
    * @param duration for which lease has to be acquired.
+   * @param eTag required to acquire lease on the path.
    * @param tracingContext for tracing the server calls.
    * @return executed rest operation containing response from server.
    * @throws AzureBlobFileSystemException if rest operation fails.
    */
-  public abstract AbfsRestOperation acquireLease(String path, int duration,
+  public abstract AbfsRestOperation acquireLease(String path,
+      int duration,
+      String eTag,
       TracingContext tracingContext) throws AzureBlobFileSystemException;
 
   /**
@@ -614,7 +632,6 @@ public abstract class AbfsClient implements Closeable {
    * @param sourceEtag                etag of source file. may be null or empty
    * @param isMetadataIncompleteState was there a rename failure due to
    *                                  incomplete metadata state?
-   * @param isNamespaceEnabled        whether namespace enabled account or not
    * @return AbfsClientRenameResult result of rename operation indicating the
    * AbfsRest operation, rename recovery and incomplete metadata state failure.
    * @throws AzureBlobFileSystemException failure, excluding any recovery from overload failures.
@@ -625,8 +642,7 @@ public abstract class AbfsClient implements Closeable {
       String continuation,
       TracingContext tracingContext,
       String sourceEtag,
-      boolean isMetadataIncompleteState,
-      boolean isNamespaceEnabled)
+      boolean isMetadataIncompleteState)
       throws IOException;
 
   /**
@@ -875,14 +891,12 @@ public abstract class AbfsClient implements Closeable {
    * @param recursive if the path is a directory, delete recursively.
    * @param continuation to specify continuation token.
    * @param tracingContext for tracing the server calls.
-   * @param isNamespaceEnabled specify if the namespace is enabled.
    * @return executed rest operation containing response from server.
    * @throws AzureBlobFileSystemException if rest operation fails.
    */
   public abstract AbfsRestOperation deletePath(String path, boolean recursive,
       String continuation,
-      TracingContext tracingContext,
-      boolean isNamespaceEnabled)
+      TracingContext tracingContext)
       throws AzureBlobFileSystemException;
 
   /**

+ 33 - 14
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.azurebfs.services;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.UnsupportedEncodingException;
@@ -39,6 +40,7 @@ import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
+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;
@@ -63,6 +65,7 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.util.StringUtils;
 
 import static org.apache.commons.lang3.StringUtils.isEmpty;
+import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_GET_FILE_STATUS;
 import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ACQUIRE_LEASE_ACTION;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
@@ -409,6 +412,25 @@ public class AbfsDfsClient extends AbfsClient {
     return op;
   }
 
+  /** {@inheritDoc} */
+  public void createNonRecursivePreCheck(Path parentPath,
+      TracingContext tracingContext)
+      throws IOException {
+    try {
+      getPathStatus(parentPath.toUri().getPath(), false,
+          tracingContext, null);
+    } catch (AbfsRestOperationException ex) {
+      if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
+        throw new FileNotFoundException("Cannot create file "
+            + parentPath.toUri().getPath()
+            + " because parent folder does not exist.");
+      }
+      throw ex;
+    } finally {
+      getAbfsCounters().incrementCounter(CALL_GET_FILE_STATUS, 1);
+    }
+  }
+
   /**
    * Get Rest Operation for API
    * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/lease">
@@ -421,8 +443,10 @@ public class AbfsDfsClient extends AbfsClient {
    * @throws AzureBlobFileSystemException if rest operation fails.
    */
   @Override
-  public AbfsRestOperation acquireLease(final String path, final int duration,
-      TracingContext tracingContext) throws AzureBlobFileSystemException {
+  public AbfsRestOperation acquireLease(final String path,
+                                        final int duration,
+                                        final String eTag,
+                                        TracingContext tracingContext) throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
     requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION));
     requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_DURATION, Integer.toString(duration)));
@@ -532,8 +556,7 @@ public class AbfsDfsClient extends AbfsClient {
    * @param continuation continuation.
    * @param tracingContext for tracing the server calls.
    * @param sourceEtag etag of source file. may be null or empty
-   * @param isMetadataIncompleteState was there a rename failure due to incomplete metadata state?
-   * @param isNamespaceEnabled whether namespace enabled account or not
+   * @param isMetadataIncompleteState was there a rename failure due to incomplete metadata state
    * @return executed rest operation containing response from server.
    * @throws IOException if rest operation fails.
    */
@@ -544,13 +567,12 @@ public class AbfsDfsClient extends AbfsClient {
       final String continuation,
       final TracingContext tracingContext,
       String sourceEtag,
-      boolean isMetadataIncompleteState,
-      boolean isNamespaceEnabled) throws IOException {
+      boolean isMetadataIncompleteState) throws IOException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
 
     final boolean hasEtag = !isEmpty(sourceEtag);
 
-    boolean shouldAttemptRecovery = isRenameResilience() && isNamespaceEnabled;
+    boolean shouldAttemptRecovery = isRenameResilience() && getIsNamespaceEnabled();
     if (!hasEtag && shouldAttemptRecovery) {
       // in case eTag is already not supplied to the API
       // and rename resilience is expected and it is an HNS enabled account
@@ -638,8 +660,7 @@ public class AbfsDfsClient extends AbfsClient {
           sourceEtagAfterFailure = extractEtagHeader(sourceStatusResult);
         }
         renamePath(source, destination, continuation, tracingContext,
-            sourceEtagAfterFailure, isMetadataIncompleteState,
-            isNamespaceEnabled);
+            sourceEtagAfterFailure, isMetadataIncompleteState);
       }
       // if we get out of the condition without a successful rename, then
       // it isn't metadata incomplete state issue.
@@ -1017,7 +1038,6 @@ public class AbfsDfsClient extends AbfsClient {
    * @param recursive if the path is a directory, delete recursively.
    * @param continuation to specify continuation token.
    * @param tracingContext for tracing the server calls.
-   * @param isNamespaceEnabled specify if the namespace is enabled.
    * @return executed rest operation containing response from server.
    * @throws AzureBlobFileSystemException if rest operation fails.
    */
@@ -1025,8 +1045,7 @@ public class AbfsDfsClient extends AbfsClient {
   public AbfsRestOperation deletePath(final String path,
       final boolean recursive,
       final String continuation,
-      TracingContext tracingContext,
-      final boolean isNamespaceEnabled) throws AzureBlobFileSystemException {
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
     /*
      * If Pagination is enabled and current API version is old,
      * use the minimum required version for pagination.
@@ -1035,14 +1054,14 @@ public class AbfsDfsClient extends AbfsClient {
      * If pagination is disabled, use the current API version only.
      */
     final List<AbfsHttpHeader> requestHeaders = (isPaginatedDelete(recursive,
-        isNamespaceEnabled) && getxMsVersion().compareTo(
+        getIsNamespaceEnabled()) && getxMsVersion().compareTo(
         ApiVersion.AUG_03_2023) < 0)
         ? createDefaultHeaders(ApiVersion.AUG_03_2023)
         : createDefaultHeaders();
     final AbfsUriQueryBuilder abfsUriQueryBuilder
         = createDefaultUriQueryBuilder();
 
-    if (isPaginatedDelete(recursive, isNamespaceEnabled)) {
+    if (isPaginatedDelete(recursive, getIsNamespaceEnabled())) {
       // Add paginated query parameter
       abfsUriQueryBuilder.addQuery(QUERY_PARAM_PAGINATED, TRUE);
     }

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

@@ -62,5 +62,11 @@ public final class AbfsErrors {
    * 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.";
+  public static final String ERR_RENAME_BLOB =
+      "FNS-Blob rename was not successful for source and destination path: ";
+  public static final String ERR_DELETE_BLOB =
+      "FNS-Blob delete was not successful for path: ";
+  public static final String ATOMIC_DIR_RENAME_RECOVERY_ON_GET_PATH_EXCEPTION =
+      "Path had to be recovered from atomic rename operation.";
   private AbfsErrors() {}
 }

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

@@ -59,7 +59,7 @@ public abstract class AbfsHttpOperation implements AbfsPerfLoggable {
 
   private static final int CLEAN_UP_BUFFER_SIZE = 64 * 1024;
 
-  private static final int ONE_THOUSAND = 1000;
+  public static final int ONE_THOUSAND = 1000;
 
   private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND;
 

+ 145 - 20
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java

@@ -19,27 +19,29 @@
 package org.apache.hadoop.fs.azurebfs.services;
 
 import java.io.IOException;
+import java.util.Timer;
+import java.util.TimerTask;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback;
-import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture;
-import org.apache.hadoop.thirdparty.org.checkerframework.checker.nullness.qual.Nullable;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture;
+import org.apache.hadoop.thirdparty.org.checkerframework.checker.nullness.qual.Nullable;
 
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION;
 import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE;
 import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_FUTURE_EXISTS;
 import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation.ONE_THOUSAND;
 
 /**
  * AbfsLease manages an Azure blob lease. It acquires an infinite lease on instantiation and
@@ -70,6 +72,11 @@ public final class AbfsLease {
   private volatile Throwable exception = null;
   private volatile int acquireRetryCount = 0;
   private volatile ListenableScheduledFuture<AbfsRestOperation> future = null;
+  private final long leaseRefreshDuration;
+  private final int leaseRefreshDurationInSeconds;
+  private final Timer timer;
+  private LeaseTimerTask leaseTimerTask;
+  private final boolean isAsync;
 
   public static class LeaseException extends AzureBlobFileSystemException {
     public LeaseException(Throwable t) {
@@ -81,27 +88,60 @@ public final class AbfsLease {
     }
   }
 
-  public AbfsLease(AbfsClient client, String path, TracingContext tracingContext) throws AzureBlobFileSystemException {
-    this(client, path, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES,
-        DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, tracingContext);
+    /**
+     * Create a new lease object and acquire a lease on the given path.
+     *
+     * @param client              AbfsClient
+     * @param path                Path to acquire lease on
+     * @param isAsync             Whether to acquire lease asynchronously
+     * @param leaseRefreshDuration Duration in milliseconds to renew the lease
+     * @param eTag                ETag of the file
+     * @param tracingContext      Tracing context
+     * @throws AzureBlobFileSystemException if the lease cannot be acquired
+     */
+  public AbfsLease(AbfsClient client, String path,
+                   final boolean isAsync, final long leaseRefreshDuration,
+                   final String eTag, TracingContext tracingContext) throws AzureBlobFileSystemException {
+    this(client, path, isAsync, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES,
+            DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, leaseRefreshDuration, eTag, tracingContext);
   }
 
+  /**
+   * Create a new lease object and acquire a lease on the given path.
+   *
+   * @param client              AbfsClient
+   * @param path                Path to acquire lease on
+   * @param isAsync             Whether to acquire lease asynchronously
+   * @param acquireMaxRetries   Maximum number of retries to acquire lease
+   * @param acquireRetryInterval Retry interval in seconds to acquire lease
+   * @param leaseRefreshDuration Duration in milliseconds to renew the lease
+   * @param eTag                ETag of the file
+   * @param tracingContext      Tracing context
+   * @throws AzureBlobFileSystemException if the lease cannot be acquired
+   */
   @VisibleForTesting
-  public AbfsLease(AbfsClient client, String path, int acquireMaxRetries,
-      int acquireRetryInterval, TracingContext tracingContext) throws AzureBlobFileSystemException {
+  public AbfsLease(AbfsClient client, String path, final boolean isAsync, int acquireMaxRetries,
+                   int acquireRetryInterval, final long leaseRefreshDuration,
+                   final String eTag,
+                   TracingContext tracingContext) throws AzureBlobFileSystemException {
     this.leaseFreed = false;
     this.client = client;
     this.path = path;
     this.tracingContext = tracingContext;
+    this.leaseRefreshDuration = leaseRefreshDuration;
+    this.leaseRefreshDurationInSeconds = (int) leaseRefreshDuration / ONE_THOUSAND;
+    this.isAsync = isAsync;
 
-    if (client.getNumLeaseThreads() < 1) {
+    if (isAsync && client.getNumLeaseThreads() < 1) {
       throw new LeaseException(ERR_NO_LEASE_THREADS);
     }
 
     // Try to get the lease a specified number of times, else throw an error
     RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
         acquireMaxRetries, acquireRetryInterval, TimeUnit.SECONDS);
-    acquireLease(retryPolicy, 0, acquireRetryInterval, 0,
+    this.timer = new Timer(
+            String.format("lease-refresh-timer-%s", path), true);
+    acquireLease(retryPolicy, 0, acquireRetryInterval, 0, eTag,
         new TracingContext(tracingContext));
 
     while (leaseID == null && exception == null) {
@@ -120,20 +160,34 @@ public final class AbfsLease {
     LOG.debug("Acquired lease {} on {}", leaseID, path);
   }
 
+  /**
+   * Acquire a lease on the given path.
+   *
+   * @param retryPolicy        Retry policy
+   * @param numRetries         Number of retries
+   * @param retryInterval      Retry interval in seconds
+   * @param delay              Delay in seconds
+   * @param eTag               ETag of the file
+   * @param tracingContext     Tracing context
+   * @throws LeaseException if the lease cannot be acquired
+   */
   private void acquireLease(RetryPolicy retryPolicy, int numRetries,
-      int retryInterval, long delay, TracingContext tracingContext)
+      int retryInterval, long delay, final String eTag, TracingContext tracingContext)
       throws LeaseException {
     LOG.debug("Attempting to acquire lease on {}, retry {}", path, numRetries);
     if (future != null && !future.isDone()) {
       throw new LeaseException(ERR_LEASE_FUTURE_EXISTS);
     }
-    future = client.schedule(() -> client.acquireLease(path,
-        INFINITE_LEASE_DURATION, tracingContext),
-        delay, TimeUnit.SECONDS);
-    client.addCallback(future, new FutureCallback<AbfsRestOperation>() {
+    FutureCallback<AbfsRestOperation> acquireCallback = new FutureCallback<AbfsRestOperation>() {
       @Override
       public void onSuccess(@Nullable AbfsRestOperation op) {
         leaseID = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID);
+        if (leaseRefreshDuration != INFINITE_LEASE_DURATION) {
+          leaseTimerTask = new LeaseTimerTask(client, path,
+                  leaseID, tracingContext);
+          timer.scheduleAtFixedRate(leaseTimerTask, leaseRefreshDuration / 2,
+                  leaseRefreshDuration / 2);
+        }
         LOG.debug("Acquired lease {} on {}", leaseID, path);
       }
 
@@ -145,7 +199,7 @@ public final class AbfsLease {
             LOG.debug("Failed to acquire lease on {}, retrying: {}", path, throwable);
             acquireRetryCount++;
             acquireLease(retryPolicy, numRetries + 1, retryInterval,
-                retryInterval, tracingContext);
+                retryInterval, eTag, tracingContext);
           } else {
             exception = throwable;
           }
@@ -153,7 +207,21 @@ public final class AbfsLease {
           exception = throwable;
         }
       }
-    });
+    };
+    if (!isAsync) {
+      try {
+        AbfsRestOperation op = client.acquireLease(path,
+            leaseRefreshDurationInSeconds, eTag, tracingContext);
+        acquireCallback.onSuccess(op);
+        return;
+      } catch (AzureBlobFileSystemException ex) {
+        acquireCallback.onFailure(ex);
+      }
+    }
+    future = client.schedule(() -> client.acquireLease(path,
+                    INFINITE_LEASE_DURATION, eTag, tracingContext),
+            delay, TimeUnit.SECONDS);
+    client.addCallback(future, acquireCallback);
   }
 
   /**
@@ -170,6 +238,7 @@ public final class AbfsLease {
       if (future != null && !future.isDone()) {
         future.cancel(true);
       }
+      cancelTimer();
       TracingContext tracingContext = new TracingContext(this.tracingContext);
       tracingContext.setOperation(FSOperationType.RELEASE_LEASE);
       client.releaseLease(path, leaseID, tracingContext);
@@ -184,21 +253,77 @@ public final class AbfsLease {
     }
   }
 
+    /**
+     * Cancel the lease renewal timer.
+     * Also purge the lease refresh timer.
+     */
+  public void cancelTimer() {
+    if (leaseTimerTask != null) {
+      leaseTimerTask.cancel();
+    }
+    timer.purge();
+  }
+
+  /**
+   * Check if the lease has been freed.
+   *
+   * @return true if the lease has been freed
+   */
   public boolean isFreed() {
     return leaseFreed;
   }
 
+  /**
+   * Get the lease ID.
+   *
+   * @return lease ID
+   */
   public String getLeaseID() {
     return leaseID;
   }
 
+  /**
+   * Get the number of times the lease was retried.
+   *
+   * @return number of acquired retry count
+   */
   @VisibleForTesting
   public int getAcquireRetryCount() {
     return acquireRetryCount;
   }
 
+  /**
+   * Get Tracing Context.
+   *
+   * @return TracingContext tracing context
+   */
   @VisibleForTesting
   public TracingContext getTracingContext() {
     return tracingContext;
   }
+
+  /**
+   * Class to track lease renewal.
+   * If the lease is not renewed, the lease will expire and the file will be available for write.
+   */
+  private static class LeaseTimerTask extends TimerTask {
+    private final AbfsClient client;
+    private final String path;
+    private final String leaseID;
+    private final TracingContext tracingContext;
+    LeaseTimerTask(AbfsClient client, String path, String leaseID, TracingContext tracingContext) {
+      this.client = client;
+      this.path = path;
+      this.leaseID = leaseID;
+      this.tracingContext = tracingContext;
+    }
+    @Override
+    public void run() {
+      try {
+        client.renewLease(path, leaseID, tracingContext);
+      } catch (Exception e) {
+        LOG.error("Failed to renew lease on {}", path, e);
+      }
+    }
+  }
 }

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

@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.fs.azurebfs.services;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 
@@ -68,7 +69,7 @@ public class AbfsPermission extends FsPermission {
    * @return a permission object for the provided string representation
    */
   public static AbfsPermission valueOf(final String abfsSymbolicPermission) {
-    if (abfsSymbolicPermission == null) {
+    if (StringUtils.isEmpty(abfsSymbolicPermission)) {
       return null;
     }
 
@@ -100,7 +101,7 @@ public class AbfsPermission extends FsPermission {
    * extended ACL; otherwise false.
    */
   public static boolean isExtendedAcl(final String abfsSymbolicPermission) {
-    if (abfsSymbolicPermission == null) {
+    if (StringUtils.isEmpty(abfsSymbolicPermission)) {
       return false;
     }
 

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

@@ -263,6 +263,7 @@ public class AbfsRestOperation {
    * Execute a AbfsRestOperation. Track the Duration of a request if
    * abfsCounters isn't null.
    * @param tracingContext TracingContext instance to track correlation IDs
+   * @throws AzureBlobFileSystemException if the operation fails.
    */
   public void execute(TracingContext tracingContext)
       throws AzureBlobFileSystemException {

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

@@ -0,0 +1,201 @@
+/**
+ * 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.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+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 org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.NON_EMPTY_DIRECTORY_DELETE;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.PATH_NOT_FOUND;
+
+/**
+ * Orchestrator for delete over Blob endpoint. Blob endpoint for flat-namespace
+ * account does not support directory delete. This class is responsible for
+ * deleting the blobs and creating the parent directory marker file if needed.
+ */
+public class BlobDeleteHandler extends ListActionTaker {
+
+  private final Path path;
+
+  private final boolean recursive;
+
+  private boolean nonRecursiveDeleteDirectoryFailed = false;
+
+  private final TracingContext tracingContext;
+
+  private final AtomicInteger deleteCount = new AtomicInteger(0);
+
+  /** Constructor
+   *
+   * @param path path to delete.
+   * @param recursive if true, delete the path recursively.
+   * @param abfsBlobClient client to use for blob operations.
+   * @param tracingContext tracing context.
+   */
+  public BlobDeleteHandler(final Path path,
+      final boolean recursive,
+      final AbfsBlobClient abfsBlobClient,
+      final TracingContext tracingContext) {
+    super(path, abfsBlobClient, tracingContext);
+    this.path = path;
+    this.recursive = recursive;
+    this.tracingContext = tracingContext;
+  }
+
+  /**{@inheritDoc}
+   *
+   * @return the maximum number of parallelism for delete operation.
+   */
+  @Override
+  int getMaxConsumptionParallelism() {
+    return getAbfsClient().getAbfsConfiguration()
+        .getBlobDeleteDirConsumptionParallelism();
+  }
+
+  /** Delete the path.
+   *
+   * @param path path to delete.
+   * @return true if the path is deleted.
+   * @throws AzureBlobFileSystemException server error.
+   */
+  private boolean deleteInternal(final Path path)
+      throws AzureBlobFileSystemException {
+    getAbfsClient().deleteBlobPath(path, null, tracingContext);
+    deleteCount.incrementAndGet();
+    return true;
+  }
+
+  /**
+   * Orchestrate the delete operation.
+   *
+   * @return true if the delete operation is successful.
+   * @throws AzureBlobFileSystemException if deletion fails due to server error or path doesn't exist.
+   */
+  public boolean execute() throws AzureBlobFileSystemException {
+    /*
+     * ABFS is not aware if it's a file or directory. So, we need to list the
+     * path and delete the listed objects. The listing returns the children of
+     * the path and not the path itself.
+     */
+    listRecursiveAndTakeAction();
+    if (nonRecursiveDeleteDirectoryFailed) {
+      throw new AbfsRestOperationException(HTTP_CONFLICT,
+          NON_EMPTY_DIRECTORY_DELETE.getErrorCode(),
+          NON_EMPTY_DIRECTORY_DELETE.getErrorMessage(),
+          new PathIOException(path.toString(),
+              "Non-recursive delete of non-empty directory"));
+    }
+    tracingContext.setOperatedBlobCount(deleteCount.get() + 1);
+    /*
+     * If path is actually deleted.
+     */
+    boolean deleted;
+    try {
+      /*
+       * Delete the required path.
+       * Directory should be safely deleted as the path might be implicit.
+       */
+      deleted = recursive ? safeDelete(path) : deleteInternal(path);
+    } finally {
+      tracingContext.setOperatedBlobCount(null);
+    }
+    if (deleteCount.get() == 0) {
+      /*
+       * DeleteCount can be zero only if the path does not exist.
+       */
+      throw new AbfsRestOperationException(HTTP_NOT_FOUND,
+          PATH_NOT_FOUND.getErrorCode(), PATH_NOT_FOUND.getErrorMessage(),
+          new PathIOException(path.toString(), "Path not found"));
+    }
+
+    /*
+     * Ensure that parent directory of the deleted path is marked as a folder. This
+     * is required if the parent is an implicit directory (path with no marker blob),
+     * and the given path is the only child of the parent, the parent would become
+     * non-existing.
+     */
+    if (deleted) {
+      ensurePathParentExist();
+    }
+    return deleted;
+  }
+
+  /** Ensure that the parent path exists.
+   *
+   * @throws AzureBlobFileSystemException server error.
+   */
+  private void ensurePathParentExist()
+      throws AzureBlobFileSystemException {
+    if (!path.isRoot() && !path.getParent().isRoot()) {
+      try {
+        getAbfsClient().createPath(path.getParent().toUri().getPath(),
+            false,
+            false,
+            null,
+            false,
+            null,
+            null,
+            tracingContext);
+      } catch (AbfsRestOperationException ex) {
+        if (ex.getStatusCode() != HTTP_CONFLICT) {
+          throw ex;
+        }
+      }
+    }
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  boolean takeAction(final Path path) throws AzureBlobFileSystemException {
+    if (!recursive) {
+      /*
+       * If the delete operation is non-recursive, then the path can not be a directory.
+       */
+      nonRecursiveDeleteDirectoryFailed = true;
+      return false;
+    }
+    return safeDelete(path);
+  }
+
+  /**
+   * Delete the path if it exists. Gracefully handles the case where the path does not exist.
+   *
+   * @param path path to delete.
+   * @return true if the path is deleted or is not found.
+   * @throws AzureBlobFileSystemException server error.
+   */
+  private boolean safeDelete(final Path path)
+      throws AzureBlobFileSystemException {
+    try {
+      return deleteInternal(path);
+    } catch (AbfsRestOperationException ex) {
+      if (ex.getStatusCode() == HTTP_NOT_FOUND) {
+        return true;
+      }
+      throw ex;
+    }
+  }
+}

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

@@ -0,0 +1,634 @@
+/**
+ * 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.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+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.TimeoutException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.enums.BlobCopyProgress;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_ABORTED;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_FAILED;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_SUCCESS;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_ID;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_SOURCE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_STATUS;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_STATUS_DESCRIPTION;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_ABORTED;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_FAILED;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND;
+
+/**
+ * Orchestrator for rename over Blob endpoint. Handles both directory and file
+ * renames. Blob Endpoint does not expose rename API, this class is responsible
+ * for copying the blobs and deleting the source blobs.
+ * <p>
+ * For directory rename, it recursively lists the blobs in the source directory and
+ * copies them to the destination directory.
+ */
+public class BlobRenameHandler extends ListActionTaker {
+
+  public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+
+  private final String srcEtag;
+
+  private final Path src, dst;
+
+  private final boolean isAtomicRename, isAtomicRenameRecovery;
+
+  private final TracingContext tracingContext;
+
+  private AbfsLease srcAbfsLease;
+
+  private String srcLeaseId;
+
+  private final List<AbfsLease> leases = new ArrayList<>();
+
+  private final AtomicInteger operatedBlobCount = new AtomicInteger(0);
+
+  /** Constructor.
+   *
+   * @param src source path
+   * @param dst destination path
+   * @param abfsClient AbfsBlobClient to use for the rename operation
+   * @param srcEtag eTag of the source path
+   * @param isAtomicRename true if the rename operation is atomic
+   * @param isAtomicRenameRecovery true if the rename operation is a recovery of a previous failed atomic rename operation
+   * @param tracingContext object of tracingContext used for the tracing of the server calls.
+   */
+  public BlobRenameHandler(final String src,
+      final String dst,
+      final AbfsBlobClient abfsClient,
+      final String srcEtag,
+      final boolean isAtomicRename,
+      final boolean isAtomicRenameRecovery,
+      final TracingContext tracingContext) {
+    super(new Path(src), abfsClient, tracingContext);
+    this.srcEtag = srcEtag;
+    this.tracingContext = tracingContext;
+    this.src = new Path(src);
+    this.dst = new Path(dst);
+    this.isAtomicRename = isAtomicRename;
+    this.isAtomicRenameRecovery = isAtomicRenameRecovery;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  int getMaxConsumptionParallelism() {
+    return getAbfsClient().getAbfsConfiguration()
+        .getBlobRenameDirConsumptionParallelism();
+  }
+
+  /**
+   * Orchestrates the rename operation.
+   *
+   * @return AbfsClientRenameResult containing the result of the rename operation
+   * @throws AzureBlobFileSystemException if server call fails
+   */
+  public boolean execute() throws AzureBlobFileSystemException {
+    PathInformation pathInformation = getPathInformation(src, tracingContext);
+    boolean result = false;
+    if (preCheck(src, dst, pathInformation)) {
+      RenameAtomicity renameAtomicity = null;
+      if (pathInformation.getIsDirectory()
+          && pathInformation.getIsImplicit()) {
+        AbfsRestOperation createMarkerOp = getAbfsClient().createPath(
+            src.toUri().getPath(),
+            false, false, null,
+            false, null, null, tracingContext);
+        pathInformation.setETag(extractEtagHeader(createMarkerOp.getResult()));
+      }
+      try {
+        if (isAtomicRename) {
+          /*
+           * Conditionally get a lease on the source blob to prevent other writers
+           * from changing it. This is used for correctness in HBase when log files
+           * are renamed. When the HBase master renames a log file folder, the lease
+           * locks out other writers.  This prevents a region server that the master
+           * thinks is dead, but is still alive, from committing additional updates.
+           * This is different than when HBase runs on HDFS, where the region server
+           * recovers the lease on a log file, to gain exclusive access to it, before
+           * it splits it.
+           */
+          if (srcAbfsLease == null) {
+            srcAbfsLease = takeLease(src, srcEtag);
+          }
+          srcLeaseId = srcAbfsLease.getLeaseID();
+          if (!isAtomicRenameRecovery && pathInformation.getIsDirectory()) {
+            /*
+             * if it is not a resume of a previous failed atomic rename operation,
+             * create renameJson file.
+             */
+            renameAtomicity = getRenameAtomicity(pathInformation);
+            renameAtomicity.preRename();
+          }
+        }
+        if (pathInformation.getIsDirectory()) {
+          result = listRecursiveAndTakeAction() && finalSrcRename();
+        } else {
+          result = renameInternal(src, dst);
+        }
+      } finally {
+        if (srcAbfsLease != null) {
+          // If the operation is successful, cancel the timer and no need to release
+          // the lease as rename on the blob-path has taken place.
+          if (result) {
+            srcAbfsLease.cancelTimer();
+          } else {
+            srcAbfsLease.free();
+          }
+        }
+      }
+      if (result && renameAtomicity != null) {
+        renameAtomicity.postRename();
+      }
+    }
+    return result;
+  }
+
+  /** Final rename operation after all the blobs have been copied.
+   *
+   * @return true if rename is successful
+   * @throws AzureBlobFileSystemException if server call fails
+   */
+  private boolean finalSrcRename() throws AzureBlobFileSystemException {
+    tracingContext.setOperatedBlobCount(operatedBlobCount.get() + 1);
+    try {
+      return renameInternal(src, dst);
+    } finally {
+      tracingContext.setOperatedBlobCount(null);
+    }
+  }
+
+  /** Gets the rename atomicity object.
+   *
+   * @param pathInformation object containing the path information of the source path
+   *
+   * @return RenameAtomicity object
+   */
+  @VisibleForTesting
+  public RenameAtomicity getRenameAtomicity(final PathInformation pathInformation) {
+    return new RenameAtomicity(src,
+        dst,
+        new Path(src.getParent(), src.getName() + RenameAtomicity.SUFFIX),
+        tracingContext,
+        pathInformation.getETag(),
+        getAbfsClient());
+  }
+
+  /** Takes a lease on the path.
+   *
+   * @param path path on which the lease is to be taken
+   * @param eTag eTag of the path
+   *
+   * @return object containing the lease information
+   * @throws AzureBlobFileSystemException if server call fails
+   */
+  private AbfsLease takeLease(final Path path, final String eTag)
+      throws AzureBlobFileSystemException {
+    AbfsLease lease = new AbfsLease(getAbfsClient(), path.toUri().getPath(),
+        false,
+        getAbfsClient().getAbfsConfiguration()
+            .getAtomicRenameLeaseRefreshDuration(),
+        eTag, tracingContext);
+    leases.add(lease);
+    return lease;
+  }
+
+  /** Checks if the path contains a colon.
+   *
+   * @param p path to check
+   *
+   * @return true if the path contains a colon
+   */
+  private boolean containsColon(Path p) {
+    return p.toUri().getPath().contains(COLON);
+  }
+
+  /**
+   * Since, server doesn't have a rename API and would not be able to check HDFS
+   * contracts, client would have to ensure that no HDFS contract is violated.
+   *
+   * @param src source path
+   * @param dst destination path
+   * @param pathInformation object in which path information of the source path would be stored
+   *
+   * @return true if the pre-checks pass
+   * @throws AzureBlobFileSystemException if server call fails or given paths are invalid.
+   */
+  private boolean preCheck(final Path src, final Path dst,
+      final PathInformation pathInformation)
+      throws AzureBlobFileSystemException {
+    validateDestinationPath(src, dst);
+    validateSourcePath(pathInformation);
+    validateDestinationPathNotExist(src, dst, pathInformation);
+    validateDestinationParentExist(src, dst, pathInformation);
+
+    return true;
+  }
+
+  /**
+   * Validate if the format of the destination path is correct and if the destination
+   * path is not a sub-directory of the source path.
+   *
+   * @param src source path
+   * @param dst destination path
+   *
+   * @throws AbfsRestOperationException if the destination path is invalid
+   */
+  private void validateDestinationPath(final Path src, final Path dst)
+      throws AbfsRestOperationException {
+    if (containsColon(dst)) {
+      throw new AbfsRestOperationException(
+          HttpURLConnection.HTTP_BAD_REQUEST,
+          AzureServiceErrorCode.INVALID_RENAME_DESTINATION.getErrorCode(), null,
+          new PathIOException(dst.toUri().getPath(),
+              "Destination path contains colon"));
+    }
+
+    validateDestinationIsNotSubDir(src, dst);
+  }
+
+  /**
+   * Validate if the destination path is not a sub-directory of the source path.
+   *
+   * @param src source path
+   * @param dst destination path
+   */
+  private void validateDestinationIsNotSubDir(final Path src,
+      final Path dst) throws AbfsRestOperationException {
+    LOG.debug("Check if the destination is subDirectory");
+    Path nestedDstParent = dst.getParent();
+    if (nestedDstParent != null && nestedDstParent.toUri()
+        .getPath()
+        .indexOf(src.toUri().getPath()) == 0) {
+      LOG.info("Rename src: {} dst: {} failed as dst is subDir of src",
+          src, dst);
+      throw new AbfsRestOperationException(HttpURLConnection.HTTP_CONFLICT,
+          AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH.getErrorCode(),
+          AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH.getErrorMessage(),
+          new Exception(
+              AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH.getErrorCode()));
+    }
+  }
+
+  /**
+   * Validate if the source path exists and if the client knows the ETag of the source path,
+   * then the ETag should match with the server.
+   *
+   * @param pathInformation object containing the path information of the source path
+   *
+   * @throws AbfsRestOperationException if the source path is not found or if the ETag of the source
+   *                                    path does not match with the server.
+   */
+  private void validateSourcePath(final PathInformation pathInformation)
+      throws AzureBlobFileSystemException {
+    if (!pathInformation.getPathExists()) {
+      throw new AbfsRestOperationException(
+          HttpURLConnection.HTTP_NOT_FOUND,
+          AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND.getErrorCode(), null,
+          new Exception(
+              AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND.getErrorCode()));
+    }
+    if (srcEtag != null && !srcEtag.equals(pathInformation.getETag())) {
+      throw new AbfsRestOperationException(
+          HttpURLConnection.HTTP_CONFLICT,
+          AzureServiceErrorCode.PATH_ALREADY_EXISTS.getErrorCode(), null,
+          new Exception(
+              AzureServiceErrorCode.PATH_ALREADY_EXISTS.getErrorCode()));
+    }
+  }
+
+  /** Validate if the destination path does not exist.
+   *
+   * @param src source path
+   * @param dst destination path
+   * @param pathInformation object containing the path information of the source path
+   *
+   * @throws AbfsRestOperationException if the destination path already exists
+   */
+  private void validateDestinationPathNotExist(final Path src,
+      final Path dst,
+      final PathInformation pathInformation)
+      throws AzureBlobFileSystemException {
+    /*
+     * Destination path name can be same to that of source path name only in the
+     * case of a directory rename.
+     *
+     * In case the directory is being renamed to some other name, the destination
+     * check would happen on the AzureBlobFileSystem#rename method.
+     */
+    if (pathInformation.getIsDirectory() && dst.getName()
+        .equals(src.getName())) {
+      PathInformation dstPathInformation = getPathInformation(
+          dst,
+          tracingContext);
+      if (dstPathInformation.getPathExists()) {
+        LOG.info(
+            "Rename src: {} dst: {} failed as qualifiedDst already exists",
+            src, dst);
+        throw new AbfsRestOperationException(
+            HttpURLConnection.HTTP_CONFLICT,
+            AzureServiceErrorCode.PATH_ALREADY_EXISTS.getErrorCode(), null,
+            null);
+      }
+    }
+  }
+
+  /** Validate if the parent of the destination path exists.
+   *
+   * @param src source path
+   * @param dst destination path
+   * @param pathInformation object containing the path information of the source path
+   *
+   * @throws AbfsRestOperationException if the parent of the destination path does not exist
+   */
+  private void validateDestinationParentExist(final Path src,
+      final Path dst,
+      final PathInformation pathInformation)
+      throws AzureBlobFileSystemException {
+    final Path nestedDstParent = dst.getParent();
+    if (!dst.isRoot() && nestedDstParent != null && !nestedDstParent.isRoot()
+        && (
+        !pathInformation.getIsDirectory() || !dst.getName()
+            .equals(src.getName()))) {
+      PathInformation nestedDstInfo = getPathInformation(
+          nestedDstParent,
+          tracingContext);
+      if (!nestedDstInfo.getPathExists() || !nestedDstInfo.getIsDirectory()) {
+        throw new AbfsRestOperationException(
+            HttpURLConnection.HTTP_NOT_FOUND,
+            RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode(), null,
+            new Exception(
+                RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode()));
+      }
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  boolean takeAction(final Path path) throws AzureBlobFileSystemException {
+    return renameInternal(path, getDstPathForBlob(dst, path, src));
+  }
+
+  /** Renames the source path to the destination path.
+   *
+   * @param path source path
+   * @param destinationPathForBlobPartOfRenameSrcDir destination path
+   *
+   * @return true if rename is successful
+   * @throws AzureBlobFileSystemException if server call fails
+   */
+  private boolean renameInternal(final Path path,
+      final Path destinationPathForBlobPartOfRenameSrcDir)
+      throws AzureBlobFileSystemException {
+    final String leaseId;
+    AbfsLease abfsLease = null;
+    if (isAtomicRename) {
+      /*
+       * To maintain atomicity of rename of the path, lease is taken on the path.
+       */
+      if (path.equals(src)) {
+        abfsLease = srcAbfsLease;
+        leaseId = srcLeaseId;
+      } else {
+        abfsLease = takeLease(path, null);
+        leaseId = abfsLease.getLeaseID();
+      }
+    } else {
+      leaseId = null;
+    }
+    boolean operated = false;
+    try {
+      copyPath(path, destinationPathForBlobPartOfRenameSrcDir, leaseId);
+      getAbfsClient().deleteBlobPath(path, leaseId, tracingContext);
+      operated = true;
+    } finally {
+      if (abfsLease != null) {
+        // If the operation is successful, cancel the timer and no need to release
+        // the lease as delete on the blob-path has taken place.
+        if (operated) {
+          abfsLease.cancelTimer();
+        } else {
+          abfsLease.free();
+        }
+      }
+    }
+    operatedBlobCount.incrementAndGet();
+    return true;
+  }
+
+  /** Copies the source path to the destination path.
+   *
+   * @param src source path
+   * @param dst destination path
+   * @param leaseId lease id for the source path
+   *
+   * @throws AzureBlobFileSystemException if server call fails
+   */
+  private void copyPath(final Path src, final Path dst, final String leaseId)
+      throws AzureBlobFileSystemException {
+    String copyId;
+    try {
+      AbfsRestOperation copyPathOp = getAbfsClient().copyBlob(src, dst, leaseId,
+          tracingContext);
+      final String progress = copyPathOp.getResult()
+          .getResponseHeader(X_MS_COPY_STATUS);
+      if (COPY_STATUS_SUCCESS.equalsIgnoreCase(progress)) {
+        return;
+      }
+      copyId = copyPathOp.getResult()
+          .getResponseHeader(X_MS_COPY_ID);
+    } catch (AbfsRestOperationException ex) {
+      if (ex.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) {
+        AbfsRestOperation dstPathStatus = getAbfsClient().getPathStatus(
+            dst.toUri().getPath(),
+            tracingContext, null, false);
+        final String srcCopyPath = ROOT_PATH + getAbfsClient().getFileSystem()
+            + src.toUri().getPath();
+        if (dstPathStatus != null && dstPathStatus.getResult() != null
+            && (srcCopyPath.equals(getDstSource(dstPathStatus)))) {
+          return;
+        }
+      }
+      throw ex;
+    }
+    final long pollWait = getAbfsClient().getAbfsConfiguration()
+        .getBlobCopyProgressPollWaitMillis();
+    final long maxWait = getAbfsClient().getAbfsConfiguration()
+        .getBlobCopyProgressMaxWaitMillis();
+    long startTime = System.currentTimeMillis();
+    while (handleCopyInProgress(dst, tracingContext, copyId)
+        == BlobCopyProgress.PENDING) {
+      if (System.currentTimeMillis() - startTime > maxWait) {
+        throw new TimeoutException(
+            String.format("Blob copy progress wait time exceeded "
+                + "for source: %s and destination: %s", src, dst));
+      }
+      try {
+        Thread.sleep(pollWait);
+      } catch (InterruptedException ignored) {
+
+      }
+    }
+  }
+
+  /** Gets the source path of the copy operation.
+   *
+   * @param dstPathStatus server response for the GetBlobProperties API on the
+   * destination path.
+   *
+   * @return source path of the copy operation
+   */
+  private String getDstSource(final AbfsRestOperation dstPathStatus) {
+    try {
+      String responseHeader = dstPathStatus.getResult()
+          .getResponseHeader(X_MS_COPY_SOURCE);
+      if (responseHeader == null) {
+        return null;
+      }
+      return new URL(responseHeader).toURI().getPath();
+    } catch (URISyntaxException | MalformedURLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Verifies if the blob copy is success or a failure or still in progress.
+   *
+   * @param dstPath path of the destination for the copying
+   * @param tracingContext object of tracingContext used for the tracing of the
+   * server calls.
+   * @param copyId id returned by server on the copy server-call. This id gets
+   * attached to blob and is returned by GetBlobProperties API on the destination.
+   *
+   * @return BlobCopyProgress indicating the status of the copy operation
+   *
+   * @throws AzureBlobFileSystemException exception returned in making server call
+   * for GetBlobProperties on the path. It can be thrown if the copyStatus is failure
+   * or is aborted.
+   */
+  @VisibleForTesting
+  public BlobCopyProgress handleCopyInProgress(final Path dstPath,
+      final TracingContext tracingContext,
+      final String copyId) throws AzureBlobFileSystemException {
+    AbfsRestOperation op = getAbfsClient().getPathStatus(
+        dstPath.toUri().getPath(),
+        tracingContext, null, false);
+
+    if (op.getResult() != null && copyId != null
+        && copyId.equals(op.getResult().getResponseHeader(X_MS_COPY_ID))) {
+      final String copyStatus = op.getResult()
+          .getResponseHeader(X_MS_COPY_STATUS);
+      if (COPY_STATUS_SUCCESS.equalsIgnoreCase(copyStatus)) {
+        return BlobCopyProgress.SUCCESS;
+      }
+      if (COPY_STATUS_FAILED.equalsIgnoreCase(copyStatus)) {
+        throw new AbfsRestOperationException(
+            COPY_BLOB_FAILED.getStatusCode(), COPY_BLOB_FAILED.getErrorCode(),
+            String.format("copy to path %s failed due to: %s",
+                dstPath.toUri().getPath(),
+                op.getResult().getResponseHeader(X_MS_COPY_STATUS_DESCRIPTION)),
+            new Exception(COPY_BLOB_FAILED.getErrorCode()));
+      }
+      if (COPY_STATUS_ABORTED.equalsIgnoreCase(copyStatus)) {
+        throw new AbfsRestOperationException(
+            COPY_BLOB_ABORTED.getStatusCode(), COPY_BLOB_ABORTED.getErrorCode(),
+            String.format("copy to path %s aborted", dstPath.toUri().getPath()),
+            new Exception(COPY_BLOB_ABORTED.getErrorCode()));
+      }
+    }
+    return BlobCopyProgress.PENDING;
+  }
+
+  /**
+   * Translates the destination path for a blob part of a source directory getting
+   * renamed.
+   *
+   * @param destinationDir destination directory for the rename operation
+   * @param blobPath path of blob inside sourceDir being renamed.
+   * @param sourceDir source directory for the rename operation
+   *
+   * @return translated path for the blob
+   */
+  private Path getDstPathForBlob(final Path destinationDir,
+      final Path blobPath, final Path sourceDir) {
+    String destinationPathStr = destinationDir.toUri().getPath();
+    String sourcePathStr = sourceDir.toUri().getPath();
+    String srcBlobPropertyPathStr = blobPath.toUri().getPath();
+    if (sourcePathStr.equals(srcBlobPropertyPathStr)) {
+      return destinationDir;
+    }
+    return new Path(
+        destinationPathStr + ROOT_PATH + srcBlobPropertyPathStr.substring(
+            sourcePathStr.length()));
+  }
+
+  /** Get information of the path.
+   *
+   * @param path path for which the path information is to be fetched
+   * @param tracingContext object of tracingContext used for the tracing of the
+   * server calls.
+   *
+   * @return object containing the path information
+   * @throws AzureBlobFileSystemException if server call fails
+   */
+  private PathInformation getPathInformation(Path path,
+      TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    try {
+      AbfsRestOperation op = getAbfsClient().getPathStatus(path.toString(),
+          tracingContext, null, true);
+
+      return new PathInformation(true,
+          getAbfsClient().checkIsDir(op.getResult()),
+          extractEtagHeader(op.getResult()),
+          op.getResult() instanceof AbfsHttpOperation.AbfsHttpOperationWithFixedResultForGetFileStatus);
+    } catch (AbfsRestOperationException e) {
+      if (e.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
+        return new PathInformation(false, false, null, false);
+      }
+      throw e;
+    }
+  }
+
+  @VisibleForTesting
+  public List<AbfsLease> getLeases() {
+    return leases;
+  }
+}

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

@@ -0,0 +1,270 @@
+/**
+ * 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.List;
+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.AtomicBoolean;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobListResultSchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+
+/**
+ * ListActionTaker is an abstract class that provides a way to list the paths
+ * recursively and take action on each path. The implementations of this class
+ * should provide the action to be taken on each listed path.
+ */
+public abstract class ListActionTaker {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ListActionTaker.class);
+
+  private final Path path;
+
+  private final AbfsBlobClient abfsClient;
+
+  private final TracingContext tracingContext;
+
+  private final ExecutorService executorService;
+
+  private final AtomicBoolean producerThreadToBeStopped = new AtomicBoolean(
+      false);
+
+  /** Constructor.
+   *
+   * @param path the path to list recursively.
+   * @param abfsClient the AbfsBlobClient to use for listing.
+   * @param tracingContext the tracing context to use for listing.
+   */
+  public ListActionTaker(Path path,
+      AbfsBlobClient abfsClient,
+      TracingContext tracingContext) {
+    this.path = path;
+    this.abfsClient = abfsClient;
+    this.tracingContext = tracingContext;
+    executorService = Executors.newFixedThreadPool(
+        getMaxConsumptionParallelism());
+  }
+
+  public AbfsBlobClient getAbfsClient() {
+    return abfsClient;
+  }
+
+  /** Get the maximum number of parallelism for consumption.
+   *
+   * @return the maximum number of parallelism for consumption.
+   */
+  abstract int getMaxConsumptionParallelism();
+
+  /** Take action on a path.
+   *
+   * @param path the path to take action on.
+   * @return true if the action is successful.
+   * @throws AzureBlobFileSystemException if the action fails.
+   */
+  abstract boolean takeAction(Path path) throws AzureBlobFileSystemException;
+
+  private boolean takeAction(List<Path> paths)
+      throws AzureBlobFileSystemException {
+    List<Future<Boolean>> futureList = new ArrayList<>();
+    for (Path path : paths) {
+      Future<Boolean> future = executorService.submit(() -> takeAction(path));
+      futureList.add(future);
+    }
+
+    AzureBlobFileSystemException executionException = null;
+    boolean actionResult = true;
+    for (Future<Boolean> future : futureList) {
+      try {
+        Boolean result = future.get();
+        if (!result) {
+          actionResult = false;
+        }
+      } catch (InterruptedException e) {
+        LOG.debug("Thread interrupted while taking action on path: {}",
+            path.toUri().getPath());
+      } catch (ExecutionException e) {
+        executionException = (AzureBlobFileSystemException) e.getCause();
+      }
+    }
+    if (executionException != null) {
+      throw executionException;
+    }
+    return actionResult;
+  }
+
+  /**
+   * Spawns a producer thread that list the children of the path recursively and queue
+   * them in into {@link ListBlobQueue}. On the main thread, it dequeues the
+   * path and supply them to parallel thread for relevant action which is defined
+   * in {@link #takeAction(Path)}.
+   *
+   * @return true if the action is successful.
+   * @throws AzureBlobFileSystemException if the action fails.
+   */
+  public boolean listRecursiveAndTakeAction()
+      throws AzureBlobFileSystemException {
+    AbfsConfiguration configuration = getAbfsClient().getAbfsConfiguration();
+    Thread producerThread = null;
+    try {
+      ListBlobQueue listBlobQueue = createListBlobQueue(configuration);
+      producerThread = new Thread(() -> {
+        try {
+          produceConsumableList(listBlobQueue);
+        } catch (AzureBlobFileSystemException e) {
+          listBlobQueue.markProducerFailure(e);
+        }
+      });
+      producerThread.start();
+
+      while (!listBlobQueue.getIsCompleted()) {
+        List<Path> paths = listBlobQueue.consume();
+        if (paths == null) {
+          continue;
+        }
+        try {
+          boolean resultOnPartAction = takeAction(paths);
+          if (!resultOnPartAction) {
+            return false;
+          }
+        } catch (AzureBlobFileSystemException parallelConsumptionException) {
+          listBlobQueue.markConsumptionFailed();
+          throw parallelConsumptionException;
+        }
+      }
+      return true;
+    } finally {
+      if (producerThread != null) {
+        producerThreadToBeStopped.set(true);
+      }
+      executorService.shutdownNow();
+    }
+  }
+
+  /**
+   * Create a {@link ListBlobQueue} instance.
+   *
+   * @param configuration the configuration to use.
+   * @return the created {@link ListBlobQueue} instance.
+   * @throws InvalidConfigurationValueException if the configuration is invalid.
+   */
+  @VisibleForTesting
+  protected ListBlobQueue createListBlobQueue(final AbfsConfiguration configuration)
+      throws InvalidConfigurationValueException {
+    return new ListBlobQueue(
+        configuration.getProducerQueueMaxSize(),
+        getMaxConsumptionParallelism(),
+        configuration.getListingMaxConsumptionLag()
+    );
+  }
+
+  /**
+   * Produce the consumable list of paths.
+   *
+   * @param listBlobQueue the {@link ListBlobQueue} to enqueue the paths.
+   * @throws AzureBlobFileSystemException if the listing fails.
+   */
+  private void produceConsumableList(final ListBlobQueue listBlobQueue)
+      throws AzureBlobFileSystemException {
+    String continuationToken = null;
+    do {
+      continuationToken = listAndEnqueue(listBlobQueue, continuationToken);
+    } while (!producerThreadToBeStopped.get() && continuationToken != null
+        && !listBlobQueue.getConsumptionFailed());
+    listBlobQueue.complete();
+  }
+
+  /**
+   * List the paths and enqueue them into the {@link ListBlobQueue}.
+   *
+   * @param listBlobQueue the {@link ListBlobQueue} to enqueue the paths.
+   * @param continuationToken the continuation token to use for listing.
+   * @return the continuation token for the next listing.
+   * @throws AzureBlobFileSystemException if the listing fails.
+   */
+  @VisibleForTesting
+  protected String listAndEnqueue(final ListBlobQueue listBlobQueue,
+      String continuationToken) throws AzureBlobFileSystemException {
+    final int queueAvailableSizeForProduction = Math.min(
+        DEFAULT_AZURE_LIST_MAX_RESULTS,
+        listBlobQueue.availableSizeForProduction());
+    if (queueAvailableSizeForProduction == 0) {
+      return null;
+    }
+    final AbfsRestOperation op;
+    try {
+      op = getAbfsClient().listPath(path.toUri().getPath(),
+          true,
+          queueAvailableSizeForProduction, continuationToken,
+          tracingContext);
+    } catch (AzureBlobFileSystemException ex) {
+      throw ex;
+    } catch (IOException ex) {
+      throw new AbfsRestOperationException(-1, null,
+          "Unknown exception from listing: " + ex.getMessage(), ex);
+    }
+
+    ListResultSchema retrievedSchema = op.getResult().getListResultSchema();
+    if (retrievedSchema == null) {
+      return continuationToken;
+    }
+    continuationToken
+        = ((BlobListResultSchema) retrievedSchema).getNextMarker();
+    List<Path> paths = new ArrayList<>();
+    addPaths(paths, retrievedSchema);
+    listBlobQueue.enqueue(paths);
+    return continuationToken;
+  }
+
+  /**
+   * Add the paths from the retrieved schema to the list of paths.
+   *
+   * @param paths the list of paths to add to.
+   * @param retrievedSchema the retrieved schema.
+   */
+  @VisibleForTesting
+  protected void addPaths(final List<Path> paths,
+      final ListResultSchema retrievedSchema) {
+    for (ListResultEntrySchema entry : retrievedSchema.paths()) {
+      Path entryPath = new Path(ROOT_PATH, entry.name());
+      if (!entryPath.equals(this.path)) {
+        paths.add(entryPath);
+      }
+    }
+  }
+}

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

@@ -0,0 +1,197 @@
+/**
+ * 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.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Queue;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CONSUMER_MAX_LAG;
+
+/**
+ * Data-structure to hold the list of paths to be processed. The paths are
+ * enqueued by the producer and dequeued by the consumer. The producer can
+ * enqueue the paths until the queue is full. The consumer can consume the paths
+ * until the queue is empty. The producer can mark the queue as completed once
+ * all the paths are enqueued and there is no more paths that can be returned from
+ * server. The consumer can mark the queue as failed if it encounters any exception
+ * while consuming the paths.
+ */
+class ListBlobQueue {
+
+  private final Queue<Path> pathQueue = new ArrayDeque<>();
+
+  private final int maxSize;
+
+  private final int consumeSetSize;
+
+  private volatile boolean isCompleted = false;
+
+  private volatile boolean isConsumptionFailed = false;
+
+  private volatile AzureBlobFileSystemException failureFromProducer;
+
+  /**
+   * Maximum number of entries in the queue allowed for letting the producer to
+   * produce. If the current size of the queue is greater than or equal to
+   * maxConsumptionLag, the producer will wait until the current size of the queue
+   * becomes lesser than maxConsumptionLag. This parameter is used to control the
+   * behavior of the producer-consumer pattern and preventing producer from
+   * rapidly producing very small amount of items.
+   * <p>
+   * For example, let's say maxSize is 10000 and maxConsumptionLag is 5000.
+   * The producer will stop producing when the current size of the queue is 5000
+   * and will wait until the current size of the queue becomes lesser than 5000.
+   * Once, the size becomes lesser than 5000, producer can produce (maxSize - currentSize)
+   * of items, which would make the current size of the queue to be 10000. Then again
+   * it will wait for 5000 items to be consumed before generating next 5000 items.
+   * <p>
+   * If this is not used, the producer will keep on producing items as soon as
+   * the queue become available with small size. Let say, 5 items got consumed,
+   * producer would make a server call for only 5 items and would populate the queue.
+   * <p>
+   * This mechanism would prevent producer making server calls for very small amount
+   * of items.
+   */
+  private final int maxConsumptionLag;
+
+  ListBlobQueue(int maxSize, int consumeSetSize, int maxConsumptionLag)
+      throws InvalidConfigurationValueException {
+    this.maxSize = maxSize;
+    this.maxConsumptionLag = maxConsumptionLag;
+    this.consumeSetSize = consumeSetSize;
+
+    if (maxConsumptionLag >= maxSize) {
+      throw new InvalidConfigurationValueException(FS_AZURE_CONSUMER_MAX_LAG,
+          "maxConsumptionLag should be lesser than maxSize");
+    }
+  }
+
+  /** Mark the queue as failed.*/
+  void markProducerFailure(AzureBlobFileSystemException failure) {
+    failureFromProducer = failure;
+  }
+
+  /** Mark the queue as completed.*/
+  void complete() {
+    isCompleted = true;
+  }
+
+  /** Mark the consumption as failed.*/
+  synchronized void markConsumptionFailed() {
+    isConsumptionFailed = true;
+    notify();
+  }
+
+  /** Check if the consumption has failed.
+   *
+   * @return true if the consumption has failed
+   */
+  boolean getConsumptionFailed() {
+    return isConsumptionFailed;
+  }
+
+  /** Check if the queue is completed.
+   *
+   * @return true if the queue is completed
+   */
+  boolean getIsCompleted() {
+    return isCompleted && size() == 0;
+  }
+
+  /** Get the exception from producer.
+   *
+   * @return exception from producer
+   */
+  private AzureBlobFileSystemException getException() {
+    return failureFromProducer;
+  }
+
+  /** Enqueue the paths.
+   *
+   * @param pathList list of paths to be enqueued
+   */
+  synchronized void enqueue(List<Path> pathList) {
+    if (isCompleted) {
+      throw new IllegalStateException(
+          "Cannot enqueue paths as the queue is already marked as completed");
+    }
+    pathQueue.addAll(pathList);
+  }
+
+  /** Consume the paths.
+   *
+   * @return list of paths to be consumed
+   * @throws AzureBlobFileSystemException if the consumption fails
+   */
+  synchronized List<Path> consume() throws AzureBlobFileSystemException {
+    AzureBlobFileSystemException exception = getException();
+    if (exception != null) {
+      throw exception;
+    }
+    return dequeue();
+  }
+
+  /** Dequeue the paths.
+   *
+   * @return list of paths to be consumed
+   */
+  private List<Path> dequeue() {
+    List<Path> pathListForConsumption = new ArrayList<>();
+    int counter = 0;
+    while (counter < consumeSetSize && !pathQueue.isEmpty()) {
+      pathListForConsumption.add(pathQueue.poll());
+      counter++;
+    }
+    if (counter > 0) {
+      notify();
+    }
+    return pathListForConsumption;
+  }
+
+  synchronized int size() {
+    return pathQueue.size();
+  }
+
+  /**
+   * Returns the available size of the queue for production. This is calculated by subtracting
+   * the current size of the queue from its maximum size. This method waits until
+   * the current size of the queue becomes lesser than the maxConsumptionLag. This
+   * method is synchronized to prevent concurrent modifications of the queue.
+   *
+   * @return the available size of the queue.
+   */
+  synchronized int availableSizeForProduction() {
+    while (size() >= maxConsumptionLag) {
+      if (isConsumptionFailed) {
+        return 0;
+      }
+      try {
+        wait();
+      } catch (InterruptedException ignored) {
+      }
+    }
+    return maxSize - size();
+  }
+}

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

@@ -0,0 +1,108 @@
+/**
+ * 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;
+
+/**
+ * Information about a path.
+ */
+public class PathInformation {
+
+  private Boolean pathExists;
+
+  private Boolean isDirectory;
+
+  private Boolean isImplicit;
+
+  private String eTag;
+
+  /**
+   * Constructor.
+   * @param pathExists The path exists.
+   * @param isDirectory Is the path a directory?
+   * @param eTag The ETag of the path.
+   * @param isImplicit Is the path implicit?
+   */
+  public PathInformation(Boolean pathExists,
+      Boolean isDirectory,
+      String eTag,
+      Boolean isImplicit) {
+    this.pathExists = pathExists;
+    this.isDirectory = isDirectory;
+    this.eTag = eTag;
+    this.isImplicit = isImplicit;
+  }
+
+  public PathInformation() {
+  }
+
+  /**
+   * Copy the path information.
+   * @param pathInformation The path information to copy.
+   */
+  public void copy(PathInformation pathInformation) {
+    this.pathExists = pathInformation.getPathExists();
+    this.isDirectory = pathInformation.getIsDirectory();
+    this.eTag = pathInformation.getETag();
+    this.isImplicit = pathInformation.getIsImplicit();
+  }
+
+  /**
+   * Get the ETag of the path.
+   *
+   * @return the etag value*/
+  public String getETag() {
+    return eTag;
+  }
+
+  /**
+   * Get value of pathExists.
+   *
+   * @return true if path exists, false otherwise.
+   */
+  public Boolean getPathExists() {
+    return pathExists;
+  }
+
+  /**
+   * Get value of isDirectory.
+   *
+   * @return true if path is a directory, false otherwise.
+   */
+  public Boolean getIsDirectory() {
+    return isDirectory;
+  }
+
+  /**
+   * Get value of isImplicit.
+   *
+   * @return true if path is implicit, false otherwise.
+   */
+  public Boolean getIsImplicit() {
+    return isImplicit;
+  }
+
+  /**
+   * Set the eTag value.
+   *
+   * @param eTag The eTag value to set.
+   */
+  void setETag(String eTag) {
+    this.eTag = eTag;
+  }
+}

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

@@ -0,0 +1,336 @@
+/**
+ * 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.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
+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.contracts.services.BlobAppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
+import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
+import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient.generateBlockListXml;
+
+/**
+ * For a directory enabled for atomic-rename, before rename starts, a file with
+ * -RenamePending.json suffix is created. In this file, the states required for the
+ * rename operation are given. This file is created by {@link #preRename()} method.
+ * This is important in case the JVM process crashes during rename, the atomicity
+ * will be maintained, when the job calls {@link AzureBlobFileSystem#listStatus(Path)}
+ * or {@link AzureBlobFileSystem#getFileStatus(Path)}. On these API calls to filesystem,
+ * it will be checked if there is any RenamePending JSON file. If yes, the crashed rename
+ * operation would be resumed as per the file.
+ */
+public class RenameAtomicity {
+
+  private final TracingContext tracingContext;
+
+  private Path src, dst;
+
+  private String srcEtag;
+
+  private final AbfsBlobClient abfsClient;
+
+  private final Path renameJsonPath;
+
+  public static final String SUFFIX = "-RenamePending.json";
+
+  private int preRenameRetryCount = 0;
+
+  private int renamePendingJsonLen;
+
+  private final ObjectMapper objectMapper = new ObjectMapper();
+
+  private static final Random RANDOM = new Random();
+
+  /**
+   * Performs pre-rename operations. Creates a file with -RenamePending.json
+   * suffix in the source parent directory. This file contains the states
+   * required for the rename operation.
+   *
+   * @param src Source path
+   * @param dst Destination path
+   * @param renameJsonPath Path of the JSON file to be created
+   * @param tracingContext Tracing context
+   * @param srcEtag ETag of the source directory
+   * @param abfsClient AbfsClient instance
+   */
+  public RenameAtomicity(final Path src, final Path dst,
+      final Path renameJsonPath,
+      TracingContext tracingContext,
+      final String srcEtag,
+      final AbfsClient abfsClient) {
+    this.src = src;
+    this.dst = dst;
+    this.abfsClient = (AbfsBlobClient) abfsClient;
+    this.renameJsonPath = renameJsonPath;
+    this.tracingContext = tracingContext;
+    this.srcEtag = srcEtag;
+  }
+
+  /**
+   * Resumes the rename operation from the JSON file.
+   *
+   * @param renameJsonPath Path of the JSON file
+   * @param renamePendingJsonFileLen Length of the JSON file
+   * @param tracingContext Tracing context
+   * @param srcEtag ETag of the source directory
+   * @param abfsClient AbfsClient instance
+   */
+  public RenameAtomicity(final Path renameJsonPath,
+      final int renamePendingJsonFileLen,
+      TracingContext tracingContext,
+      final String srcEtag,
+      final AbfsClient abfsClient) {
+    this.abfsClient = (AbfsBlobClient) abfsClient;
+    this.renameJsonPath = renameJsonPath;
+    this.tracingContext = tracingContext;
+    this.srcEtag = srcEtag;
+    this.renamePendingJsonLen = renamePendingJsonFileLen;
+  }
+
+  /**
+   * Redo the rename operation from the JSON file.
+   *
+   * @throws AzureBlobFileSystemException If the redo operation fails.
+   */
+  public void redo() throws AzureBlobFileSystemException {
+    byte[] buffer = readRenamePendingJson(renameJsonPath, renamePendingJsonLen);
+    String contents = new String(buffer, Charset.defaultCharset());
+    try {
+      final RenamePendingJsonFormat renamePendingJsonFormatObj;
+      try {
+        renamePendingJsonFormatObj = objectMapper.readValue(contents,
+            RenamePendingJsonFormat.class);
+      } catch (JsonProcessingException e) {
+        return;
+      }
+      if (renamePendingJsonFormatObj != null && StringUtils.isNotEmpty(
+          renamePendingJsonFormatObj.getOldFolderName())
+          && StringUtils.isNotEmpty(
+          renamePendingJsonFormatObj.getNewFolderName())
+          && StringUtils.isNotEmpty(renamePendingJsonFormatObj.getETag())) {
+        this.src = new Path(renamePendingJsonFormatObj.getOldFolderName());
+        this.dst = new Path(renamePendingJsonFormatObj.getNewFolderName());
+        this.srcEtag = renamePendingJsonFormatObj.getETag();
+
+        BlobRenameHandler blobRenameHandler = new BlobRenameHandler(
+            this.src.toUri().getPath(), dst.toUri().getPath(),
+            abfsClient, srcEtag, true,
+            true, tracingContext);
+
+        blobRenameHandler.execute();
+      }
+    } finally {
+      deleteRenamePendingJson();
+    }
+  }
+
+  /** Read the JSON file.
+   *
+   * @param path Path of the JSON file
+   * @param len Length of the JSON file
+   * @return Contents of the JSON file
+   * @throws AzureBlobFileSystemException If the read operation fails.
+   */
+  @VisibleForTesting
+  byte[] readRenamePendingJson(Path path, int len)
+      throws AzureBlobFileSystemException {
+    byte[] bytes = new byte[len];
+    abfsClient.read(path.toUri().getPath(), 0, bytes, 0,
+        len, null, null, null,
+        tracingContext);
+    return bytes;
+  }
+
+  /** Generate a random block ID.
+   *
+   * @return Random block ID
+   */
+  public static String generateBlockId() {
+    // PutBlock on the path.
+    byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH];
+    RANDOM.nextBytes(blockIdByteArray);
+    return new String(Base64.encodeBase64(blockIdByteArray),
+        StandardCharsets.UTF_8);
+  }
+
+  /** Create the JSON file with the contents.
+   *
+   * @param path Path of the JSON file
+   * @param bytes Contents of the JSON file
+   * @throws AzureBlobFileSystemException If the create operation fails.
+   */
+  @VisibleForTesting
+  void createRenamePendingJson(Path path, byte[] bytes)
+      throws AzureBlobFileSystemException {
+    // PutBlob on the path.
+    AbfsRestOperation putBlobOp = abfsClient.createPath(path.toUri().getPath(),
+        true,
+        true,
+        null,
+        false,
+        null,
+        null,
+        tracingContext);
+    String eTag = extractEtagHeader(putBlobOp.getResult());
+
+    String blockId = generateBlockId();
+    AppendRequestParameters appendRequestParameters
+        = new AppendRequestParameters(0, 0,
+        bytes.length, AppendRequestParameters.Mode.APPEND_MODE, false, null,
+        abfsClient.getAbfsConfiguration().isExpectHeaderEnabled(),
+        new BlobAppendRequestParameters(blockId, eTag));
+
+    abfsClient.append(path.toUri().getPath(), bytes,
+        appendRequestParameters, null, null, tracingContext);
+
+        List<String> blockIdList = new ArrayList<>(Collections.singleton(blockId));
+        String blockList = generateBlockListXml(blockIdList);
+    // PutBlockList on the path.
+    abfsClient.flush(blockList.getBytes(StandardCharsets.UTF_8),
+        path.toUri().getPath(), true, null, null, eTag, null, tracingContext);
+  }
+
+  /**
+   * Before starting the atomic rename, create a file with -RenamePending.json
+   * suffix in the source parent directory. This file contains the states
+   * required source, destination, and source-eTag for the rename operation.
+   * <p>
+   * If the path that is getting renamed is a /sourcePath, then the JSON file
+   * will be /sourcePath-RenamePending.json.
+   *
+   * @return Length of the JSON file.
+   * @throws AzureBlobFileSystemException If the pre-rename operation fails.
+   */
+  @VisibleForTesting
+  public int preRename() throws AzureBlobFileSystemException {
+    String makeRenamePendingFileContents = makeRenamePendingFileContents(
+        srcEtag);
+
+    try {
+      createRenamePendingJson(renameJsonPath,
+          makeRenamePendingFileContents.getBytes(StandardCharsets.UTF_8));
+      return makeRenamePendingFileContents.length();
+    } catch (AzureBlobFileSystemException e) {
+      /*
+       * Scenario: file has been deleted by parallel thread before the RenameJSON
+       * could be written and flushed. In such case, there has to be one retry of
+       * preRename.
+       * ref: https://issues.apache.org/jira/browse/HADOOP-12678
+       * On DFS endpoint, flush API is called. If file is not there, server returns
+       * 404.
+       * On blob endpoint, flush API is not there. PutBlockList is called with
+       * if-match header. If file is not there, the conditional header will fail,
+       * the server will return 412.
+       */
+      if (isPreRenameRetriableException(e)) {
+        preRenameRetryCount++;
+        if (preRenameRetryCount == 1) {
+          return preRename();
+        }
+      }
+      throw e;
+    }
+  }
+
+  /** Check if the exception is retryable for pre-rename operation.
+   *
+   * @param e Exception to be checked
+   * @return true if the exception is retryable, false otherwise
+   */
+  private boolean isPreRenameRetriableException(IOException e) {
+    AbfsRestOperationException ex;
+    while (e != null) {
+      if (e instanceof AbfsRestOperationException) {
+        ex = (AbfsRestOperationException) e;
+        return ex.getStatusCode() == HTTP_NOT_FOUND
+            || ex.getStatusCode() == HTTP_PRECON_FAILED;
+      }
+      e = (IOException) e.getCause();
+    }
+    return false;
+  }
+
+  /** Delete the JSON file after rename is done.
+   * @throws AzureBlobFileSystemException If the delete operation fails.
+   */
+  public void postRename() throws AzureBlobFileSystemException {
+    deleteRenamePendingJson();
+  }
+
+  /** Delete the JSON file.
+   *
+   * @throws AzureBlobFileSystemException If the delete operation fails.
+   */
+  private void deleteRenamePendingJson() throws AzureBlobFileSystemException {
+    try {
+      abfsClient.deleteBlobPath(renameJsonPath, null,
+          tracingContext);
+    } catch (AzureBlobFileSystemException e) {
+      if (e instanceof AbfsRestOperationException
+          && ((AbfsRestOperationException) e).getStatusCode()
+          == HTTP_NOT_FOUND) {
+        return;
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Return the contents of the JSON file to represent the operations
+   * to be performed for a folder rename.
+   *
+   * @return JSON string which represents the operation.
+   */
+  private String makeRenamePendingFileContents(String eTag) throws
+      AzureBlobFileSystemException {
+
+    final RenamePendingJsonFormat renamePendingJsonFormat
+        = new RenamePendingJsonFormat();
+    renamePendingJsonFormat.setOldFolderName(src.toUri().getPath());
+    renamePendingJsonFormat.setNewFolderName(dst.toUri().getPath());
+    renamePendingJsonFormat.setETag(eTag);
+    try {
+      return objectMapper.writeValueAsString(renamePendingJsonFormat);
+    } catch (JsonProcessingException e) {
+      throw new AbfsDriverException(e);
+    }
+  }
+}

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

@@ -0,0 +1,58 @@
+/**
+ * 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 com.fasterxml.jackson.annotation.JsonProperty;
+
+class RenamePendingJsonFormat {
+
+  @JsonProperty(value = "OldFolderName")
+  private String oldFolderName;
+
+  @JsonProperty(value = "NewFolderName")
+  private String newFolderName;
+
+  @JsonProperty(value = "ETag")
+  private String eTag;
+
+  /** Getters and Setters */
+  String getOldFolderName() {
+    return oldFolderName;
+  }
+
+  String getNewFolderName() {
+    return newFolderName;
+  }
+
+  String getETag() {
+    return eTag;
+  }
+
+  void setOldFolderName(String oldFolderName) {
+    this.oldFolderName = oldFolderName;
+  }
+
+  void setNewFolderName(String newFolderName) {
+    this.newFolderName = newFolderName;
+  }
+
+  void setETag(String eTag) {
+    this.eTag = eTag;
+  }
+}

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

@@ -78,6 +78,8 @@ public class TracingContext {
    */
   private String primaryRequestIdForRetry;
 
+  private Integer operatedBlobCount = null;
+
   private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
   public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72;
   public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*";
@@ -135,6 +137,7 @@ public class TracingContext {
     this.format = originalTracingContext.format;
     this.position = originalTracingContext.getPosition();
     this.ingressHandler = originalTracingContext.getIngressHandler();
+    this.operatedBlobCount = originalTracingContext.operatedBlobCount;
     if (originalTracingContext.listener != null) {
       this.listener = originalTracingContext.listener.getClone();
     }
@@ -196,13 +199,16 @@ public class TracingContext {
               + getPrimaryRequestIdForHeader(retryCount > 0) + ":" + streamID
               + ":" + opType + ":" + retryCount;
       header = addFailureReasons(header, previousFailure, retryPolicyAbbreviation);
-      header += (":" + httpOperation.getTracingContextSuffix());
       if (!(ingressHandler.equals(EMPTY_STRING))) {
         header += ":" + ingressHandler;
       }
       if (!(position.equals(EMPTY_STRING))) {
         header += ":" + position;
       }
+      if (operatedBlobCount != null) {
+        header += (":" + operatedBlobCount);
+      }
+      header += (":" + httpOperation.getTracingContextSuffix());
       metricHeader += !(metricResults.trim().isEmpty()) ? metricResults  : "";
       break;
     case TWO_ID_FORMAT:
@@ -259,6 +265,14 @@ public class TracingContext {
     return String.format("%s_%s", header, previousFailure);
   }
 
+  public void setOperatedBlobCount(Integer count) {
+    operatedBlobCount = count;
+  }
+
+  public FSOperationType getOpType() {
+    return opType;
+  }
+
   /**
    * Return header representing the request associated with the tracingContext
    * @return Header string set into X_MS_CLIENT_REQUEST_ID

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

@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.azurebfs.utils;
 
 import java.io.UnsupportedEncodingException;
 import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URLEncoder;
 import java.nio.charset.StandardCharsets;
@@ -30,7 +32,11 @@ import java.util.List;
 import java.util.Set;
 import java.util.regex.Pattern;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
@@ -48,6 +54,10 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARA
  * Utility class to help with Abfs url transformation to blob urls.
  */
 public final class UriUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      UriUtils.class);
+
   private static final String ABFS_URI_REGEX = "[^.]+\\.dfs\\.(preprod\\.){0,1}core\\.windows\\.net";
   private static final Pattern ABFS_URI_PATTERN = Pattern.compile(ABFS_URI_REGEX);
   private static final Set<String> FULL_MASK_PARAM_KEYS = new HashSet<>(
@@ -230,6 +240,34 @@ public final class UriUtils {
     return sb.toString();
   }
 
+  /**
+   * Checks if the key is for a directory in the set of directories.
+   * @param key the key to check.
+   * @param dirSet the set of directories.
+   * @return true if the key is for a directory in the set of directories.
+   */
+  public static boolean isKeyForDirectorySet(String key, Set<String> dirSet) {
+    for (String dir : dirSet) {
+      if (dir.isEmpty() || key.startsWith(
+          dir + AbfsHttpConstants.FORWARD_SLASH)) {
+        return true;
+      }
+
+      try {
+        URI uri = new URI(dir);
+        if (null == uri.getAuthority()) {
+          if (key.startsWith(dir + "/")) {
+            return true;
+          }
+        }
+      } catch (URISyntaxException e) {
+        LOG.info("URI syntax error creating URI for {}", dir);
+      }
+    }
+
+    return false;
+  }
+
   private UriUtils() {
   }
 }

+ 13 - 21
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java

@@ -28,15 +28,6 @@ import java.util.Hashtable;
 import java.util.List;
 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;
 import org.assertj.core.api.Assumptions;
 import org.junit.Test;
@@ -46,16 +37,25 @@ import org.junit.runners.Parameterized;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobAppendRequestParameters;
 import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
 import org.apache.hadoop.fs.azurebfs.extensions.MockEncryptionContextProvider;
 import org.apache.hadoop.fs.azurebfs.security.ContextProviderEncryptionAdapter;
+import org.apache.hadoop.fs.azurebfs.security.EncodingHelper;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClientUtils;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.utils.EncryptionType;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -63,10 +63,7 @@ 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;
@@ -77,6 +74,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ENCRYPTION_KEY_LEN;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient.generateBlockListXml;
 import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
 import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.ENCRYPTION_CONTEXT;
 import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.GLOBAL_KEY;
@@ -191,12 +189,6 @@ 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");
@@ -328,7 +320,7 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
           return ingressClient.flush(path, 3, false, false, null,
               null, encryptionAdapter, getTestTracingContext(fs, false));
         } else {
-          byte[] buffer = generateBlockListXml().getBytes(StandardCharsets.UTF_8);
+          byte[] buffer = generateBlockListXml(new ArrayList<>()).getBytes(StandardCharsets.UTF_8);
           return ingressClient.flush(buffer, path, false, null,
               null, null, encryptionAdapter, getTestTracingContext(fs, false));
         }
@@ -354,11 +346,11 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
       case RENAME:
         TracingContext tc = getTestTracingContext(fs, true);
         return client.renamePath(path, new Path(path + "_2").toString(),
-          null, tc, null, false, fs.getIsNamespaceEnabled(tc)).getOp();
+          null, tc, null, false).getOp();
       case DELETE:
         TracingContext testTC = getTestTracingContext(fs, false);
         return client.deletePath(path, false, null,
-            testTC, fs.getIsNamespaceEnabled(testTC));
+            testTC);
       case GET_ATTR:
         return client.getPathStatus(path, true,
             getTestTracingContext(fs, false),

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

@@ -99,7 +99,6 @@ public class ITestAbfsStatistics extends AbstractAbfsIntegrationTest {
     getFileStatus is called 1 time after creating file and 1 time at time of
     initialising.
      */
-    assertAbfsStatistics(AbfsStatistic.CALL_CREATE, 1, metricMap);
     assertAbfsStatistics(AbfsStatistic.CALL_CREATE_NON_RECURSIVE, 1, metricMap);
     assertAbfsStatistics(AbfsStatistic.FILES_CREATED, 1, metricMap);
     assertAbfsStatistics(AbfsStatistic.DIRECTORIES_CREATED, 1, metricMap);
@@ -126,7 +125,6 @@ public class ITestAbfsStatistics extends AbstractAbfsIntegrationTest {
     getFileStatus is called 1 time at initialise() plus number of times file
     is created.
      */
-    assertAbfsStatistics(AbfsStatistic.CALL_CREATE, NUMBER_OF_OPS, metricMap);
     assertAbfsStatistics(AbfsStatistic.CALL_CREATE_NON_RECURSIVE, NUMBER_OF_OPS,
         metricMap);
     assertAbfsStatistics(AbfsStatistic.FILES_CREATED, NUMBER_OF_OPS, metricMap);

+ 162 - 18
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java

@@ -37,41 +37,45 @@ import org.mockito.Mockito;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataOutputStream;
 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;
-import org.apache.hadoop.test.ReflectionUtils;
-
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException;
+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.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.RenameAtomicity;
+import org.apache.hadoop.fs.azurebfs.utils.DirectoryStateHelper;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.test.ReflectionUtils;
 
 import static java.net.HttpURLConnection.HTTP_CONFLICT;
 import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
 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.AbfsStatistic.CONNECTIONS_MADE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_MKDIR_OVERWRITE;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicity.SUFFIX;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.ArgumentMatchers.nullable;
@@ -79,10 +83,6 @@ import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
-import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
-
 /**
  * Test create operation.
  */
@@ -137,7 +137,9 @@ public class ITestAzureBlobFileSystemCreate extends
     Path testFolderPath = path(TEST_FOLDER_PATH);
     Path testFile = new Path(testFolderPath, TEST_CHILD_FILE);
     try {
-      fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null);
+      fs.createNonRecursive(testFile, FsPermission.getDefault(),
+          EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1,
+          1024, null);
       fail("Should've thrown");
     } catch (FileNotFoundException expected) {
     }
@@ -156,7 +158,8 @@ public class ITestAzureBlobFileSystemCreate extends
     Path testFolderPath = path(TEST_FOLDER_PATH);
     Path testFile = new Path(testFolderPath, TEST_CHILD_FILE);
     try {
-      fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null);
+      fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024,
+          (short) 1, 1024, null);
       fail("Should've thrown");
     } catch (FileNotFoundException e) {
     }
@@ -166,6 +169,147 @@ public class ITestAzureBlobFileSystemCreate extends
     assertIsFile(fs, testFile);
   }
 
+  /**
+   * Test createNonRecursive when parent exist.
+   *
+   * @throws Exception in case of failure
+   */
+  @Test
+  public void testCreateNonRecursiveWhenParentExist() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path createDirectoryPath = new Path("hbase/A");
+    fs.mkdirs(createDirectoryPath);
+    fs.createNonRecursive(new Path(createDirectoryPath, "B"), FsPermission
+        .getDefault(), false, 1024,
+        (short) 1, 1024, null);
+    Assertions.assertThat(fs.exists(new Path(createDirectoryPath, "B")))
+        .describedAs("File should be created").isTrue();
+    fs.close();
+  }
+
+  /**
+   * Test createNonRecursive when parent does not exist.
+   *
+   * @throws Exception in case of failure
+   */
+  @Test
+  public void testCreateNonRecursiveWhenParentNotExist() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path createDirectoryPath = new Path("A/");
+    fs.mkdirs(createDirectoryPath);
+    intercept(FileNotFoundException.class,
+        () -> fs.createNonRecursive(new Path("A/B/C"), FsPermission
+            .getDefault(), false, 1024, (short) 1, 1024, null));
+    Assertions.assertThat(fs.exists(new Path("A/B/C")))
+        .describedAs("New File should not be created.").isFalse();
+    fs.close();
+  }
+
+  /**
+   * Helper method to create a json file.
+   * @param path parent path
+   * @param renameJson rename json path
+   *
+   * @return file system
+   * @throws IOException in case of failure
+   */
+  private AzureBlobFileSystem createJsonFile(Path path, Path renameJson) throws IOException {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    AbfsClient client = Mockito.spy(store.getClient());
+    Mockito.doReturn(client).when(store).getClient();
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    fs.mkdirs(new Path(path, "test3"));
+    AzureBlobFileSystemStore.VersionedFileStatus fileStatus
+        = (AzureBlobFileSystemStore.VersionedFileStatus) fs.getFileStatus(path);
+    new RenameAtomicity(path,
+        new Path("/hbase/test4"), renameJson,
+        getTestTracingContext(fs, true), fileStatus.getEtag(),
+        client).preRename();
+    Assertions.assertThat(fs.exists(renameJson))
+        .describedAs("Rename Pending Json file should exist.")
+        .isTrue();
+    return fs;
+  }
+
+  /**
+   * Test createNonRecursive when parent does not exist and rename pending exists.
+   * Rename redo should fail.
+   * Json file should be deleted.
+   * No new File creation.
+   *
+   * @throws Exception in case of failure
+   */
+  @Test
+  public void testCreateNonRecursiveWhenParentNotExistAndRenamePendingExist() throws Exception {
+    AzureBlobFileSystem fs = null;
+    try {
+      Path path = new Path("/hbase/test1/test2");
+      Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX);
+      fs = createJsonFile(path, renameJson);
+      fs.delete(path, true);
+      Assertions.assertThat(fs.exists(renameJson)).isTrue();
+      AzureBlobFileSystem finalFs = fs;
+      intercept(FileNotFoundException.class,
+          () -> finalFs.createNonRecursive(new Path(path, "test4"), FsPermission
+              .getDefault(), false, 1024, (short) 1, 1024, null));
+      Assertions.assertThat(finalFs.exists(new Path(path, "test4")))
+          .describedAs("New File should not be created.")
+          .isFalse();
+      Assertions.assertThat(finalFs.exists(renameJson))
+          .describedAs("Rename Pending Json file should be deleted.")
+          .isFalse();
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+    }
+  }
+
+  /**
+   * Test createNonRecursive when parent and rename pending exist.
+   * Rename redo should be successful.
+   * Json file should be deleted.
+   * No file should be created.
+   *
+   * @throws Exception in case of failure
+   */
+  @Test
+  public void testCreateNonRecursiveWhenParentAndRenamePendingExist() throws Exception {
+    AzureBlobFileSystem fs = null;
+    try {
+      Path path = new Path("/hbase/test1/test2");
+      Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX);
+      fs = createJsonFile(path, renameJson);
+      AzureBlobFileSystem finalFs = fs;
+      intercept(FileNotFoundException.class,
+          () -> finalFs.createNonRecursive(new Path(path, "test4"), FsPermission
+              .getDefault(), false, 1024, (short) 1, 1024, null));
+      Assertions.assertThat(finalFs.exists(path))
+          .describedAs("Old path should be deleted.")
+          .isFalse();
+      Assertions.assertThat(finalFs.exists(new Path(path, "test4")))
+          .describedAs("New File should not be created.")
+          .isFalse();
+      Assertions.assertThat(finalFs.exists(renameJson))
+          .describedAs("Rename Pending Json file should be deleted.")
+          .isFalse();
+      Assertions.assertThat(finalFs.exists(new Path("/hbase/test4")))
+          .describedAs("Rename should be successful.")
+          .isTrue();
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+    }
+  }
+
   @Test
   public void testCreateOnRoot() throws Exception {
     final AzureBlobFileSystem fs = getFileSystem();

+ 5 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java

@@ -403,7 +403,8 @@ public class ITestAzureBlobFileSystemDelegationSAS extends AbstractAbfsIntegrati
     fs.create(new Path(src)).close();
     AbfsRestOperation abfsHttpRestOperation = fs.getAbfsClient()
         .renamePath(src, "/testABC" + "/abc.txt", null,
-            getTestTracingContext(fs, false), null, false, isHNSEnabled)
+            getTestTracingContext(fs, false), null,
+            false)
         .getOp();
     AbfsHttpOperation result = abfsHttpRestOperation.getResult();
     String url = result.getMaskedUrl();
@@ -420,8 +421,9 @@ public class ITestAzureBlobFileSystemDelegationSAS extends AbstractAbfsIntegrati
   public void testSignatureMaskOnExceptionMessage() throws Exception {
     intercept(IOException.class, "sig=XXXX",
         () -> getFileSystem().getAbfsClient()
-            .renamePath("testABC/test.xt", "testABC/abc.txt", null,
-                getTestTracingContext(getFileSystem(), false), null, false, isHNSEnabled));
+            .renamePath("testABC/test.xt", "testABC/abc.txt",
+                null, getTestTracingContext(getFileSystem(), false),
+                null, false));
   }
 
   @Test

+ 313 - 16
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.fs.azurebfs;
 
 import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -32,39 +34,42 @@ import org.junit.Test;
 import org.mockito.Mockito;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.StorageErrorResponseSchema;
+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.AbfsClientTestUtil;
-import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.TestAbfsPerfTracker;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.ReflectionUtils;
 
 import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
+import static java.net.HttpURLConnection.HTTP_FORBIDDEN;
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_OK;
-
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT;
 import static org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType.DeletePath;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * Test delete operation.
@@ -74,6 +79,8 @@ public class ITestAzureBlobFileSystemDelete extends
 
   private static final int REDUCED_RETRY_COUNT = 1;
   private static final int REDUCED_MAX_BACKOFF_INTERVALS_MS = 5000;
+  private static final int MAX_ITERATIONS = 20;
+  private static final int BLOB_COUNT = 11;
 
   public ITestAzureBlobFileSystemDelete() throws Exception {
     super();
@@ -161,7 +168,7 @@ public class ITestAzureBlobFileSystemDelete extends
         fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
         fs.getFileSystemId(), FSOperationType.DELETE, false, 0));
     // first try a non-recursive delete, expect failure
-    intercept(FileAlreadyExistsException.class,
+    intercept(IOException.class,
         () -> fs.delete(dir, false));
     fs.registerListener(null);
     assertDeleted(fs, dir, true);
@@ -227,6 +234,13 @@ public class ITestAzureBlobFileSystemDelete extends
   public void testDeleteIdempotencyTriggerHttp404() throws Exception {
 
     final AzureBlobFileSystem fs = getFileSystem();
+    /*
+     * Delete call for a folder on DFS endpoint is one-server call and the
+     * orchestration of delete of paths inside the directory. For Blob
+     * endpoint, the orchestration would be done by the client. The idempotency
+     * issue would not happen for blob endpoint.
+     */
+    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
     AbfsClient client = ITestAbfsClient.createTestClientFromCurrentContext(
         fs.getAbfsStore().getClient(),
         this.getConfiguration());
@@ -243,8 +257,7 @@ public class ITestAzureBlobFileSystemDelete extends
         "/NonExistingPath",
         false,
         null,
-        getTestTracingContext(fs, true),
-        fs.getIsNamespaceEnabled(getTestTracingContext(fs, true))));
+        getTestTracingContext(fs, true)));
 
     // mock idempotency check to mimic retried case
     AbfsClient mockClient = ITestAbfsClient.getMockAbfsClient(
@@ -274,15 +287,32 @@ public class ITestAzureBlobFileSystemDelete extends
     doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any());
     TracingContext tracingContext = getTestTracingContext(fs, false);
     doReturn(tracingContext).when(idempotencyRetOp).createNewTracingContext(any());
+    if (mockClient instanceof AbfsBlobClient) {
+      doCallRealMethod().when((AbfsBlobClient) mockClient)
+              .getBlobDeleteHandler(Mockito.nullable(String.class),
+                      Mockito.anyBoolean(), Mockito.nullable(TracingContext.class));
+      doCallRealMethod().when(mockClient)
+              .listPath(Mockito.nullable(String.class), Mockito.anyBoolean(),
+                      Mockito.anyInt(), Mockito.nullable(String.class),
+                      Mockito.nullable(TracingContext.class));
+      doCallRealMethod().when((AbfsBlobClient) mockClient)
+              .listPath(Mockito.nullable(String.class), Mockito.anyBoolean(),
+                      Mockito.anyInt(), Mockito.nullable(String.class),
+                      Mockito.nullable(TracingContext.class),
+                      Mockito.anyBoolean());
+      doCallRealMethod().when((AbfsBlobClient) mockClient)
+              .getPathStatus(Mockito.nullable(String.class), Mockito.nullable(TracingContext.class),
+                      Mockito.nullable(ContextEncryptionAdapter.class), Mockito.anyBoolean());
+    }
     when(mockClient.deletePath("/NonExistingPath", false, null,
-        tracingContext, fs.getIsNamespaceEnabled(tracingContext)))
+        tracingContext))
         .thenCallRealMethod();
 
     Assertions.assertThat(mockClient.deletePath(
         "/NonExistingPath",
         false,
         null,
-        tracingContext, fs.getIsNamespaceEnabled(tracingContext))
+        tracingContext)
         .getResult()
         .getStatusCode())
         .describedAs("Idempotency check reports successful "
@@ -314,4 +344,271 @@ public class ITestAzureBlobFileSystemDelete extends
     fs.delete(new Path("/testDir"), true);
     fs.close();
   }
+
+  /**
+   * Tests deleting an implicit directory and its contents. The test verifies that after deletion,
+   * both the directory and its child file no longer exist.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testDeleteImplicitDir() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    fs.mkdirs(new Path("/testDir/dir1"));
+    fs.create(new Path("/testDir/dir1/file1"));
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient();
+    client.deleteBlobPath(new Path("/testDir/dir1"),
+            null, getTestTracingContext(fs, true));
+    fs.delete(new Path("/testDir/dir1"), true);
+    Assertions.assertThat(!fs.exists(new Path("/testDir/dir1")))
+            .describedAs("FileStatus of the deleted directory should not exist")
+            .isTrue();
+    Assertions.assertThat(!fs.exists(new Path("/testDir/dir1/file1")))
+            .describedAs("Child of a deleted directory should not be present")
+            .isTrue();
+  }
+
+  /**
+   * Tests deleting an implicit directory when a single list result is returned.
+   * The test verifies that the directory is properly deleted and no residual file status remains.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testDeleteImplicitDirWithSingleListResults() throws Exception {
+    AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
+            getRawConfiguration());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient();
+    AbfsBlobClient spiedClient = Mockito.spy(client);
+    fs.getAbfsStore().setClient(spiedClient);
+    fs.mkdirs(new Path("/testDir/dir1"));
+    for (int i = 0; i < 10; i++) {
+      fs.create(new Path("/testDir/dir1/file" + i));
+    }
+    Mockito.doAnswer(answer -> {
+              String path = answer.getArgument(0);
+              boolean recursive = answer.getArgument(1);
+              String continuation = answer.getArgument(3);
+              TracingContext context = answer.getArgument(4);
+              return client.listPath(path, recursive, 1, continuation, context);
+            })
+            .when(spiedClient)
+            .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(),
+                    Mockito.nullable(String.class),
+                    Mockito.any(TracingContext.class));
+    client.deleteBlobPath(new Path("/testDir/dir1"),
+            null, getTestTracingContext(fs, true));
+    fs.delete(new Path("/testDir/dir1"), true);
+    Assertions.assertThat(fs.exists(new Path("/testDir/dir1")))
+            .describedAs("FileStatus of the deleted directory should not exist")
+            .isFalse();
+  }
+
+  /**
+   * Tests deleting an explicit directory within an implicit parent directory.
+   * It verifies that the directory and its contents are deleted, while the parent directory remains.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testDeleteExplicitDirInImplicitParentDir() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient();
+    fs.mkdirs(new Path("/testDir/dir1"));
+    fs.create(new Path("/testDir/dir1/file1"));
+    client.deleteBlobPath(new Path("/testDir/"),
+            null, getTestTracingContext(fs, true));
+    fs.delete(new Path("/testDir/dir1"), true);
+    Assertions.assertThat(fs.exists(new Path("/testDir/dir1")))
+            .describedAs("Deleted directory should not exist")
+            .isFalse();
+    Assertions.assertThat(fs.exists(new Path("/testDir/dir1/file1")))
+            .describedAs("Child of a deleted directory should not be present")
+            .isFalse();
+    Assertions.assertThat(fs.exists(new Path("/testDir")))
+            .describedAs("Parent Implicit directory should exist")
+            .isTrue();
+  }
+
+  /**
+   * Tests handling of a parallel delete operation failure when deleting multiple files in a directory.
+   * The test verifies that an `AccessDeniedException` is thrown when the delete operation fails
+   * due to an `AbfsRestOperationException`.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testDeleteParallelBlobFailure() throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = Mockito.spy((AbfsBlobClient) fs.getAbfsClient());
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    store.setClient(client);
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    fs.mkdirs(new Path("/testDir"));
+    fs.create(new Path("/testDir/file1"));
+    fs.create(new Path("/testDir/file2"));
+    fs.create(new Path("/testDir/file3"));
+    Mockito.doThrow(
+                    new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", new Exception()))
+            .when(client)
+            .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class),
+                    Mockito.any(TracingContext.class));
+    LambdaTestUtils.intercept(
+            AccessDeniedException.class,
+            () -> {
+              fs.delete(new Path("/testDir"), true);
+            });
+  }
+
+  /**
+   * Tests deleting the root directory without recursion. The test verifies that
+   * the delete operation returns `false` since the root directory cannot be deleted
+   * without recursion.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testDeleteRootWithNonRecursion() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testDir"));
+    Assertions.assertThat(fs.delete(new Path(ROOT_PATH), false)).isFalse();
+  }
+
+  /**
+   * Tests that the producer stops when a delete operation fails with an `AccessDeniedException`.
+   * The test simulates a failure during a delete operation and verifies that the system properly
+   * handles the exception by stopping further actions.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testProducerStopOnDeleteFailure() throws Exception {
+    assumeBlobServiceType();
+    Configuration configuration = Mockito.spy(getRawConfiguration());
+    AzureBlobFileSystem fs = Mockito.spy(
+            (AzureBlobFileSystem) FileSystem.get(configuration));
+    fs.mkdirs(new Path("/src"));
+    ExecutorService executorService = Executors.newFixedThreadPool(10);
+    List<Future> futureList = new ArrayList<>();
+    for (int i = 0; i < MAX_ITERATIONS; i++) {
+      int iter = i;
+      Future future = executorService.submit(() -> {
+        try {
+          fs.create(new Path("/src/file" + iter));
+        } catch (IOException ex) {}
+      });
+      futureList.add(future);
+    }
+    for (Future future : futureList) {
+      future.get();
+    }
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient();
+    AbfsBlobClient spiedClient = Mockito.spy(client);
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    store.setClient(spiedClient);
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    final int[] deleteCallInvocation = new int[1];
+    Mockito.doAnswer(answer -> {
+              throw new AbfsRestOperationException(HTTP_FORBIDDEN, "", "",
+                      new Exception());
+            }).when(spiedClient)
+            .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class),
+                    Mockito.any(TracingContext.class));
+    AbfsClientTestUtil.mockGetDeleteBlobHandler(spiedClient,
+            (blobDeleteHandler) -> {
+              Mockito.doAnswer(answer -> {
+                        try {
+                          answer.callRealMethod();
+                        } catch (AbfsRestOperationException ex) {
+                          if (ex.getStatusCode() == HTTP_FORBIDDEN) {
+                            deleteCallInvocation[0]++;
+                          }
+                          throw ex;
+                        }
+                        throw new AssertionError("List Consumption should have failed");
+                      })
+                      .when(blobDeleteHandler).listRecursiveAndTakeAction();
+              return null;
+            });
+    final int[] listCallInvocation = new int[1];
+    Mockito.doAnswer(answer -> {
+              if (listCallInvocation[0] == 1) {
+                while (deleteCallInvocation[0] == 0) {}
+              }
+              listCallInvocation[0]++;
+              return answer.callRealMethod();
+            })
+            .when(spiedClient)
+            .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(),
+                    Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+    intercept(AccessDeniedException.class,
+            () -> {
+              fs.delete(new Path("/src"), true);
+            });
+    Mockito.verify(spiedClient, Mockito.times(1))
+            .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(),
+                    Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+  }
+
+  /**
+   * Test to assert that the CID in src marker delete contains the
+   * total number of blobs operated in the delete directory.
+   * Also, to assert that all operations in the delete-directory flow have same
+   * primaryId and opType.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testDeleteEmitDeletionCountInClientRequestId() throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient();
+    AbfsBlobClient spiedClient = Mockito.spy(client);
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    store.setClient(spiedClient);
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    String dirPathStr = "/testDir/dir1";
+    fs.mkdirs(new Path(dirPathStr));
+    ExecutorService executorService = Executors.newFixedThreadPool(5);
+    List<Future> futures = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      final int iter = i;
+      Future future = executorService.submit(() -> fs.create(new Path("/testDir/dir1/file" + iter)));
+      futures.add(future);
+    }
+    for (Future future : futures) {
+      future.get();
+    }
+    executorService.shutdown();
+    final TracingHeaderValidator tracingHeaderValidator
+            = new TracingHeaderValidator(
+            fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
+            fs.getFileSystemId(), FSOperationType.DELETE, false, 0);
+    fs.registerListener(tracingHeaderValidator);
+    Mockito.doAnswer(answer -> {
+              Mockito.doAnswer(deleteAnswer -> {
+                        if (dirPathStr.equalsIgnoreCase(
+                                ((Path) deleteAnswer.getArgument(0)).toUri().getPath())) {
+                          tracingHeaderValidator.setOperatedBlobCount(BLOB_COUNT);
+                          Object result = deleteAnswer.callRealMethod();
+                          tracingHeaderValidator.setOperatedBlobCount(null);
+                          return result;
+                        }
+                        return deleteAnswer.callRealMethod();
+                      })
+                      .when(spiedClient)
+                      .deleteBlobPath(Mockito.any(Path.class),
+                              Mockito.nullable(String.class),
+                              Mockito.any(TracingContext.class));
+              return answer.callRealMethod();
+            })
+            .when(store)
+            .delete(Mockito.any(Path.class), Mockito.anyBoolean(),
+                    Mockito.any(TracingContext.class));
+    fs.delete(new Path(dirPathStr), true);
+  }
 }

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

@@ -58,6 +58,7 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
   private static final int TEST_STABLE_DEFAULT_CONNECTION_TIMEOUT_MS = 500;
   private static final int TEST_STABLE_DEFAULT_READ_TIMEOUT_MS = 30000;
   private static final int TEST_UNSTABLE_READ_TIMEOUT_MS = 1;
+  private static final int TEST_WRITE_BYTE_VALUE = 20;
 
   public ITestAzureBlobFileSystemE2E() throws Exception {
     super();
@@ -234,7 +235,7 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
 
       fs.delete(testFilePath, true);
       assertPathDoesNotExist(fs, "This path should not exist", testFilePath);
-
+      stream.write(TEST_WRITE_BYTE_VALUE);
       if (client instanceof AbfsDfsClient) {
         intercept(FileNotFoundException.class, stream::close);
       } else {

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

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

+ 14 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java

@@ -28,6 +28,7 @@ import org.junit.Test;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
@@ -42,6 +43,7 @@ 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.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION;
 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;
@@ -68,6 +70,7 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
   private static final int LONG_TEST_EXECUTION_TIMEOUT = 90 * 1000;
   private static final String TEST_FILE = "testfile";
   private final boolean isHNSEnabled;
+  private static final int TEST_BYTES = 20;
 
   public ITestAzureBlobFileSystemLease() throws Exception {
     super();
@@ -259,6 +262,9 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
 
     LambdaTestUtils.intercept(IOException.class, client instanceof AbfsBlobClient
         ? ERR_LEASE_EXPIRED_BLOB : ERR_LEASE_EXPIRED, () -> {
+      if (isAppendBlobEnabled() && getIngressServiceType() == AbfsServiceType.BLOB) {
+        out.write(TEST_BYTES);
+      }
       out.close();
       return "Expected exception on close after lease break but got " + out;
     });
@@ -370,7 +376,8 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
     tracingContext.setListener(listener);
 
     AbfsLease lease = new AbfsLease(fs.getAbfsClient(),
-        testFilePath.toUri().getPath(), tracingContext);
+            testFilePath.toUri().getPath(), true, INFINITE_LEASE_DURATION,
+            null, tracingContext);
     Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID());
     listener.setOperation(FSOperationType.RELEASE_LEASE);
     lease.free();
@@ -382,19 +389,20 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
     doThrow(new AbfsLease.LeaseException("failed to acquire 1"))
         .doThrow(new AbfsLease.LeaseException("failed to acquire 2"))
         .doCallRealMethod().when(mockClient)
-        .acquireLease(anyString(), anyInt(), any(TracingContext.class));
+        .acquireLease(anyString(), anyInt(), any(), any(TracingContext.class));
 
-    lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1, tracingContext);
+    lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), true, 5, 1,
+            INFINITE_LEASE_DURATION, null, tracingContext);
     Assert.assertNotNull("Acquire lease should have retried", lease.getLeaseID());
     lease.free();
     Assert.assertEquals("Unexpected acquire retry count", 2, lease.getAcquireRetryCount());
 
     doThrow(new AbfsLease.LeaseException("failed to acquire")).when(mockClient)
-        .acquireLease(anyString(), anyInt(), any(TracingContext.class));
+        .acquireLease(anyString(), anyInt(), any(), any(TracingContext.class));
 
     LambdaTestUtils.intercept(AzureBlobFileSystemException.class, () -> {
-      new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1,
-          tracingContext);
+      new AbfsLease(mockClient, testFilePath.toUri().getPath(), true, 5, 1,
+              INFINITE_LEASE_DURATION, null, tracingContext);
     });
   }
 }

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

@@ -28,6 +28,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
+import org.junit.Assume;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.stubbing.Stubber;
@@ -38,6 +39,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.contracts.services.DfsListResultEntrySchema;
@@ -122,6 +124,7 @@ public class ITestAzureBlobFileSystemListStatus extends
    */
   @Test
   public void testListPathTracingContext() throws Exception {
+    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
     final AzureBlobFileSystem fs = getFileSystem();
     final AzureBlobFileSystem spiedFs = Mockito.spy(fs);
     final AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore());

+ 1448 - 5
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java

@@ -18,22 +18,64 @@
 
 package org.apache.hadoop.fs.azurebfs;
 
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
-import org.junit.Assert;
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.stubbing.Answer;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+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.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.AbfsClientTestUtil;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsLease;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.BlobRenameHandler;
+import org.apache.hadoop.fs.azurebfs.services.RenameAtomicity;
+import org.apache.hadoop.fs.azurebfs.services.RenameAtomicityTestUtils;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
 import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.util.functional.FunctionRaisingIOE;
 
+import static java.net.HttpURLConnection.HTTP_CONFLICT;
+import static java.net.HttpURLConnection.HTTP_FORBIDDEN;
+import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_ABORTED;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_FAILED;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_PENDING;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_ABORTED;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_FAILED;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND;
+import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicity.SUFFIX;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
@@ -41,6 +83,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test rename operation.
@@ -48,6 +91,10 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
 public class ITestAzureBlobFileSystemRename extends
     AbstractAbfsIntegrationTest {
 
+  private static final int MAX_ITERATIONS = 20;
+
+  private static final int BLOB_COUNT = 11;
+
   public ITestAzureBlobFileSystemRename() throws Exception {
     super();
   }
@@ -160,7 +207,8 @@ public class ITestAzureBlobFileSystemRename extends
     Path testDir2 = path("testDir2");
     fs.mkdirs(new Path(testDir2 + "/test1/test2/test3"));
     fs.mkdirs(new Path(testDir2 + "/test4"));
-    Assert.assertTrue(fs.rename(new Path(testDir2 + "/test1/test2/test3"), new Path(testDir2 + "/test4")));
+    assertTrue(fs.rename(new Path(testDir2 + "/test1/test2/test3"),
+        new Path(testDir2 + "/test4")));
     assertPathExists(fs, "This path should exist", testDir2);
     assertPathExists(fs, "This path should exist",
         new Path(testDir2 + "/test1/test2"));
@@ -192,10 +240,1405 @@ public class ITestAzureBlobFileSystemRename extends
     // Verify that metadata was in an incomplete state after the rename
     // failure, and we retired the rename once more.
     IOStatistics ioStatistics = fs.getIOStatistics();
+    AbfsClient client = fs.getAbfsStore().getClient();
     IOStatisticAssertions.assertThatStatisticCounter(ioStatistics,
-        RENAME_PATH_ATTEMPTS.getStatName())
-        .describedAs("There should be 2 rename attempts if metadata "
-            + "incomplete state failure is hit")
+            RENAME_PATH_ATTEMPTS.getStatName())
+        .describedAs("For Dfs endpoint: There should be 2 rename "
+            + "attempts if metadata incomplete state failure is hit."
+            + "For Blob endpoint: There would be only one rename attempt which "
+            + "would have a failed precheck.")
+        .isEqualTo(client instanceof AbfsDfsClient ? 2 : 1);
+  }
+
+  /**
+   * Tests renaming a directory to the root directory. This test ensures that a directory can be renamed
+   * successfully to the root directory and that the renamed directory appears as expected.
+   *
+   * The test creates a directory (`/src1/src2`), renames it to the root (`/`), and verifies that
+   * the renamed directory (`/src2`) exists in the root.
+   *
+   * @throws Exception if an error occurs during test execution
+   */
+  @Test
+  public void testRenameToRoot() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/src1/src2"));
+    assertTrue(fs.rename(new Path("/src1/src2"), new Path("/")));
+    assertTrue(fs.exists(new Path("/src2")));
+  }
+
+  /**
+   * Tests renaming a non-existent file to the root directory. This test ensures that the rename
+   * operation returns `false` when attempting to rename a file that does not exist.
+   *
+   * The test attempts to rename a file located at `/file` (which does not exist) to the root directory `/`
+   * and verifies that the rename operation fails.
+   *
+   * @throws Exception if an error occurs during test execution
+   */
+  @Test
+  public void testRenameNotFoundBlobToEmptyRoot() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assertFalse(fs.rename(new Path("/file"), new Path("/")));
+  }
+
+  /**
+   * Tests renaming a source path to a destination path that contains a colon in the path.
+   * This verifies that the rename operation handles paths with special characters like a colon.
+   *
+   * The test creates a source directory and renames it to a destination path that includes a colon,
+   * ensuring that the operation succeeds without errors.
+   *
+   * @throws Exception if an error occurs during test execution
+   */
+  @Test(expected = IOException.class)
+  public void testRenameBlobToDstWithColonInPath() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    fs.create(new Path("/src"));
+    fs.rename(new Path("/src"), new Path("/dst:file"));
+  }
+
+  /**
+   * Tests renaming a directory within the same parent directory when there is no marker file.
+   * This test ensures that the rename operation succeeds even when no special marker file is present.
+   *
+   * The test creates a file in a directory, deletes the blob path using the client, and then attempts
+   * to rename the directory. It verifies that the rename operation completes successfully.
+   *
+   * @throws Exception if an error occurs during test execution
+   */
+  @Test
+  public void testRenameBlobInSameDirectoryWithNoMarker() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsStore().getClient();
+    fs.create(new Path("/srcDir/dir/file"));
+    client.deleteBlobPath(new Path("/srcDir/dir"), null,
+        getTestTracingContext(fs, true));
+    assertTrue(fs.rename(new Path("/srcDir/dir"), new Path("/srcDir")));
+  }
+
+  /**
+   * <pre>
+   * Test to check behaviour of rename API if the destination directory is already
+   * there. The HNS call and the one for Blob endpoint should have same behaviour.
+   *
+   * /testDir2/test1/test2/test3 contains (/file)
+   * There is another path that exists: /testDir2/test4/test3
+   * On rename(/testDir2/test1/test2/test3, /testDir2/test4).
+   * </pre>
+   *
+   * Expectation for HNS / Blob endpoint:<ol>
+   * <li>Rename should fail</li>
+   * <li>No file should be transferred to destination directory</li>
+   * </ol>
+   */
+  @Test
+  public void testPosixRenameDirectoryWhereDirectoryAlreadyThereOnDestination()
+      throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("testDir2/test1/test2/test3"));
+    fs.create(new Path("testDir2/test1/test2/test3/file"));
+    fs.mkdirs(new Path("testDir2/test4/test3"));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    assertFalse(fs.rename(new Path("testDir2/test1/test2/test3"),
+        new Path("testDir2/test4")));
+    assertTrue(fs.exists(new Path("testDir2")));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2")));
+    assertTrue(fs.exists(new Path("testDir2/test4")));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2/test3")));
+    if (getIsNamespaceEnabled(fs)
+        || fs.getAbfsClient() instanceof AbfsBlobClient) {
+      assertFalse(fs.exists(new Path("testDir2/test4/test3/file")));
+      assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    } else {
+      assertTrue(fs.exists(new Path("testDir2/test4/test3/file")));
+      assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    }
+  }
+
+  /**
+   * <pre>
+   * Test to check behaviour of rename API if the destination directory is already
+   * there. The HNS call and the one for Blob endpoint should have same behaviour.
+   *
+   * /testDir2/test1/test2/test3 contains (/file)
+   * There is another path that exists: /testDir2/test4/test3
+   * On rename(/testDir2/test1/test2/test3, /testDir2/test4).
+   * </pre>
+   *
+   * Expectation for HNS / Blob endpoint:<ol>
+   * <li>Rename should fail</li>
+   * <li>No file should be transferred to destination directory</li>
+   * </ol>
+   */
+  @Test
+  public void testPosixRenameDirectoryWherePartAlreadyThereOnDestination()
+      throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("testDir2/test1/test2/test3"));
+    fs.create(new Path("testDir2/test1/test2/test3/file"));
+    fs.create(new Path("testDir2/test1/test2/test3/file1"));
+    fs.mkdirs(new Path("testDir2/test4/"));
+    fs.create(new Path("testDir2/test4/file1"));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file1")));
+    assertTrue(fs.rename(new Path("testDir2/test1/test2/test3"),
+        new Path("testDir2/test4")));
+    assertTrue(fs.exists(new Path("testDir2")));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2")));
+    assertTrue(fs.exists(new Path("testDir2/test4")));
+    assertFalse(fs.exists(new Path("testDir2/test1/test2/test3")));
+    assertFalse(fs.exists(new Path("testDir2/test4/file")));
+    assertTrue(fs.exists(new Path("testDir2/test4/file1")));
+    assertTrue(fs.exists(new Path("testDir2/test4/test3/file")));
+    assertTrue(fs.exists(new Path("testDir2/test4/test3/file1")));
+    assertTrue(fs.exists(new Path("testDir2/test4/file1")));
+    assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file1")));
+  }
+
+  /**
+   * Test that after completing rename for a directory which is enabled for
+   * AtomicRename, the RenamePending JSON file is deleted.
+   */
+  @Test
+  public void testRenamePendingJsonIsRemovedPostSuccessfulRename()
+      throws Exception {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.setWorkingDirectory(new Path("/"));
+    fs.mkdirs(new Path("hbase/test1/test2/test3"));
+    fs.create(new Path("hbase/test1/test2/test3/file"));
+    fs.create(new Path("hbase/test1/test2/test3/file1"));
+    fs.mkdirs(new Path("hbase/test4/"));
+    fs.create(new Path("hbase/test4/file1"));
+    final Integer[] correctDeletePathCount = new Integer[1];
+    correctDeletePathCount[0] = 0;
+    Mockito.doAnswer(answer -> {
+          final String correctDeletePath = "/hbase/test1/test2/test3" + SUFFIX;
+          if (correctDeletePath.equals(
+              ((Path) answer.getArgument(0)).toUri().getPath())) {
+            correctDeletePathCount[0] = 1;
+          }
+          return null;
+        })
+        .when(client)
+        .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    assertTrue(fs.rename(new Path("hbase/test1/test2/test3"),
+        new Path("hbase/test4")));
+    assertEquals("RenamePendingJson should be deleted",
+        1,
+        (int) correctDeletePathCount[0]);
+  }
+
+  /**
+   * Spies on the AzureBlobFileSystem's store and client to enable mocking and verification
+   * of client interactions in tests. It replaces the actual store and client with mocked versions.
+   *
+   * @param fs the AzureBlobFileSystem instance
+   * @return the spied AbfsClient for interaction verification
+   */
+  private AbfsClient addSpyHooksOnClient(final AzureBlobFileSystem fs) {
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    AbfsClient client = Mockito.spy(store.getClient());
+    Mockito.doReturn(client).when(store).getClient();
+    return client;
+  }
+
+  /**
+   * A helper method to set up the test environment and execute the common logic for handling
+   * failed rename operations and recovery in HBase. This method performs the necessary setup
+   * (creating directories and files) and then triggers the `crashRenameAndRecover` method
+   * with a provided recovery action.
+   *
+   * This method is used by different tests that require different recovery actions, such as
+   * performing `listStatus` or checking the existence of a path after a failed rename.
+   *
+   * @param fs the AzureBlobFileSystem instance to be used in the test
+   * @param client the AbfsBlobClient instance to be used in the test
+   * @param srcPath the source path for the rename operation
+   * @param failedCopyPath the path that simulates a failed copy during rename
+   * @param recoveryAction the specific recovery action to be performed after the rename failure
+   *                       (e.g., listing directory status or checking path existence)
+   * @throws Exception if any error occurs during setup or execution of the recovery action
+   */
+  private void setupAndTestHBaseFailedRenameRecovery(
+      final AzureBlobFileSystem fs,
+      final AbfsBlobClient client,
+      final String srcPath,
+      final String failedCopyPath,
+      final FunctionRaisingIOE<AzureBlobFileSystem, Void> recoveryAction)
+      throws Exception {
+    fs.setWorkingDirectory(new Path("/"));
+    fs.mkdirs(new Path(srcPath));
+    fs.mkdirs(new Path(srcPath, "test3"));
+    fs.create(new Path(srcPath + "/test3/file"));
+    fs.create(new Path(failedCopyPath));
+    fs.mkdirs(new Path("hbase/test4/"));
+    fs.create(new Path("hbase/test4/file1"));
+    crashRenameAndRecover(fs, client, srcPath, recoveryAction);
+  }
+
+  /**
+   * Test for a directory in /hbase directory. To simulate the crash of process,
+   * test will throw an exception with 403 on a copy of one of the blob.<br>
+   * ListStatus API will be called on the directory. Expectation is that the ListStatus
+   * API of {@link AzureBlobFileSystem} should recover the paused rename.
+   */
+  @Test
+  public void testHBaseHandlingForFailedRenameWithListRecovery()
+      throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    String srcPath = "hbase/test1/test2";
+    final String failedCopyPath = srcPath + "/test3/file1";
+
+    setupAndTestHBaseFailedRenameRecovery(fs, client, srcPath, failedCopyPath,
+        (abfsFs) -> {
+          abfsFs.listStatus(new Path(srcPath).getParent());
+          return null;
+        });
+  }
+
+  /**
+   * Test for a directory in /hbase directory. To simulate the crash of process,
+   * test will throw an exception with 403 on a copy of one of the blob. The
+   * source directory is a nested directory.<br>
+   * GetFileStatus API will be called on the directory. Expectation is that the
+   * GetFileStatus API of {@link AzureBlobFileSystem} should recover the paused
+   * rename.
+   */
+  @Test
+  public void testHBaseHandlingForFailedRenameWithGetFileStatusRecovery()
+      throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    String srcPath = "hbase/test1/test2";
+    final String failedCopyPath = srcPath + "/test3/file1";
+
+    setupAndTestHBaseFailedRenameRecovery(fs, client, srcPath, failedCopyPath,
+        (abfsFs) -> {
+          abfsFs.exists(new Path(srcPath));
+          return null;
+        });
+  }
+
+
+  /**
+   * Simulates a rename failure, performs a recovery action, and verifies that the "RenamePendingJson"
+   * file is deleted. It checks that the rename operation is successfully completed after recovery.
+   *
+   * @param fs the AzureBlobFileSystem instance
+   * @param client the AbfsBlobClient instance
+   * @param srcPath the source path for the rename operation
+   * @param recoveryCallable the recovery action to perform
+   * @throws Exception if an error occurs during recovery or verification
+   */
+  private void crashRenameAndRecover(final AzureBlobFileSystem fs,
+      AbfsBlobClient client,
+      final String srcPath,
+      final FunctionRaisingIOE<AzureBlobFileSystem, Void> recoveryCallable)
+      throws Exception {
+    crashRename(fs, client, srcPath);
+    AzureBlobFileSystem fs2 = Mockito.spy(getFileSystem());
+    fs2.setWorkingDirectory(new Path(ROOT_PATH));
+    client = (AbfsBlobClient) addSpyHooksOnClient(fs2);
+    int[] renameJsonDeleteCounter = new int[1];
+    Mockito.doAnswer(answer -> {
+          if ((ROOT_PATH + srcPath + SUFFIX)
+              .equalsIgnoreCase(((Path) answer.getArgument(0)).toUri().getPath())) {
+            renameJsonDeleteCounter[0] = 1;
+          }
+          return answer.callRealMethod();
+        })
+        .when(client)
+        .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    recoveryCallable.apply(fs2);
+    Assertions.assertThat(renameJsonDeleteCounter[0])
+        .describedAs("RenamePendingJson should be deleted")
+        .isEqualTo(1);
+    //List would complete the rename orchestration.
+    assertFalse(fs2.exists(new Path("hbase/test1/test2")));
+    assertFalse(fs2.exists(new Path("hbase/test1/test2/test3")));
+    assertTrue(fs2.exists(new Path("hbase/test4/test2/test3")));
+    assertFalse(fs2.exists(new Path("hbase/test1/test2/test3/file")));
+    assertTrue(fs2.exists(new Path("hbase/test4/test2/test3/file")));
+    assertFalse(fs2.exists(new Path("hbase/test1/test2/test3/file1")));
+    assertTrue(fs2.exists(new Path("hbase/test4/test2/test3/file1")));
+  }
+
+  /**
+   * Simulates a rename failure by triggering an `AbfsRestOperationException` during the rename process.
+   * It intercepts the exception and ensures that all leases acquired during the atomic rename are released.
+   *
+   * @param fs the AzureBlobFileSystem instance used for the rename operation
+   * @param client the AbfsBlobClient instance used for mocking the rename failure
+   * @param srcPath the source path for the rename operation
+   * @throws Exception if an error occurs during the simulated failure or lease release
+   */
+  private void crashRename(final AzureBlobFileSystem fs,
+      final AbfsBlobClient client,
+      final String srcPath) throws Exception {
+    BlobRenameHandler[] blobRenameHandlers = new BlobRenameHandler[1];
+    AbfsClientTestUtil.mockGetRenameBlobHandler(client,
+        blobRenameHandler -> {
+          blobRenameHandlers[0] = blobRenameHandler;
+          return null;
+        });
+    //Fail rename orchestration on path hbase/test1/test2/test3/file1
+    Mockito.doThrow(new AbfsRestOperationException(HTTP_FORBIDDEN, "", "",
+            new Exception()))
+        .when(client)
+        .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+            Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    LambdaTestUtils.intercept(AccessDeniedException.class, () -> {
+      fs.rename(new Path(srcPath),
+          new Path("hbase/test4"));
+    });
+    //Release all the leases taken by atomic rename orchestration
+    List<AbfsLease> leases = new ArrayList<>(blobRenameHandlers[0].getLeases());
+    for (AbfsLease lease : leases) {
+      lease.free();
+    }
+  }
+
+  /**
+   * Simulates a scenario where HMaster in Hbase starts up and executes listStatus
+   * API on the directory that has to be renamed by some other executor-machine.
+   * The scenario is that RenamePending JSON is created but before it could be
+   * appended, it has been opened by the HMaster. The HMaster will delete it. The
+   * machine doing rename would have to recreate the JSON file.
+   * ref: <a href="https://issues.apache.org/jira/browse/HADOOP-12678">issue</a>
+   */
+  @Test
+  public void testHbaseListStatusBeforeRenamePendingFileAppendedWithIngressOnBlob()
+      throws Exception {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    testRenamePreRenameFailureResolution(fs);
+    testAtomicityRedoInvalidFile(fs);
+  }
+
+  /**
+   * Tests renaming a directory in AzureBlobFileSystem when the creation of the "RenamePendingJson"
+   * file fails on the first attempt. It ensures the renaming operation is retried.
+   *
+   * The test verifies that the creation of the "RenamePendingJson" file is attempted twice:
+   * once on failure and once on retry.
+   *
+   * @param fs the AzureBlobFileSystem instance for the test
+   * @throws Exception if an error occurs during the test
+   */
+  private void testRenamePreRenameFailureResolution(final AzureBlobFileSystem fs)
+      throws Exception {
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    Path src = new Path("hbase/test1/test2");
+    Path dest = new Path("hbase/test4");
+    fs.mkdirs(src);
+    fs.mkdirs(new Path(src, "test3"));
+    final int[] renamePendingJsonWriteCounter = new int[1];
+    /*
+     * Fail the creation of RenamePendingJson file on the first attempt.
+     */
+    Answer renamePendingJsonCreateAns = createAnswer -> {
+      Path path = createAnswer.getArgument(0);
+      Mockito.doAnswer(clientFlushAns -> {
+            if (renamePendingJsonWriteCounter[0]++ == 0) {
+              fs.delete(path, true);
+            }
+            return clientFlushAns.callRealMethod();
+          })
+          .when(client)
+          .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));
+      return createAnswer.callRealMethod();
+    };
+    RenameAtomicityTestUtils.addCreatePathMock(client,
+        renamePendingJsonCreateAns);
+    fs.rename(src, dest);
+    Assertions.assertThat(renamePendingJsonWriteCounter[0])
+        .describedAs("Creation of RenamePendingJson should be attempted twice")
         .isEqualTo(2);
   }
+
+  /**
+   * Tests the behavior of the redo operation when an invalid "RenamePendingJson" file exists.
+   * It verifies that the file is deleted and that no copy operation is performed.
+   *
+   * The test simulates a scenario where the "RenamePendingJson" file is partially written and
+   * ensures that the `redo` method correctly deletes the file and does not trigger a copy operation.
+   *
+   * @param fs the AzureBlobFileSystem instance for the test
+   * @throws Exception if an error occurs during the test
+   */
+  private void testAtomicityRedoInvalidFile(final AzureBlobFileSystem fs)
+      throws Exception {
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    Path path = new Path("/hbase/test1/test2");
+    fs.mkdirs(new Path(path, "test3"));
+    Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX);
+    OutputStream os = fs.create(renameJson);
+    os.write("{".getBytes(StandardCharsets.UTF_8));
+    os.close();
+    int[] renameJsonDeleteCounter = new int[1];
+    Mockito.doAnswer(deleteAnswer -> {
+          Path ansPath = deleteAnswer.getArgument(0);
+          if (renameJson.toUri()
+              .getPath()
+              .equalsIgnoreCase(ansPath.toUri().getPath())) {
+            renameJsonDeleteCounter[0]++;
+          }
+          return deleteAnswer.callRealMethod();
+        })
+        .when(client)
+        .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    new RenameAtomicity(renameJson, 1,
+        getTestTracingContext(fs, true), null, client).redo();
+    Assertions.assertThat(renameJsonDeleteCounter[0])
+        .describedAs("RenamePendingJson should be deleted")
+        .isEqualTo(1);
+    Mockito.verify(client, Mockito.times(0)).copyBlob(Mockito.any(Path.class),
+        Mockito.any(Path.class), Mockito.nullable(String.class),
+        Mockito.any(TracingContext.class));
+  }
+
+  /**
+   * Test to check the atomicity of rename operation. The rename operation should
+   * be atomic and should not leave any intermediate state.
+   */
+  @Test
+  public void testRenameJsonDeletedBeforeRenameAtomicityCanDelete()
+      throws Exception {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path path = new Path("/hbase/test1/test2");
+    fs.mkdirs(new Path(path, "test3"));
+    Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX);
+    OutputStream os = fs.create(renameJson);
+    os.write("{}".getBytes(StandardCharsets.UTF_8));
+    os.close();
+    int[] renameJsonDeleteCounter = new int[1];
+    Mockito.doAnswer(deleteAnswer -> {
+          Path ansPath = deleteAnswer.getArgument(0);
+          if (renameJson.toUri()
+              .getPath()
+              .equalsIgnoreCase(ansPath.toUri().getPath())) {
+            renameJsonDeleteCounter[0]++;
+          }
+          getFileSystem().delete(ansPath, true);
+          return deleteAnswer.callRealMethod();
+        })
+        .when(client)
+        .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    new RenameAtomicity(renameJson, 2,
+        getTestTracingContext(fs, true), null, client);
+  }
+
+  /**
+   * Tests the scenario where the rename operation is complete before the redo
+   * operation for atomicity, leading to a failure. This test verifies that the
+   * system correctly handles the case when a rename operation is attempted after
+   * the source path has already been deleted, which should result in an error.
+   * <p>
+   * The test simulates a situation where a `renameJson` file is created for the
+   * rename operation, and the source path is deleted during the read process in
+   * the redo operation. The `redoRenameAtomicity` is then executed, and it is
+   * expected to fail with a `404` error, indicating that the source path no longer exists.
+   * <p>
+   * The test ensures that the system can handle this error condition and return
+   * the correct response, preventing a potentially invalid or inconsistent state.
+   *
+   * @throws Exception If an error occurs during file system operations.
+   */
+  @Test
+  public void testRenameCompleteBeforeRenameAtomicityRedo() throws Exception {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path path = new Path("/hbase/test1/test2");
+    fs.mkdirs(new Path(path, "test3"));
+    Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX);
+    /*
+     * Create renameJson file.
+     */
+    AzureBlobFileSystemStore.VersionedFileStatus fileStatus
+        = (AzureBlobFileSystemStore.VersionedFileStatus) fs.getFileStatus(path);
+    int jsonLen = new RenameAtomicity(path,
+        new Path("/hbase/test4"), renameJson,
+        getTestTracingContext(fs, true), fileStatus.getEtag(),
+        client).preRename();
+    RenameAtomicity redoRenameAtomicity = Mockito.spy(
+        new RenameAtomicity(renameJson, jsonLen,
+            getTestTracingContext(fs, true), null, client));
+    RenameAtomicityTestUtils.addReadPathMock(redoRenameAtomicity,
+        readCallbackAnswer -> {
+          byte[] bytes = (byte[]) readCallbackAnswer.callRealMethod();
+          fs.delete(path, true);
+          return bytes;
+        });
+    AbfsRestOperationException ex = intercept(AbfsRestOperationException.class,
+        redoRenameAtomicity::redo);
+    Assertions.assertThat(ex.getStatusCode())
+        .describedAs("RenameAtomicity redo should fail with 404")
+        .isEqualTo(SOURCE_PATH_NOT_FOUND.getStatusCode());
+    Assertions.assertThat(ex.getErrorCode())
+        .describedAs("RenameAtomicity redo should fail with 404")
+        .isEqualTo(SOURCE_PATH_NOT_FOUND);
+  }
+
+  /**
+   * Tests the idempotency of the copyBlob operation during a rename when the
+   * destination already exists. This test simulates a scenario where the source
+   * blob is copied to the destination before the actual rename operation is invoked.
+   * It ensures that the copyBlob operation can handle idempotency issues and perform
+   * the rename successfully even when the destination is pre-created.
+   * <p>
+   * The test verifies that the rename operation successfully copies the blob from
+   * the source to the destination, and the source is deleted, leaving only the
+   * destination file. This ensures that the system behaves correctly in scenarios
+   * where the destination path already contains the blob.
+   *
+   * @throws Exception If an error occurs during file system operations.
+   */
+  @Test
+  public void testCopyBlobIdempotency() throws Exception {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path src = new Path("/srcDir/src");
+    Path dst = new Path("/dst");
+    fs.create(src);
+    Mockito.doAnswer(answer -> {
+      Path srcCopy = answer.getArgument(0);
+      Path dstCopy = answer.getArgument(1);
+      String leaseId = answer.getArgument(2);
+      TracingContext tracingContext = answer.getArgument(3);
+      /*
+       * To fail copyBlob with idempotency issue, making a copy of the source to destination
+       * before the invoked copy
+       */
+      ((AbfsBlobClient) getFileSystem().getAbfsClient()).copyBlob(srcCopy,
+          dstCopy, leaseId, tracingContext);
+      return answer.callRealMethod();
+    }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+        Mockito.nullable(String.class),
+        Mockito.any(TracingContext.class));
+    Assertions.assertThat(fs.rename(src, dst))
+        .describedAs("Rename should be successful and copyBlob should"
+            + "be able to handle idempotency issue")
+        .isTrue();
+    Assertions.assertThat(fs.exists(src))
+        .describedAs("Source should not exist after rename")
+        .isFalse();
+    Assertions.assertThat(fs.exists(dst))
+        .describedAs("Destination should exist after rename")
+        .isTrue();
+  }
+
+  /**
+   * Tests the idempotency of the rename operation when the destination path is
+   * created by some other process before the rename operation. This test simulates
+   * the scenario where a source blob is renamed, and the destination path already
+   * exists due to actions from another process. It ensures that the rename operation
+   * behaves idempotently and correctly handles the case where the destination is
+   * pre-created.
+   * <p>
+   * The test verifies that the rename operation fails (since the destination already
+   * exists), but the source path remains intact, and the blob copy operation is able
+   * to handle the idempotency issue.
+   *
+   * @throws IOException If an error occurs during file system operations.
+   */
+  @Test
+  public void testRenameBlobIdempotencyWhereDstIsCreatedFromSomeOtherProcess()
+      throws IOException {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path src = new Path("/src");
+    Path dst = new Path("/dst");
+    fs.create(src);
+    Mockito.doAnswer(answer -> {
+      Path dstCopy = answer.getArgument(1);
+      fs.create(dstCopy);
+      return answer.callRealMethod();
+    }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+        Mockito.nullable(String.class),
+        Mockito.any(TracingContext.class));
+    Assertions.assertThat(fs.rename(src, dst))
+        .describedAs("Rename should be successful and copyBlob should"
+            + "be able to handle idempotency issue")
+        .isFalse();
+    Assertions.assertThat(fs.exists(src))
+        .describedAs("Source should exist after rename failure")
+        .isTrue();
+  }
+
+  /**
+   * Tests renaming a directory when the destination directory is missing a marker blob.
+   * This test involves creating multiple directories and files, deleting a blob (marker) in the
+   * destination directory, and renaming the source directory to the destination.
+   * It then verifies that the renamed directory exists at the expected destination path.
+   *
+   * @throws Exception If an error occurs during the file system operations or assertions.
+   */
+  @Test
+  public void testRenameDirWhenMarkerBlobIsAbsentOnDstDir() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    fs.mkdirs(new Path("/test1"));
+    fs.mkdirs(new Path("/test1/test2"));
+    fs.mkdirs(new Path("/test1/test2/test3"));
+    fs.create(new Path("/test1/test2/test3/file"));
+    ((AbfsBlobClient) fs.getAbfsClient())
+        .deleteBlobPath(new Path("/test1/test2"),
+            null, getTestTracingContext(fs, true));
+    fs.mkdirs(new Path("/test4/test5"));
+    fs.rename(new Path("/test4"), new Path("/test1/test2"));
+    assertTrue(fs.exists(new Path("/test1/test2/test4/test5")));
+  }
+
+  /**
+   * Tests the renaming of a directory when the source directory does not have a marker file.
+   * This test creates a file within a source directory, deletes the source directory from the blob storage,
+   * creates a new target directory, and renames the source directory to the target location.
+   * It verifies that the renamed source directory exists in the target path.
+   *
+   * @throws Exception If an error occurs during the file system operations or assertions.
+   */
+  @Test
+  public void testBlobRenameSrcDirHasNoMarker() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    fs.create(new Path("/test1/test2/file1"));
+    ((AbfsBlobClient) fs.getAbfsStore().getClient())
+        .deleteBlobPath(new Path("/test1"), null,
+            getTestTracingContext(fs, true));
+    fs.mkdirs(new Path("/test2"));
+    fs.rename(new Path("/test1"), new Path("/test2"));
+    assertTrue(fs.exists(new Path("/test2/test1")));
+  }
+
+  /**
+   * Mocks the progress status for a copy blob operation.
+   * This method simulates a copy operation that is pending and not yet completed.
+   * It intercepts the `copyBlob` method and modifies its response to return a "COPY_STATUS_PENDING"
+   * status for the copy operation.
+   *
+   * @param spiedClient The {@link AbfsBlobClient} instance that is being spied on.
+   * @throws AzureBlobFileSystemException if the mock setup fails.
+   */
+  private void addMockForProgressStatusOnCopyOperation(final AbfsBlobClient spiedClient)
+      throws AzureBlobFileSystemException {
+    Mockito.doAnswer(answer -> {
+          AbfsRestOperation op = Mockito.spy(
+              (AbfsRestOperation) answer.callRealMethod());
+          AbfsHttpOperation httpOp = Mockito.spy(op.getResult());
+          Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader(
+              HttpHeaderConfigurations.X_MS_COPY_STATUS);
+          Mockito.doReturn(httpOp).when(op).getResult();
+          return op;
+        })
+        .when(spiedClient)
+        .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+            Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+  }
+
+  /**
+   * Verifies the behavior of a blob copy operation that takes time to complete.
+   * The test ensures the following:
+   * <ul>
+   *   <li>A file is created and a rename operation is initiated.</li>
+   *   <li>The copy operation progress is mocked to simulate a time-consuming process.</li>
+   *   <li>The rename operation triggers a call to handle the copy progress.</li>
+   *   <li>The test checks that the file exists after the rename and that the
+   *       `handleCopyInProgress` method is called exactly once.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testCopyBlobTakeTime() throws Exception {
+    AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient(
+        fileSystem);
+    addMockForProgressStatusOnCopyOperation(spiedClient);
+    fileSystem.create(new Path("/test1/file"));
+    BlobRenameHandler[] blobRenameHandlers = new BlobRenameHandler[1];
+    AbfsClientTestUtil.mockGetRenameBlobHandler(spiedClient,
+        blobRenameHandler -> {
+          blobRenameHandlers[0] = blobRenameHandler;
+          return null;
+        });
+    fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2"));
+    assertTrue(fileSystem.exists(new Path("/test1/file2")));
+    Mockito.verify(blobRenameHandlers[0], Mockito.times(1))
+        .handleCopyInProgress(Mockito.any(Path.class),
+            Mockito.any(TracingContext.class), Mockito.any(String.class));
+  }
+
+  /**
+   * Mocks the final status of a blob copy operation.
+   * This method ensures that when checking the status of a copy operation in progress,
+   * it returns the specified final status (e.g., success, failure, aborted).
+   *
+   * @param spiedClient The mocked Azure Blob client to apply the mock behavior.
+   * @param requiredCopyFinalStatus The final status of the copy operation to be returned
+   *                                (e.g., COPY_STATUS_FAILED, COPY_STATUS_ABORTED).
+   */
+  private void addMockForCopyOperationFinalStatus(final AbfsBlobClient spiedClient,
+      final String requiredCopyFinalStatus) {
+    AbfsClientTestUtil.mockGetRenameBlobHandler(spiedClient,
+        blobRenameHandler -> {
+          Mockito.doAnswer(onHandleCopyInProgress -> {
+                Path handlePath = onHandleCopyInProgress.getArgument(0);
+                TracingContext tracingContext = onHandleCopyInProgress.getArgument(
+                    1);
+                Mockito.doAnswer(onStatusCheck -> {
+                      AbfsRestOperation op = Mockito.spy(
+                          (AbfsRestOperation) onStatusCheck.callRealMethod());
+                      AbfsHttpOperation httpOp = Mockito.spy(op.getResult());
+                      Mockito.doReturn(requiredCopyFinalStatus)
+                          .when(httpOp)
+                          .getResponseHeader(
+                              HttpHeaderConfigurations.X_MS_COPY_STATUS);
+                      Mockito.doReturn(httpOp).when(op).getResult();
+                      return op;
+                    })
+                    .when(spiedClient)
+                    .getPathStatus(handlePath.toUri().getPath(),
+                        tracingContext, null, false);
+                return onHandleCopyInProgress.callRealMethod();
+              })
+              .when(blobRenameHandler)
+              .handleCopyInProgress(Mockito.any(Path.class),
+                  Mockito.any(TracingContext.class), Mockito.any(String.class));
+          return null;
+        });
+  }
+
+  /**
+   * Verifies the behavior when a blob copy operation takes time and eventually fails.
+   * The test ensures the following:
+   * <ul>
+   *   <li>A file is created and a copy operation is initiated.</li>
+   *   <li>The copy operation is mocked to eventually fail.</li>
+   *   <li>The rename operation triggers an exception due to the failed copy.</li>
+   *   <li>The test checks that the appropriate 'COPY_FAILED' error code and status code are returned.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testCopyBlobTakeTimeAndEventuallyFail() throws Exception {
+    AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient(
+        fileSystem);
+    addMockForProgressStatusOnCopyOperation(spiedClient);
+    fileSystem.create(new Path("/test1/file"));
+    addMockForCopyOperationFinalStatus(spiedClient, COPY_STATUS_FAILED);
+    AbfsRestOperationException ex = intercept(AbfsRestOperationException.class,
+        () -> {
+          fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2"));
+        });
+    Assertions.assertThat(ex.getStatusCode())
+        .describedAs("Expecting COPY_FAILED status code")
+        .isEqualTo(COPY_BLOB_FAILED.getStatusCode());
+    Assertions.assertThat(ex.getErrorCode())
+        .describedAs("Expecting COPY_FAILED error code")
+        .isEqualTo(COPY_BLOB_FAILED);
+  }
+
+  /**
+   * Verifies the behavior when a blob copy operation takes time and is eventually aborted.
+   * The test ensures the following:
+   * <ul>
+   *   <li>A file is created and a copy operation is initiated.</li>
+   *   <li>The copy operation is mocked to eventually be aborted.</li>
+   *   <li>The rename operation triggers an exception due to the aborted copy.</li>
+   *   <li>The test checks that the appropriate 'COPY_ABORTED' error code and status code are returned.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testCopyBlobTakeTimeAndEventuallyAborted() throws Exception {
+    AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient(
+        fileSystem);
+    addMockForProgressStatusOnCopyOperation(spiedClient);
+    fileSystem.create(new Path("/test1/file"));
+    addMockForCopyOperationFinalStatus(spiedClient, COPY_STATUS_ABORTED);
+    AbfsRestOperationException ex = intercept(AbfsRestOperationException.class,
+        () -> {
+          fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2"));
+        });
+    Assertions.assertThat(ex.getStatusCode())
+        .describedAs("Expecting COPY_ABORTED status code")
+        .isEqualTo(COPY_BLOB_ABORTED.getStatusCode());
+    Assertions.assertThat(ex.getErrorCode())
+        .describedAs("Expecting COPY_ABORTED error code")
+        .isEqualTo(COPY_BLOB_ABORTED);
+  }
+
+  /**
+   * Verifies the behavior when a blob copy operation takes time and the destination blob
+   * is deleted during the process. The test ensures the following:
+   * <ul>
+   *   <li>A source file is created and a copy operation is initiated.</li>
+   *   <li>During the copy process, the destination file is deleted.</li>
+   *   <li>The copy operation returns a pending status.</li>
+   *   <li>The test checks that the destination file does not exist after the copy operation is interrupted.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testCopyBlobTakeTimeAndBlobIsDeleted() throws Exception {
+    AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient(
+        fileSystem);
+    String srcFile = "/test1/file";
+    String dstFile = "/test1/file2";
+    Mockito.doAnswer(answer -> {
+          AbfsRestOperation op = Mockito.spy(
+              (AbfsRestOperation) answer.callRealMethod());
+          fileSystem.delete(new Path(dstFile), false);
+          AbfsHttpOperation httpOp = Mockito.spy(op.getResult());
+          Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader(
+              HttpHeaderConfigurations.X_MS_COPY_STATUS);
+          Mockito.doReturn(httpOp).when(op).getResult();
+          return op;
+        })
+        .when(spiedClient)
+        .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+            Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+    fileSystem.create(new Path(srcFile));
+    assertFalse(fileSystem.rename(new Path(srcFile), new Path(dstFile)));
+    assertFalse(fileSystem.exists(new Path(dstFile)));
+  }
+
+  /**
+   * Verifies the behavior when attempting to copy a blob after the source has been deleted
+   * in the Azure Blob FileSystem. The test ensures the following:
+   * <ul>
+   *   <li>A source blob is created and then deleted.</li>
+   *   <li>An attempt to copy the deleted source blob results in a 'not found' error.</li>
+   *   <li>The test checks that the correct HTTP error (404 Not Found) is returned when copying a non-existent source.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testCopyAfterSourceHasBeenDeleted() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient();
+    fs.create(new Path("/src"));
+    TracingContext tracingContext = new TracingContext("clientCorrelationId",
+        "fileSystemId", FSOperationType.TEST_OP,
+        getConfiguration().getTracingHeaderFormat(),
+        null);
+    client.deleteBlobPath(new Path("/src"), null,
+        getTestTracingContext(fs, true));
+    Boolean srcBlobNotFoundExReceived = false;
+    AbfsRestOperationException ex = intercept(AbfsRestOperationException.class,
+        () -> {
+          client.copyBlob(new Path("/src"), new Path("/dst"),
+              null, getTestTracingContext(fs, true));
+        });
+    Assertions.assertThat(ex.getStatusCode())
+        .describedAs("Source has to be not found at copy")
+        .isEqualTo(HTTP_NOT_FOUND);
+  }
+
+  /**
+   * Verifies that parallel rename operations in the Azure Blob FileSystem fail when
+   * trying to perform an atomic rename with lease acquisition. The test ensures the following:
+   * <ul>
+   *   <li>A directory is created and a rename operation is attempted.</li>
+   *   <li>A parallel thread attempts to rename the directory while the lease is being acquired.</li>
+   *   <li>The parallel rename operation should fail due to a lease conflict, triggering an exception.</li>
+   *   <li>The test verifies that the expected conflict exception is thrown when attempting a parallel rename.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testParallelRenameForAtomicRenameShouldFail() throws Exception {
+    Configuration config = getRawConfiguration();
+    config.set(FS_AZURE_LEASE_THREADS, "2");
+    AzureBlobFileSystem fs = Mockito.spy(
+        (AzureBlobFileSystem) FileSystem.newInstance(config));
+    assumeBlobServiceType();
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    Path src = new Path("/hbase/src");
+    Path dst = new Path("/hbase/dst");
+    fs.mkdirs(src);
+    AtomicBoolean leaseAcquired = new AtomicBoolean(false);
+    AtomicBoolean exceptionOnParallelRename = new AtomicBoolean(false);
+    AtomicBoolean parallelThreadDone = new AtomicBoolean(false);
+    Mockito.doAnswer(answer -> {
+          AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod();
+          leaseAcquired.set(true);
+          while (!parallelThreadDone.get()) {}
+          return op;
+        })
+        .when(client)
+        .acquireLease(Mockito.anyString(), Mockito.anyInt(),
+            Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    new Thread(() -> {
+      while (!leaseAcquired.get()) {}
+      try {
+        fs.rename(src, dst);
+      } catch (Exception e) {
+        if (e.getCause() instanceof AbfsLease.LeaseException
+            && e.getCause().getCause() instanceof AbfsRestOperationException
+            && ((AbfsRestOperationException) e.getCause()
+            .getCause()).getStatusCode() == HTTP_CONFLICT) {
+          exceptionOnParallelRename.set(true);
+        }
+      } finally {
+        parallelThreadDone.set(true);
+      }
+    }).start();
+    fs.rename(src, dst);
+    while (!parallelThreadDone.get()) {}
+    Assertions.assertThat(exceptionOnParallelRename.get())
+        .describedAs("Parallel rename should fail")
+        .isTrue();
+  }
+
+  /**
+   * Verifies the behavior of appending data to a blob during a rename operation in the
+   * Azure Blob FileSystem. The test ensures the following:
+   * <ul>
+   *   <li>A file is created and data is appended to it while a rename operation is in progress.</li>
+   *   <li>The append operation should fail due to the rename operation in progress.</li>
+   *   <li>The test checks that the append operation is properly interrupted and fails as expected.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testAppendAtomicBlobDuringRename() throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    Path src = new Path("/hbase/src");
+    Path dst = new Path("/hbase/dst");
+    FSDataOutputStream os = fs.create(src);
+    AtomicBoolean copyInProgress = new AtomicBoolean(false);
+    AtomicBoolean outputStreamClosed = new AtomicBoolean(false);
+    AtomicBoolean appendFailed = new AtomicBoolean(false);
+    Mockito.doAnswer(answer -> {
+      copyInProgress.set(true);
+      while (!outputStreamClosed.get()) {}
+      return answer.callRealMethod();
+    }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+        Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+    new Thread(() -> {
+      while (!copyInProgress.get()) {}
+      try {
+        os.write(1);
+        os.close();
+      } catch (IOException e) {
+        appendFailed.set(true);
+      } finally {
+        outputStreamClosed.set(true);
+      }
+    }).start();
+    fs.rename(src, dst);
+    Assertions.assertThat(appendFailed.get())
+        .describedAs("Append should fail")
+        .isTrue();
+  }
+
+  /**
+   * Verifies the behavior of renaming a directory in the Azure Blob FileSystem when
+   * there is a neighboring directory with the same prefix. The test ensures the following:
+   * <ul>
+   *   <li>Two directories with similar prefixes are created, along with files inside them.</li>
+   *   <li>The rename operation moves one directory to a new location.</li>
+   *   <li>Files in the renamed directory are moved, while files in the neighboring directory with the same prefix remain unaffected.</li>
+   *   <li>Correct existence checks are performed to confirm the renamed directory and its files are moved, and the original directory is deleted.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testBlobRenameOfDirectoryHavingNeighborWithSamePrefix()
+      throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeBlobServiceType();
+    fs.mkdirs(new Path("/testDir/dir"));
+    fs.mkdirs(new Path("/testDir/dirSamePrefix"));
+    fs.create(new Path("/testDir/dir/file1"));
+    fs.create(new Path("/testDir/dir/file2"));
+    fs.create(new Path("/testDir/dirSamePrefix/file1"));
+    fs.create(new Path("/testDir/dirSamePrefix/file2"));
+    fs.rename(new Path("/testDir/dir"), new Path("/testDir/dir2"));
+    Assertions.assertThat(fs.exists(new Path("/testDir/dirSamePrefix/file1")))
+        .isTrue();
+    Assertions.assertThat(fs.exists(new Path("/testDir/dir/file1")))
+        .isFalse();
+    Assertions.assertThat(fs.exists(new Path("/testDir/dir/file2")))
+        .isFalse();
+    Assertions.assertThat(fs.exists(new Path("/testDir/dir/")))
+        .isFalse();
+  }
+
+  /**
+   * Verifies the behavior of renaming a directory in the Azure Blob FileSystem when
+   * the `listPath` operation returns paginated results with one object per list.
+   * The test ensures the following:
+   * <ul>
+   *   <li>A directory and its files are created.</li>
+   *   <li>The `listPath` operation is mocked to return one file at a time in each paginated result.</li>
+   *   <li>The rename operation successfully moves the directory and its files to a new location.</li>
+   *   <li>All files are verified to exist in the new location after the rename.</li>
+   * </ul>
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testBlobRenameWithListGivingPaginatedResultWithOneObjectPerList()
+      throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.mkdirs(new Path("/testDir/dir1"));
+    for (int i = 0; i < 10; i++) {
+      fs.create(new Path("/testDir/dir1/file" + i));
+    }
+    Mockito.doAnswer(answer -> {
+          String path = answer.getArgument(0);
+          boolean recursive = answer.getArgument(1);
+          String continuation = answer.getArgument(3);
+          TracingContext context = answer.getArgument(4);
+          return getFileSystem().getAbfsClient()
+              .listPath(path, recursive, 1, continuation, context);
+        })
+        .when(spiedClient)
+        .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(),
+            Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    fs.rename(new Path("/testDir/dir1"), new Path("/testDir/dir2"));
+    for (int i = 0; i < 10; i++) {
+      Assertions.assertThat(fs.exists(new Path("/testDir/dir2/file" + i)))
+          .describedAs("File " + i + " should exist in /testDir/dir2")
+          .isTrue();
+    }
+  }
+
+  /**
+   * Verifies that the producer stops on a rename failure due to an access denial
+   * (HTTP_FORBIDDEN error) in the Azure Blob FileSystem. The test ensures the following:
+   * <ul>
+   *   <li>Multiple file creation tasks are submitted concurrently.</li>
+   *   <li>The rename operation is attempted but fails with an access denied exception.</li>
+   *   <li>On failure, the list operation for the source directory is invoked at most twice.</li>
+   * </ul>
+   * The test simulates a failure scenario where the rename operation encounters an access
+   * denied error, and the list operation should stop after the failure.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testProducerStopOnRenameFailure() throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    fs.mkdirs(new Path("/src"));
+    ExecutorService executorService = Executors.newFixedThreadPool(10);
+    List<Future> futureList = new ArrayList<>();
+    for (int i = 0; i < MAX_ITERATIONS; i++) {
+      int iter = i;
+      Future future = executorService.submit(() -> {
+        try {
+          fs.create(new Path("/src/file" + iter));
+        } catch (IOException ex) {}
+      });
+      futureList.add(future);
+    }
+    for (Future future : futureList) {
+      future.get();
+    }
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient();
+    AbfsBlobClient spiedClient = Mockito.spy(client);
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    store.setClient(spiedClient);
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    final int[] copyCallInvocation = new int[1];
+    Mockito.doAnswer(answer -> {
+          throw new AbfsRestOperationException(HTTP_FORBIDDEN, "", "",
+              new Exception());
+        }).when(spiedClient)
+        .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+            Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+    AbfsClientTestUtil.mockGetRenameBlobHandler(spiedClient,
+        (blobRenameHandler) -> {
+          Mockito.doAnswer(answer -> {
+                try {
+                  answer.callRealMethod();
+                } catch (AbfsRestOperationException ex) {
+                  if (ex.getStatusCode() == HTTP_FORBIDDEN) {
+                    copyCallInvocation[0]++;
+                  }
+                  throw ex;
+                }
+                throw new AssertionError("List Consumption should have failed");
+              })
+              .when(blobRenameHandler).listRecursiveAndTakeAction();
+          return null;
+        });
+    final int[] listCallInvocation = new int[1];
+    Mockito.doAnswer(answer -> {
+          if (answer.getArgument(0).equals("/src")) {
+            if (listCallInvocation[0] == 1) {
+              while (copyCallInvocation[0] == 0) {}
+            }
+            listCallInvocation[0]++;
+            return getFileSystem().getAbfsClient().listPath(answer.getArgument(0),
+                answer.getArgument(1), 1,
+                answer.getArgument(3), answer.getArgument(4));
+          }
+          return answer.callRealMethod();
+        })
+        .when(spiedClient)
+        .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(),
+            Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+    intercept(AccessDeniedException.class,
+        () -> {
+          fs.rename(new Path("/src"), new Path("/dst"));
+        });
+    Assertions.assertThat(listCallInvocation[0])
+        .describedAs("List on src should have been invoked at-most twice."
+            + "One before consumption and the other after consumption has starting."
+            + "Once consumption fails, listing would be stopped.")
+        .isLessThanOrEqualTo(2);
+  }
+
+  /**
+   * Verifies the behavior of renaming a directory through the Azure Blob FileSystem
+   * when the source directory is deleted just before the rename operation is resumed.
+   * It ensures that:
+   * <ul>
+   *   <li>No blobs are copied during the resume operation.</li>
+   * </ul>
+   * The test simulates a crash, deletes the source directory, and checks for the expected result.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testRenameResumeThroughListStatusWithSrcDirDeletedJustBeforeResume()
+      throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path srcPath = new Path("hbase/test1/");
+    Path failurePath = new Path(srcPath, "file");
+    fs.mkdirs(srcPath);
+    fs.create(failurePath);
+    crashRename(fs, client, srcPath.toUri().getPath());
+    fs.delete(srcPath, true);
+    AtomicInteger copiedBlobs = new AtomicInteger(0);
+    Mockito.doAnswer(answer -> {
+      copiedBlobs.incrementAndGet();
+      return answer.callRealMethod();
+    }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+        Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+    fs.listStatus(new Path("hbase"));
+    Assertions.assertThat(copiedBlobs.get())
+        .describedAs("No Copy on resume")
+        .isEqualTo(0);
+  }
+
+  /**
+   * Verifies the behavior of renaming a directory through the Azure Blob FileSystem
+   * when the source directory's ETag changes just before the rename operation is resumed.
+   * It ensures that:
+   * <ul>
+   *   <li>No blobs are copied during the resume operation.</li>
+   *   <li>The pending rename JSON file is deleted.</li>
+   * </ul>
+   * The test simulates a crash, retries the operation, and checks for the expected results.
+   *
+   * @throws Exception if an error occurs during the test execution
+   */
+  @Test
+  public void testRenameResumeThroughListStatusWithSrcDirETagChangedJustBeforeResume()
+      throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path srcPath = new Path("hbase/test1/");
+    Path failurePath = new Path(srcPath, "file");
+    fs.mkdirs(srcPath);
+    fs.create(failurePath);
+    crashRename(fs, client, srcPath.toUri().getPath()
+    );
+    fs.delete(srcPath, true);
+    fs.mkdirs(srcPath);
+    AtomicInteger copiedBlobs = new AtomicInteger(0);
+    Mockito.doAnswer(answer -> {
+      copiedBlobs.incrementAndGet();
+      return answer.callRealMethod();
+    }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+        Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+    AtomicInteger pendingJsonDeleted = new AtomicInteger(0);
+    Mockito.doAnswer(listAnswer -> {
+          Path path = listAnswer.getArgument(0);
+          if (path.toUri().getPath().endsWith(SUFFIX)) {
+            pendingJsonDeleted.incrementAndGet();
+          }
+          return listAnswer.callRealMethod();
+        })
+        .when(client)
+        .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    fs.listStatus(new Path("/hbase"));
+    Assertions.assertThat(copiedBlobs.get())
+        .describedAs("No Copy on resume")
+        .isEqualTo(0);
+    Assertions.assertThat(pendingJsonDeleted.get())
+        .describedAs("RenamePendingJson should be deleted")
+        .isEqualTo(1);
+  }
+
+  /**
+   * Test case to verify the behavior of renaming a directory through the Azure Blob
+   * FileSystem when the source directory's ETag changes just before the rename operation
+   * is resumed. This test specifically checks the following:
+   *
+   * @throws Exception if any errors occur during the test execution
+   */
+  @Test
+  public void testRenameResumeThroughGetStatusWithSrcDirETagChangedJustBeforeResume()
+      throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    Path srcPath = new Path("hbase/test1/");
+    Path failurePath = new Path(srcPath, "file");
+    fs.mkdirs(srcPath);
+    fs.create(failurePath);
+    crashRename(fs, client, srcPath.toUri().getPath()
+    );
+    fs.delete(srcPath, true);
+    fs.mkdirs(srcPath);
+    AtomicInteger copiedBlobs = new AtomicInteger(0);
+    Mockito.doAnswer(answer -> {
+      copiedBlobs.incrementAndGet();
+      return answer.callRealMethod();
+    }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+        Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+    AtomicInteger pendingJsonDeleted = new AtomicInteger(0);
+    Mockito.doAnswer(listAnswer -> {
+          Path path = listAnswer.getArgument(0);
+          if (path.toUri().getPath().endsWith(SUFFIX)) {
+            pendingJsonDeleted.incrementAndGet();
+          }
+          return listAnswer.callRealMethod();
+        })
+        .when(client)
+        .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    Assertions.assertThat(fs.exists(srcPath))
+        .describedAs("Source should exist")
+        .isTrue();
+    Assertions.assertThat(copiedBlobs.get())
+        .describedAs("No Copy on resume")
+        .isEqualTo(0);
+    Assertions.assertThat(pendingJsonDeleted.get())
+        .describedAs("RenamePendingJson should be deleted")
+        .isEqualTo(1);
+  }
+
+  /**
+   * Test to assert that the CID in src marker blob copy and delete contains the
+   * total number of blobs operated in the rename directory.
+   * Also, to assert that all operations in the rename-directory flow have same
+   * primaryId and opType.
+   */
+  @Test
+  public void testRenameSrcDirDeleteEmitDeletionCountInClientRequestId()
+      throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+    assumeBlobServiceType();
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    String dirPathStr = "/testDir/dir1";
+    fs.mkdirs(new Path(dirPathStr));
+    ExecutorService executorService = Executors.newFixedThreadPool(5);
+    List<Future> futures = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      final int iter = i;
+      Future future = executorService.submit(() ->
+          fs.create(new Path("/testDir/dir1/file" + iter)));
+      futures.add(future);
+    }
+    for (Future future : futures) {
+      future.get();
+    }
+    executorService.shutdown();
+    final TracingHeaderValidator tracingHeaderValidator
+        = new TracingHeaderValidator(
+        fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
+        fs.getFileSystemId(), FSOperationType.RENAME, true, 0);
+    fs.registerListener(tracingHeaderValidator);
+    Mockito.doAnswer(copyAnswer -> {
+          if (dirPathStr.equalsIgnoreCase(
+              ((Path) copyAnswer.getArgument(0)).toUri().getPath())) {
+            tracingHeaderValidator.setOperatedBlobCount(BLOB_COUNT);
+            return copyAnswer.callRealMethod();
+          }
+          return copyAnswer.callRealMethod();
+        })
+        .when(client)
+        .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+            Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    Mockito.doAnswer(deleteAnswer -> {
+          if (dirPathStr.equalsIgnoreCase(
+              ((Path) deleteAnswer.getArgument(0)).toUri().getPath())) {
+            Object result = deleteAnswer.callRealMethod();
+            tracingHeaderValidator.setOperatedBlobCount(null);
+            return result;
+          }
+          return deleteAnswer.callRealMethod();
+        })
+        .when(client)
+        .deleteBlobPath(Mockito.any(Path.class),
+            Mockito.nullable(String.class),
+            Mockito.any(TracingContext.class));
+    fs.rename(new Path(dirPathStr), new Path("/dst/"));
+  }
 }

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

@@ -24,15 +24,21 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
 
+import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Parameterized test of rename operations of unicode paths.
@@ -84,14 +90,27 @@ public class ITestAzureBlobFileSystemRenameUnicode extends
     assertIsFile(fs, filePath);
 
     Path folderPath2 = new Path(destDir);
+    if (getAbfsServiceType() == AbfsServiceType.BLOB
+        && destDir.contains(COLON)) {
+      AbfsRestOperationException ex = intercept(
+          AbfsRestOperationException.class, () -> {
+            fs.rename(folderPath1, folderPath2);
+            return null;
+          });
+      assertTrue(ex.getCause() instanceof PathIOException);
+      assertEquals(HTTP_BAD_REQUEST, ex.getStatusCode());
+      return;
+    }
     assertRenameOutcome(fs, folderPath1, folderPath2, true);
     assertPathDoesNotExist(fs, "renamed", folderPath1);
     assertIsDirectory(fs, folderPath2);
-    assertPathExists(fs, "renamed file", new Path(folderPath2 + "/" + filename));
+    assertPathExists(fs, "renamed file",
+        new Path(folderPath2 + "/" + filename));
 
     FileStatus[] fileStatus = fs.listStatus(folderPath2);
     assertNotNull(fileStatus);
-    assertTrue("Empty listing returned from listStatus(\"" + folderPath2 + "\")",
+    assertTrue(
+        "Empty listing returned from listStatus(\"" + folderPath2 + "\")",
         fileStatus.length > 0);
     assertEquals(fileStatus[0].getPath().getName(), filename);
   }

+ 6 - 9
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java

@@ -17,12 +17,13 @@
  */
 package org.apache.hadoop.fs.azurebfs;
 
+import java.io.IOException;
+
 import org.junit.Assume;
 import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
-import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
@@ -40,23 +41,19 @@ public class ITestSharedKeyAuth extends AbstractAbfsIntegrationTest {
     Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey);
     Configuration config = this.getRawConfiguration();
     config.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
-        false);
+        true);
     String accountName = this.getAccountName();
     String configkKey = FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "." + accountName;
     // a wrong sharedKey
     String secret = "XjUjsGherkDpljuyThd7RpljhR6uhsFjhlxRpmhgD12lnj7lhfRn8kgPt5"
         + "+MJHS7UJNDER+jn6KP6Jnm2ONQlm==";
     config.set(configkKey, secret);
-
-    AbfsClient abfsClient = this.getFileSystem(config).getAbfsClient();
-    intercept(AbfsRestOperationException.class,
+    intercept(IOException.class,
         "\"Server failed to authenticate the request. Make sure the value of "
             + "Authorization header is formed correctly including the "
             + "signature.\", 403",
         () -> {
-          abfsClient
-              .getAclStatus("/", getTestTracingContext(getFileSystem(), false));
+          FileSystem.newInstance(config);
         });
   }
-
 }

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

@@ -288,4 +288,53 @@ public final class AbfsClientTestUtil {
             Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(),
             Mockito.nullable(String.class));
   }
+
+  /**
+   * Mocks the `getBlobDeleteHandler` method of `AbfsBlobClient` to apply a custom handler
+   * for the delete operation. This allows for controlling the behavior of the delete
+   * process during testing.
+   *
+   * @param blobClient the `AbfsBlobClient` instance to mock
+   * @param functionRaisingIOE the function to apply to the mocked `BlobDeleteHandler`
+   */
+  public static void mockGetDeleteBlobHandler(AbfsBlobClient blobClient,
+                                              FunctionRaisingIOE<BlobDeleteHandler, Void> functionRaisingIOE) {
+    Mockito.doAnswer(answer -> {
+              BlobDeleteHandler blobDeleteHandler = Mockito.spy(
+                      (BlobDeleteHandler) answer.callRealMethod());
+              Mockito.doAnswer(answer1 -> {
+                functionRaisingIOE.apply(blobDeleteHandler);
+                return answer1.callRealMethod();
+              }).when(blobDeleteHandler).execute();
+              return blobDeleteHandler;
+            })
+            .when(blobClient)
+            .getBlobDeleteHandler(Mockito.anyString(), Mockito.anyBoolean(),
+                    Mockito.any(TracingContext.class));
+  }
+
+  /**
+   * Mocks the `getBlobRenameHandler` method of `AbfsBlobClient` to apply a custom handler
+   * for the rename operation. This allows for controlling the behavior of the rename
+   * process during testing.
+   *
+   * @param blobClient the `AbfsBlobClient` instance to mock
+   * @param functionRaisingIOE the function to apply to the mocked `BlobRenameHandler`
+   */
+  public static void mockGetRenameBlobHandler(AbfsBlobClient blobClient,
+                                              FunctionRaisingIOE<BlobRenameHandler, Void> functionRaisingIOE) {
+    Mockito.doAnswer(answer -> {
+              BlobRenameHandler blobRenameHandler = Mockito.spy(
+                      (BlobRenameHandler) answer.callRealMethod());
+              Mockito.doAnswer(answer1 -> {
+                functionRaisingIOE.apply(blobRenameHandler);
+                return answer1.callRealMethod();
+              }).when(blobRenameHandler).execute();
+              return blobRenameHandler;
+            })
+            .when(blobClient)
+            .getBlobRenameHandler(Mockito.anyString(), Mockito.anyString(),
+                    Mockito.nullable(String.class), Mockito.anyBoolean(),
+                    Mockito.any(TracingContext.class));
+  }
 }

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

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

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

@@ -206,7 +206,7 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
     Mockito.doReturn(isPaginatedDeleteEnabled).when(spiedClient).getIsPaginatedDeleteEnabled();
 
     AbfsRestOperation op = spiedClient.deletePath(
-        testPath.toString(), true, null, testTC, isHnsEnabled);
+        testPath.toString(), true, null, testTC);
 
     // Getting the xMsVersion that was used to make the request
     String xMsVersionUsed = getHeaderValue(op.getRequestHeaders(), X_MS_VERSION);
@@ -253,7 +253,7 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
     Mockito.doReturn(isPaginatedDeleteEnabled).when(spiedClient).getIsPaginatedDeleteEnabled();
 
     AbfsRestOperation op = spiedClient.deletePath(
-        testPath.toString(), false, null, testTC, isHnsEnabled);
+        testPath.toString(), false, null, testTC);
 
     // Getting the url that was used to make the request
     String urlUsed = op.getUrl().toString();
@@ -280,7 +280,7 @@ public class ITestAbfsPaginatedDelete extends AbstractAbfsIntegrationTest {
     Mockito.doReturn(isPaginatedEnabled).when(spiedClient).getIsPaginatedDeleteEnabled();
 
     AbfsRestOperationException e = intercept(AbfsRestOperationException.class, () ->
-        spiedClient.deletePath(testPath.toString(), true, randomCT, testTC, isHnsEnabled));
+        spiedClient.deletePath(testPath.toString(), true, randomCT, testTC));
     assertStatusCode(e, HTTP_BAD_REQUEST);
   }
 

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

@@ -69,7 +69,8 @@ public class ITestApacheClientConnectionPool extends
     configuration.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.name());
     try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
         configuration)) {
-      KeepAliveCache kac = fs.getAbfsStore().getClient().getKeepAliveCache();
+      KeepAliveCache kac = fs.getAbfsStore().getClientHandler().getIngressClient()
+          .getKeepAliveCache();
       kac.close();
       AbfsDriverException ex = intercept(AbfsDriverException.class,
           KEEP_ALIVE_CACHE_CLOSED, () -> {

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

@@ -0,0 +1,79 @@
+/**
+ * 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 org.mockito.Mockito;
+import org.mockito.stubbing.Answer;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+public final class RenameAtomicityTestUtils {
+
+  private RenameAtomicityTestUtils() {
+  }
+
+  /**
+   * Creates a spied object of {@link BlobRenameHandler} and {@link RenameAtomicity}
+   * and adds mocked behavior to {@link RenameAtomicity#createRenamePendingJson(Path, byte[])}.
+   *
+   * @param client client that would supply BlobRenameHandler and RenameAtomicity.
+   * @param answer mocked behavior for {@link RenameAtomicity#createRenamePendingJson(Path, byte[])}.
+   */
+  public static void addCreatePathMock(AbfsBlobClient client, Answer answer) {
+    Mockito.doAnswer(clientHandlerAns -> {
+          BlobRenameHandler renameHandler = Mockito.spy(
+              (BlobRenameHandler) clientHandlerAns.callRealMethod());
+          Mockito.doAnswer(getRenameAtomicityAns -> {
+                RenameAtomicity renameAtomicity = Mockito.spy(
+                    (RenameAtomicity) getRenameAtomicityAns.callRealMethod());
+                Mockito.doAnswer(answer)
+                    .when(renameAtomicity)
+                    .createRenamePendingJson(Mockito.any(
+                        Path.class), Mockito.any(byte[].class));
+                return renameAtomicity;
+              })
+              .when(renameHandler)
+              .getRenameAtomicity(Mockito.any(PathInformation.class));
+          return renameHandler;
+        })
+        .when(client)
+        .getBlobRenameHandler(Mockito.anyString(), Mockito.anyString(),
+            Mockito.nullable(String.class), Mockito.anyBoolean(), Mockito.any(
+                TracingContext.class));
+  }
+
+
+  /**
+   * Adds mocked behavior to {@link RenameAtomicity#readRenamePendingJson(Path, int)}.
+   *
+   * @param redoRenameAtomicity {@link RenameAtomicity} to be spied.
+   * @param answer mocked behavior for {@link RenameAtomicity#readRenamePendingJson(Path, int)}.
+   *
+   * @throws AzureBlobFileSystemException server error or error from mocked behavior.
+   */
+  public static void addReadPathMock(RenameAtomicity redoRenameAtomicity,
+      Answer answer)
+      throws AzureBlobFileSystemException {
+    Mockito.doAnswer(answer)
+        .when(redoRenameAtomicity)
+        .readRenamePendingJson(Mockito.any(Path.class), Mockito.anyInt());
+  }
+}

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

@@ -24,10 +24,6 @@ import java.net.SocketException;
 import java.net.URL;
 import java.time.Duration;
 
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
-import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
-import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.assertj.core.api.Assertions;
 import org.junit.Assume;
 import org.junit.Test;
@@ -35,26 +31,31 @@ import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.EtagSource;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
 import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.statistics.IOStatistics;
 
+import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
+import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CONNECTION_TIMEOUT;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_READ_TIMEOUT;
-import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND;
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.PATH_ALREADY_EXISTS;
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -121,8 +122,8 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
 
     // We need to throw an exception once a rename is triggered with
     // destination having no parent, but after a retry it needs to succeed.
-    when(mockClient.renamePath(sourcePath, destNoParentPath, null, null,
-        null, false, isNamespaceEnabled))
+    when(mockClient.renamePath(sourcePath, destNoParentPath, null,
+        null, null, false))
         .thenThrow(destParentNotFound)
         .thenReturn(recoveredMetaDataIncompleteResult);
 
@@ -130,12 +131,12 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
     intercept(AzureBlobFileSystemException.class,
         () -> mockClient.renamePath(sourcePath,
         destNoParentPath, null, null,
-        null, false, isNamespaceEnabled));
+        null, false));
 
     AbfsClientRenameResult resultOfSecondRenameCall =
         mockClient.renamePath(sourcePath,
         destNoParentPath, null, null,
-        null, false, isNamespaceEnabled);
+        null, false);
 
     // the second rename call should be the recoveredResult due to
     // metaDataIncomplete
@@ -151,8 +152,9 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
 
     // Verify renamePath occurred two times implying a retry was attempted.
     verify(mockClient, times(2))
-        .renamePath(sourcePath, destNoParentPath, null, null, null, false,
-                isNamespaceEnabled);
+        .renamePath(sourcePath, destNoParentPath, null,
+            null, null,
+            false);
 
   }
 
@@ -276,7 +278,7 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
     // +2 for getPathStatus calls
     assertThatStatisticCounter(ioStats,
             CONNECTIONS_MADE.getStatName())
-            .isEqualTo(4 + connMadeBeforeRename);
+            .isEqualTo(5 + connMadeBeforeRename);
     // the RENAME_PATH_ATTEMPTS stat should be incremented by 1
     // retries happen internally within AbfsRestOperation execute()
     // the stat for RENAME_PATH_ATTEMPTS is updated only once before execute() is called
@@ -354,7 +356,7 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
     // last getPathStatus call should be skipped
     assertThatStatisticCounter(ioStats,
             CONNECTIONS_MADE.getStatName())
-            .isEqualTo(3 + connMadeBeforeRename);
+            .isEqualTo(4 + connMadeBeforeRename);
 
     // the RENAME_PATH_ATTEMPTS stat should be incremented by 1
     // retries happen internally within AbfsRestOperation execute()
@@ -397,8 +399,9 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
 
     // source eTag does not match -> throw exception
     expectErrorCode(SOURCE_PATH_NOT_FOUND, intercept(AbfsRestOperationException.class, () ->
-            spyClient.renamePath(path1, path2, null, testTracingContext, null, false,
-                    isNamespaceEnabled)));
+            spyClient.renamePath(path1, path2, null,
+                testTracingContext, null,
+                false)));
   }
 
   /**
@@ -423,8 +426,9 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
 
     // source eTag does not match -> throw exception
     expectErrorCode(PATH_ALREADY_EXISTS, intercept(AbfsRestOperationException.class, () ->
-            spyClient.renamePath(path1, path2, null, testTracingContext, null, false,
-                    isNamespaceEnabled)));
+            spyClient.renamePath(path1, path2, null,
+                testTracingContext, null,
+                false)));
   }
 
   /**
@@ -434,6 +438,9 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
   @Test
   public void testRenameRecoveryUnsupportedForFlatNamespace() throws Exception {
     Assume.assumeTrue(!isNamespaceEnabled);
+    // In DFS endpoint, renamePath is O(1) API call and idempotency issue can happen.
+    // For blob endpoint, client orchestrates the rename operation.
+    Assume.assumeTrue(getAbfsServiceType() == AbfsServiceType.DFS);
     AzureBlobFileSystem fs = getFileSystem();
     AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
     TracingContext testTracingContext = getTestTracingContext(fs, false);
@@ -456,8 +463,9 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
     Long renamePathAttemptsBeforeRename = lookupCounterStatistic(ioStats, RENAME_PATH_ATTEMPTS.getStatName());
 
     expectErrorCode(SOURCE_PATH_NOT_FOUND, intercept(AbfsRestOperationException.class, () ->
-            mockClient.renamePath(path1, path2, null, testTracingContext, null, false,
-                    isNamespaceEnabled)));
+            mockClient.renamePath(path1, path2, null,
+                testTracingContext, null,
+                false)));
 
     // validating stat counters after rename
 

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

@@ -0,0 +1,142 @@
+/**
+ * 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.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsTestWithTimeout;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.BlobListResultSchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FS_AZURE_LISTING_ACTION_THREADS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE;
+
+public class TestListActionTaker extends AbstractAbfsTestWithTimeout {
+
+  public TestListActionTaker() throws Exception {
+  }
+
+  /**
+   * This test method verifies the behavior of the producer-consumer pattern implemented in the ListActionTaker class.
+   * The producer (ListActionTaker) should only resume producing (listing and enqueuing blobs) when the consumer lag becomes tolerable.
+   * The test method mocks the necessary components and checks the behavior of the ListActionTaker under these conditions.
+   *
+   * @throws IOException if an I/O error occurs
+   */
+  @Test
+  public void testProducerResumeOnlyOnConsumerLagBecomesTolerable() throws
+      IOException {
+    Path path = new Path("test");
+    AbfsConfiguration abfsConfiguration = Mockito.mock(AbfsConfiguration.class);
+    AbfsBlobClient client = Mockito.mock(AbfsBlobClient.class);
+    Mockito.doReturn(abfsConfiguration).when(client).getAbfsConfiguration();
+    Mockito.doReturn(DEFAULT_AZURE_LIST_MAX_RESULTS)
+        .when(abfsConfiguration)
+        .getListingMaxConsumptionLag();
+    Mockito.doReturn(DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE)
+        .when(abfsConfiguration)
+        .getProducerQueueMaxSize();
+    AbfsRestOperation op = Mockito.mock(AbfsRestOperation.class);
+    AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);
+    Mockito.doReturn(httpOperation).when(op).getResult();
+    BlobListResultSchema listResultSchema = Mockito.mock(
+        BlobListResultSchema.class);
+    Mockito.doReturn(listResultSchema)
+        .when(httpOperation)
+        .getListResultSchema();
+    Mockito.doReturn("a")
+        .doReturn("b")
+        .doReturn("c")
+        .doReturn(null)
+        .when(listResultSchema).getNextMarker();
+    TracingContext tracingContext = Mockito.mock(TracingContext.class);
+    ListActionTaker listActionTaker = new ListActionTaker(path, client,
+        tracingContext) {
+      private ListBlobQueue listBlobQueue;
+      private boolean isListAndEnqueueInProgress;
+      private boolean completed;
+
+      @Override
+      protected ListBlobQueue createListBlobQueue(final AbfsConfiguration configuration)
+          throws InvalidConfigurationValueException {
+        listBlobQueue = super.createListBlobQueue(configuration);
+        return listBlobQueue;
+      }
+
+      @Override
+      int getMaxConsumptionParallelism() {
+        return DEFAULT_FS_AZURE_LISTING_ACTION_THREADS;
+      }
+
+      @Override
+      boolean takeAction(final Path path) throws AzureBlobFileSystemException {
+        while (!isListAndEnqueueInProgress
+            && listBlobQueue.size() < DEFAULT_AZURE_LIST_MAX_RESULTS
+            && !completed) {
+          // wait for the producer to produce more items
+        }
+        return true;
+      }
+
+
+      @Override
+      protected String listAndEnqueue(final ListBlobQueue listBlobQueue,
+          final String continuationToken) throws AzureBlobFileSystemException {
+        isListAndEnqueueInProgress = true;
+        String contToken = super.listAndEnqueue(listBlobQueue,
+            continuationToken);
+        isListAndEnqueueInProgress = false;
+        if (contToken == null) {
+          completed = true;
+        }
+        return contToken;
+      }
+
+      @Override
+      protected void addPaths(final List<Path> paths,
+          final ListResultSchema retrievedSchema) {
+        for (int i = 0; i < DEFAULT_AZURE_LIST_MAX_RESULTS; i++) {
+          paths.add(new Path("test" + i));
+        }
+      }
+    };
+    final int[] occurrences = {0};
+    Mockito.doAnswer(answer -> {
+          occurrences[0]++;
+          Assertions.assertThat((int) answer.getArgument(2))
+              .isEqualTo(DEFAULT_AZURE_LIST_MAX_RESULTS);
+          return op;
+        }).when(client)
+        .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(),
+            Mockito.nullable(String.class), Mockito.any(TracingContext.class));
+
+    listActionTaker.listRecursiveAndTakeAction();
+  }
+}

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

@@ -18,9 +18,10 @@
 
 package org.apache.hadoop.fs.azurebfs.utils;
 
-import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.assertj.core.api.Assertions;
 
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
 
 /**
@@ -41,6 +42,8 @@ public class TracingHeaderValidator implements Listener {
   private String ingressHandler = null;
   private String position = null;
 
+  private Integer operatedBlobCount = null;
+
   @Override
   public void callTracingHeaderValidator(String tracingContextHeader,
       TracingHeaderFormat format) {
@@ -56,6 +59,7 @@ public class TracingHeaderValidator implements Listener {
     tracingHeaderValidator.primaryRequestId = primaryRequestId;
     tracingHeaderValidator.ingressHandler = ingressHandler;
     tracingHeaderValidator.position = position;
+    tracingHeaderValidator.operatedBlobCount = operatedBlobCount;
     return tracingHeaderValidator;
   }
 
@@ -82,6 +86,13 @@ public class TracingHeaderValidator implements Listener {
     if (format != TracingHeaderFormat.ALL_ID_FORMAT) {
       return;
     }
+    if (idList.length >= 8) {
+      if (operatedBlobCount != null) {
+        Assertions.assertThat(Integer.parseInt(idList[7]))
+                .describedAs("OperatedBlobCount is incorrect")
+                .isEqualTo(operatedBlobCount);
+      }
+    }
     if (!primaryRequestId.isEmpty() && !idList[3].isEmpty()) {
       Assertions.assertThat(idList[3])
           .describedAs("PrimaryReqID should be common for these requests")
@@ -97,6 +108,9 @@ public class TracingHeaderValidator implements Listener {
   private void validateBasicFormat(String[] idList) {
     if (format == TracingHeaderFormat.ALL_ID_FORMAT) {
       int expectedSize = 8;
+      if (operatedBlobCount != null) {
+        expectedSize += 1;
+      }
       if (ingressHandler != null) {
         expectedSize += 2;
       }
@@ -171,4 +185,12 @@ public class TracingHeaderValidator implements Listener {
   public void updatePosition(String position) {
     this.position = position;
   }
+
+  /**
+   * Sets the value of the number of blobs operated on
+   * @param operatedBlobCount number of blobs operated on
+   */
+  public void setOperatedBlobCount(Integer operatedBlobCount) {
+    this.operatedBlobCount = operatedBlobCount;
+  }
 }