Browse Source

HADOOP-19187: [ABFS][FNSOverBlob] AbfsClient Refactoring to Support Multiple Implementation of Clients. (#6879)

Refactor AbfsClient into DFS and Blob Client.

Contributed by Anuj Modi
Anuj Modi 11 tháng trước cách đây
mục cha
commit
b15ed27cfb
22 tập tin đã thay đổi với 2162 bổ sung769 xóa
  1. 2 0
      hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
  2. 80 3
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
  3. 51 5
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
  4. 43 59
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
  5. 2 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
  6. 37 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsServiceType.java
  7. 27 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
  8. 2 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FSOperationType.java
  9. 1 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
  10. 4 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java
  11. 4 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java
  12. 296 685
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
  13. 127 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java
  14. 1302 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
  15. 36 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java
  16. 82 0
      hadoop-tools/hadoop-azure/src/site/markdown/fns_blob.md
  17. 1 0
      hadoop-tools/hadoop-azure/src/site/markdown/index.md
  18. 1 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java
  19. 7 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java
  20. 41 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java
  21. 10 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java
  22. 6 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java

+ 2 - 0
hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml

@@ -44,6 +44,8 @@
 <suppressions>
     <suppress checks="ParameterNumber|MagicNumber"
               files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]AzureBlobFileSystemStore.java"/>
+    <suppress checks="ParameterNumber"
+              files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]services[\\/]AbfsClient.java"/>
     <suppress checks="ParameterNumber|MagicNumber"
               files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]utils[\\/]Base64.java"/>
     <suppress checks="ParameterNumber|VisibilityModifier"

+ 80 - 3
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java

@@ -22,6 +22,7 @@ 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;
@@ -74,6 +75,7 @@ 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.*;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.*;
@@ -87,6 +89,8 @@ public class AbfsConfiguration{
 
   private final Configuration rawConfig;
   private final String accountName;
+  // Service type identified from URL used to initialize FileSystem.
+  private final AbfsServiceType fsConfiguredServiceType;
   private final boolean isSecure;
   private static final Logger LOG = LoggerFactory.getLogger(AbfsConfiguration.class);
 
@@ -94,6 +98,10 @@ public class AbfsConfiguration{
       DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED)
   private String isNamespaceEnabledAccount;
 
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK,
+      DefaultValue = DEFAULT_FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK)
+  private boolean isDfsToBlobFallbackEnabled;
+
   @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_MAX_CONCURRENT_REQUESTS,
       DefaultValue = -1)
   private int writeMaxConcurrentRequestCount;
@@ -408,11 +416,14 @@ public class AbfsConfiguration{
   private String clientProvidedEncryptionKey;
   private String clientProvidedEncryptionKeySHA;
 
-  public AbfsConfiguration(final Configuration rawConfig, String accountName)
-      throws IllegalAccessException, InvalidConfigurationValueException, IOException {
+  public AbfsConfiguration(final Configuration rawConfig,
+      String accountName,
+      AbfsServiceType fsConfiguredServiceType)
+      throws IllegalAccessException, IOException {
     this.rawConfig = ProviderUtils.excludeIncompatibleCredentialProviders(
         rawConfig, AzureBlobFileSystem.class);
     this.accountName = accountName;
+    this.fsConfiguredServiceType = fsConfiguredServiceType;
     this.isSecure = getBoolean(FS_AZURE_SECURE_MODE, false);
 
     Field[] fields = this.getClass().getDeclaredFields();
@@ -434,10 +445,75 @@ public class AbfsConfiguration{
     }
   }
 
+  public AbfsConfiguration(final Configuration rawConfig, String accountName)
+      throws IllegalAccessException, IOException {
+    this(rawConfig, accountName, AbfsServiceType.DFS);
+  }
+
   public Trilean getIsNamespaceEnabledAccount() {
     return Trilean.getTrilean(isNamespaceEnabledAccount);
   }
 
+  /**
+   * Returns the service type to be used based on the filesystem configuration.
+   * Precedence is given to service type configured for FNS Accounts using
+   * "fs.azure.fns.account.service.type". If not configured, then the service
+   * type identified from url used to initialize filesystem will be used.
+   * @return the service type.
+   */
+  public AbfsServiceType getFsConfiguredServiceType() {
+    return getEnum(FS_AZURE_FNS_ACCOUNT_SERVICE_TYPE, fsConfiguredServiceType);
+  }
+
+  /**
+   * Returns the service type configured for FNS Accounts to override the
+   * service type identified by URL used to initialize the filesystem.
+   * @return the service type.
+   */
+  public AbfsServiceType getConfiguredServiceTypeForFNSAccounts() {
+    return getEnum(FS_AZURE_FNS_ACCOUNT_SERVICE_TYPE, null);
+  }
+
+  /**
+   * Returns the service type to be used for Ingress Operations irrespective of account type.
+   * Default value is the same as the service type configured for the file system.
+   * @return the service type.
+   */
+  public AbfsServiceType getIngressServiceType() {
+    return getEnum(FS_AZURE_INGRESS_SERVICE_TYPE, getFsConfiguredServiceType());
+  }
+
+  /**
+   * Returns whether there is a need to move traffic from DFS to Blob.
+   * Needed when the service type is DFS and operations are experiencing compatibility issues.
+   * @return true if fallback enabled.
+   */
+  public boolean isDfsToBlobFallbackEnabled() {
+    return isDfsToBlobFallbackEnabled;
+  }
+
+  /**
+   * Checks if the service type configured is valid for account type used.
+   * HNS Enabled accounts cannot have service type as BLOB.
+   * @param isHNSEnabled Flag to indicate if HNS is enabled for the account.
+   * @throws InvalidConfigurationValueException if the service type is invalid.
+   */
+  public void validateConfiguredServiceType(boolean isHNSEnabled)
+      throws InvalidConfigurationValueException {
+    // Todo: [FnsOverBlob] - Remove this check, Failing FS Init with Blob Endpoint Until FNS over Blob is ready.
+    if (getFsConfiguredServiceType() == AbfsServiceType.BLOB) {
+      throw new InvalidConfigurationValueException(FS_DEFAULT_NAME_KEY,
+          "Blob Endpoint Support not yet available");
+    }
+    if (isHNSEnabled && getConfiguredServiceTypeForFNSAccounts() == AbfsServiceType.BLOB) {
+      throw new InvalidConfigurationValueException(
+          FS_AZURE_FNS_ACCOUNT_SERVICE_TYPE, "Cannot be BLOB for HNS Account");
+    } else if (isHNSEnabled && fsConfiguredServiceType == AbfsServiceType.BLOB) {
+      throw new InvalidConfigurationValueException(FS_DEFAULT_NAME_KEY,
+          "Blob Endpoint Url Cannot be used to initialize filesystem for HNS Account");
+    }
+  }
+
   /**
    * Gets the Azure Storage account name corresponding to this instance of configuration.
    * @return the Azure Storage account name
@@ -478,6 +554,7 @@ public class AbfsConfiguration{
    * Returns the account-specific value if it exists, then looks for an
    * account-agnostic value.
    * @param key Account-agnostic configuration key
+   * @param defaultValue Value returned if none is configured
    * @return value if one exists, else the default value
    */
   public String getString(String key, String defaultValue) {
@@ -522,7 +599,7 @@ public class AbfsConfiguration{
    * looks for an account-agnostic value.
    * @param key Account-agnostic configuration key
    * @return value in String form if one exists, else null
-   * @throws IOException
+   * @throws IOException if parsing fails.
    */
   public String getPasswordString(String key) throws IOException {
     char[] passchars = rawConfig.getPassword(accountConf(key));

+ 51 - 5
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

@@ -45,6 +45,7 @@ import javax.annotation.Nullable;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.hadoop.fs.impl.BackReference;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.util.Preconditions;
@@ -109,13 +110,16 @@ import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.LambdaUtils;
 import org.apache.hadoop.util.Progressable;
 
+import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
 import static java.net.HttpURLConnection.HTTP_CONFLICT;
+import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT;
 import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS;
 import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CPK_IN_NON_HNS_ACCOUNT_ERROR_MESSAGE;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLOCKS_BUFFER;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_HNS_ENABLED;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_ACTIVE_BLOCKS;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_BUFFER_DIR;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT;
@@ -213,6 +217,23 @@ public class AzureBlobFileSystem extends FileSystem
 
     TracingContext tracingContext = new TracingContext(clientCorrelationId,
             fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener);
+
+    /*
+     * Validate the service type configured in the URI is valid for account type used.
+     * HNS Account Cannot have Blob Endpoint URI.
+     */
+    try {
+      abfsConfiguration.validateConfiguredServiceType(
+          tryGetIsNamespaceEnabled(new TracingContext(tracingContext)));
+    } catch (InvalidConfigurationValueException ex) {
+      LOG.debug("File system configured with Invalid Service Type", ex);
+      throw ex;
+    } catch (AzureBlobFileSystemException ex) {
+      LOG.debug("Failed to determine account type for service type validation", ex);
+      throw new InvalidConfigurationValueException(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, ex);
+    }
+
+    // Create the file system if it does not exist.
     if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) {
       if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), tracingContext) == null) {
         try {
@@ -230,10 +251,7 @@ public class AzureBlobFileSystem extends FileSystem
      */
     if ((isEncryptionContextCPK(abfsConfiguration) || isGlobalKeyCPK(
         abfsConfiguration))
-        && !getIsNamespaceEnabled(
-        new TracingContext(clientCorrelationId, fileSystemId,
-            FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat,
-            listener))) {
+        && !getIsNamespaceEnabled(new TracingContext(tracingContext))) {
       /*
        * Close the filesystem gracefully before throwing exception. Graceful close
        * will ensure that all resources are released properly.
@@ -1400,6 +1418,34 @@ public class AzureBlobFileSystem extends FileSystem
     }
   }
 
+  /**
+   * Utility function to check if the namespace is enabled on the storage account.
+   * If request fails with 4xx other than 400, it will be inferred as HNS.
+   * @param tracingContext tracing context
+   * @return true if namespace is enabled, false otherwise.
+   * @throws AzureBlobFileSystemException if any other error occurs.
+   */
+  private boolean tryGetIsNamespaceEnabled(TracingContext tracingContext)
+      throws AzureBlobFileSystemException{
+    try {
+      return getIsNamespaceEnabled(tracingContext);
+    } catch (AbfsRestOperationException ex) {
+      /*
+       * Exception will be thrown for any non 400 error code.
+       * If status code is in 4xx range, it means it's an HNS account.
+       * If status code is in 5xx range, it means nothing can be inferred.
+       * In case of network errors status code will be -1.
+       */
+      int statusCode = ex.getStatusCode();
+      if (statusCode > HTTP_BAD_REQUEST && statusCode < HTTP_INTERNAL_ERROR) {
+        LOG.debug("getNamespace failed with non 400 user error", ex);
+        statIncrement(ERROR_IGNORED);
+        return true;
+      }
+      throw ex;
+    }
+  }
+
   private boolean fileSystemExists() throws IOException {
     LOG.debug(
         "AzureBlobFileSystem.fileSystemExists uri: {}", uri);
@@ -1660,7 +1706,7 @@ public class AzureBlobFileSystem extends FileSystem
   @VisibleForTesting
   boolean getIsNamespaceEnabled(TracingContext tracingContext)
       throws AzureBlobFileSystemException {
-    return abfsStore.getIsNamespaceEnabled(tracingContext);
+    return getAbfsStore().getIsNamespaceEnabled(tracingContext);
   }
 
   /**

+ 43 - 59
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java

@@ -29,11 +29,9 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.nio.ByteBuffer;
-import java.nio.CharBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
-import java.nio.charset.CharsetEncoder;
 import java.nio.charset.StandardCharsets;
 import java.text.SimpleDateFormat;
 import java.time.Instant;
@@ -55,11 +53,13 @@ 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;
@@ -158,6 +158,7 @@ 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.FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT;
 
 /**
@@ -169,6 +170,13 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
   private static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystemStore.class);
 
   private AbfsClient client;
+
+  /**
+   * Variable to hold the client handler which will determine the operative
+   * client based on the service type configured.
+   * Initialized in the {@link #initializeClient(URI, String, String, boolean)}.
+   */
+  private AbfsClientHandler clientHandler;
   private URI uri;
   private String userName;
   private String primaryUserGroup;
@@ -221,7 +229,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     leaseRefs = Collections.synchronizedMap(new WeakHashMap<>());
 
     try {
-      this.abfsConfiguration = new AbfsConfiguration(abfsStoreBuilder.configuration, accountName);
+      this.abfsConfiguration = new AbfsConfiguration(abfsStoreBuilder.configuration,
+          accountName, getAbfsServiceTypeFromUrl());
     } catch (IllegalAccessException exception) {
       throw new FileSystemOperationUnhandledException(exception);
     }
@@ -286,6 +295,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
   /**
    * Checks if the given key in Azure Storage should be stored as a page
    * blob instead of block blob.
+   * @param key The key to check.
+   * @return True if the key should be stored as a page blob, false otherwise.
    */
   public boolean isAppendBlobKey(String key) {
     return isKeyForDirectorySet(key, appendBlobDirSet);
@@ -497,15 +508,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
 
     try (AbfsPerfInfo perfInfo = startTracking("setFilesystemProperties",
             "setFilesystemProperties")) {
-      final String commaSeparatedProperties;
-      try {
-        commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
-      } catch (CharacterCodingException ex) {
-        throw new InvalidAbfsRestOperationException(ex);
-      }
 
-      final AbfsRestOperation op = client
-          .setFilesystemProperties(commaSeparatedProperties, tracingContext);
+      final AbfsRestOperation op = getClient()
+          .setFilesystemProperties(properties, tracingContext);
       perfInfo.registerResult(op.getResult()).registerSuccess(true);
     }
   }
@@ -590,18 +595,13 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
               path,
               properties);
 
-      final String commaSeparatedProperties;
-      try {
-        commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
-      } catch (CharacterCodingException ex) {
-        throw new InvalidAbfsRestOperationException(ex);
-      }
+
       final String relativePath = getRelativePath(path);
       final ContextEncryptionAdapter contextEncryptionAdapter
           = createEncryptionAdapterFromServerStoreContext(relativePath,
           tracingContext);
       final AbfsRestOperation op = client
-          .setPathProperties(getRelativePath(path), commaSeparatedProperties,
+          .setPathProperties(getRelativePath(path), properties,
               tracingContext, contextEncryptionAdapter);
       contextEncryptionAdapter.destroy();
       perfInfo.registerResult(op.getResult()).registerSuccess(true);
@@ -1090,7 +1090,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
         final AbfsClientRenameResult abfsClientRenameResult =
             client.renamePath(sourceRelativePath, destinationRelativePath,
                 continuation, tracingContext, sourceEtag, false,
-                    isNamespaceEnabled);
+                  isNamespaceEnabled);
 
         AbfsRestOperation op = abfsClientRenameResult.getOp();
         perfInfo.registerResult(op.getResult());
@@ -1369,7 +1369,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     SimpleDateFormat simpleDateFormat = new SimpleDateFormat(TOKEN_DATE_PATTERN, Locale.US);
     String date = simpleDateFormat.format(new Date());
     String token = String.format("%06d!%s!%06d!%s!%06d!%s!",
-            path.length(), path, startFrom.length(), startFrom, date.length(), date);
+          path.length(), path, startFrom.length(), startFrom, date.length(), date);
     String base64EncodedToken = Base64.encode(token.getBytes(StandardCharsets.UTF_8));
 
     StringBuilder encodedTokenBuilder = new StringBuilder(base64EncodedToken.length() + 5);
@@ -1810,18 +1810,29 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
 
     LOG.trace("Initializing AbfsClient for {}", baseUrl);
     if (tokenProvider != null) {
-      this.client = new AbfsClient(baseUrl, creds, abfsConfiguration,
+      this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration,
           tokenProvider, encryptionContextProvider,
           populateAbfsClientContext());
     } else {
-      this.client = new AbfsClient(baseUrl, creds, abfsConfiguration,
+      this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration,
           sasTokenProvider, encryptionContextProvider,
           populateAbfsClientContext());
     }
 
+    this.client = getClientHandler().getClient();
     LOG.trace("AbfsClient init complete");
   }
 
+  private AbfsServiceType getAbfsServiceTypeFromUrl() {
+    if (uri.toString().contains(ABFS_BLOB_DOMAIN_NAME)) {
+      return AbfsServiceType.BLOB;
+    }
+    // In case of DFS Domain name or any other custom endpoint, the service
+    // type is to be identified as default DFS.
+    LOG.debug("Falling back to default service type DFS");
+    return AbfsServiceType.DFS;
+  }
+
   /**
    * Populate a new AbfsClientContext instance with the desired properties.
    *
@@ -1861,43 +1872,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
         && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
   }
 
-  /**
-   * Convert properties stored in a Map into a comma separated string. For map
-   * <key1:value1; key2:value2: keyN:valueN>, method would convert to:
-   * key1=value1,key2=value,...,keyN=valueN
-   * */
-   @VisibleForTesting
-   String convertXmsPropertiesToCommaSeparatedString(final Map<String,
-      String> properties) throws
-          CharacterCodingException {
-    StringBuilder commaSeparatedProperties = new StringBuilder();
-
-    final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder();
-
-    for (Map.Entry<String, String> propertyEntry : properties.entrySet()) {
-      String key = propertyEntry.getKey();
-      String value = propertyEntry.getValue();
-
-      Boolean canEncodeValue = encoder.canEncode(value);
-      if (!canEncodeValue) {
-        throw new CharacterCodingException();
-      }
-
-      String encodedPropertyValue = Base64.encode(encoder.encode(CharBuffer.wrap(value)).array());
-      commaSeparatedProperties.append(key)
-              .append(AbfsHttpConstants.EQUAL)
-              .append(encodedPropertyValue);
-
-      commaSeparatedProperties.append(AbfsHttpConstants.COMMA);
-    }
-
-    if (commaSeparatedProperties.length() != 0) {
-      commaSeparatedProperties.deleteCharAt(commaSeparatedProperties.length() - 1);
-    }
-
-    return commaSeparatedProperties.toString();
-  }
-
   private Hashtable<String, String> parseCommaSeparatedXmsProperties(String xMsProperties) throws
           InvalidFileSystemPropertyException, InvalidAbfsRestOperationException {
     Hashtable<String, String> properties = new Hashtable<>();
@@ -2176,6 +2150,16 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     return this.client;
   }
 
+  @VisibleForTesting
+  public AbfsClient getClient(AbfsServiceType serviceType) {
+    return getClientHandler().getClient(serviceType);
+  }
+
+  @VisibleForTesting
+  public AbfsClientHandler getClientHandler() {
+    return this.clientHandler;
+  }
+
   @VisibleForTesting
   void setClient(AbfsClient client) {
     this.client = client;

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

@@ -101,6 +101,8 @@ public final class AbfsHttpConstants {
   public static final String GMT_TIMEZONE = "GMT";
   public static final String APPLICATION_JSON = "application/json";
   public static final String APPLICATION_OCTET_STREAM = "application/octet-stream";
+  public static final String XMS_PROPERTIES_ENCODING_ASCII = "ISO-8859-1";
+  public static final String XMS_PROPERTIES_ENCODING_UNICODE = "UTF-8";
 
   public static final String ROOT_PATH = "/";
   public static final String ACCESS_MASK = "mask:";

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

@@ -0,0 +1,37 @@
+/**
+ * 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.constants;
+
+/**
+ * Azure Storage Offers two sets of Rest APIs for interacting with the storage account.
+ * <ol>
+ *   <li>Blob Rest API: <a href = https://learn.microsoft.com/en-us/rest/api/storageservices/blob-service-rest-api></a></li>
+ *   <li>Data Lake Rest API: <a href = https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/operation-groups></a></li>
+ * </ol>
+ */
+public enum AbfsServiceType {
+  /**
+   * Service type to set operative endpoint as Data Lake Rest API.
+   */
+  DFS,
+  /**
+   * Service type to set operative endpoint as Blob Rest API.
+   */
+  BLOB
+}

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

@@ -32,9 +32,35 @@ public final class ConfigurationKeys {
   /**
    * Config to specify if the configured account is HNS enabled or not. If
    * this config is not set, getacl call is made on account filesystem root
-   * path to determine HNS status.
+   * path on DFS Endpoint to determine HNS status.
    */
   public static final String FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "fs.azure.account.hns.enabled";
+
+  /**
+   * Config to specify which {@link  AbfsServiceType} to use with HNS-Disabled Account type.
+   * Default value will be identified from URL used to initialize filesystem.
+   * This will allow an override to choose service endpoint in cases where any
+   * local DNS resolution is set for account and driver is unable to detect
+   * intended endpoint from the url used to initialize filesystem.
+   * If configured Blob for HNS-Enabled account, FS init will fail.
+   * Value {@value} case-insensitive "DFS" or "BLOB"
+   */
+  public static final String FS_AZURE_FNS_ACCOUNT_SERVICE_TYPE = "fs.azure.fns.account.service.type";
+
+  /**
+   * Config to specify which {@link AbfsServiceType} to use only for Ingress Operations.
+   * Other operations will continue to move to the FS configured service endpoint.
+   * Value {@value} case-insensitive "DFS" or "BLOB"
+   */
+  public static final String FS_AZURE_INGRESS_SERVICE_TYPE = "fs.azure.ingress.service.type";
+
+  /**
+   * Config to be set only for cases where traffic over dfs endpoint is
+   * experiencing compatibility issues and need to move to blob for mitigation.
+   * Value {@value} case-insensitive "True" or "False"
+   */
+  public static final String FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK = "fs.azure.enable.dfstoblob.fallback";
+
   /**
    * Enable or disable expect hundred continue header.
    * Value: {@value}.

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

@@ -45,7 +45,8 @@ public enum FSOperationType {
     SET_OWNER("SO"),
     SET_ACL("SA"),
     TEST_OP("TS"),
-    WRITE("WR");
+    WRITE("WR"),
+    INIT("IN");
 
     private final String opCode;
 

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

@@ -32,6 +32,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_ST
 public final class FileSystemConfigurations {
 
   public static final String DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "";
+  public static final boolean DEFAULT_FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK = false;
   public static final boolean DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = true;
   public static final String USER_HOME_DIRECTORY_PREFIX = "/user";
 

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

@@ -38,5 +38,8 @@ public final class FileSystemUriSchemes {
   public static final String WASB_SECURE_SCHEME = "wasbs";
   public static final String WASB_DNS_PREFIX = "blob";
 
+  public static final String ABFS_DFS_DOMAIN_NAME = "dfs.core.windows.net";
+  public static final String ABFS_BLOB_DOMAIN_NAME = "blob.core.windows.net";
+
   private FileSystemUriSchemes() {}
-}
+}

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

@@ -34,4 +34,8 @@ public class InvalidConfigurationValueException extends AzureBlobFileSystemExcep
   public InvalidConfigurationValueException(String configKey) {
     super("Invalid configuration value detected for " + configKey);
   }
+
+  public InvalidConfigurationValueException(String configKey, String message) {
+    super(String.format("Invalid configuration value detected for \"%s\". %s ", configKey, message));
+  }
 }

Những thai đổi đã bị hủy bỏ vì nó quá lớn
+ 296 - 685
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java


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

@@ -0,0 +1,127 @@
+/**
+ * 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.net.URL;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
+import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
+import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
+import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
+
+import static org.apache.hadoop.fs.azurebfs.utils.UriUtils.changeUrlFromBlobToDfs;
+
+/**
+ * AbfsClientHandler is a class that provides a way to get the AbfsClient
+ * based on the service type.
+ */
+public class AbfsClientHandler {
+  public static final Logger LOG = LoggerFactory.getLogger(AbfsClientHandler.class);
+
+  private AbfsServiceType defaultServiceType;
+  private final AbfsDfsClient dfsAbfsClient;
+
+  public AbfsClientHandler(final URL baseUrl,
+      final SharedKeyCredentials sharedKeyCredentials,
+      final AbfsConfiguration abfsConfiguration,
+      final AccessTokenProvider tokenProvider,
+      final EncryptionContextProvider encryptionContextProvider,
+      final AbfsClientContext abfsClientContext) throws IOException {
+    this.dfsAbfsClient = createDfsClient(baseUrl, sharedKeyCredentials,
+        abfsConfiguration, tokenProvider, null, encryptionContextProvider,
+        abfsClientContext);
+    initServiceType(abfsConfiguration);
+  }
+
+  public AbfsClientHandler(final URL baseUrl,
+      final SharedKeyCredentials sharedKeyCredentials,
+      final AbfsConfiguration abfsConfiguration,
+      final SASTokenProvider sasTokenProvider,
+      final EncryptionContextProvider encryptionContextProvider,
+      final AbfsClientContext abfsClientContext) throws IOException {
+    this.dfsAbfsClient = createDfsClient(baseUrl, sharedKeyCredentials,
+        abfsConfiguration, null, sasTokenProvider, encryptionContextProvider,
+        abfsClientContext);
+    initServiceType(abfsConfiguration);
+  }
+
+  /**
+   * Initialize the default service type based on the user configuration.
+   * @param abfsConfiguration set by user.
+   */
+  private void initServiceType(final AbfsConfiguration abfsConfiguration) {
+    this.defaultServiceType = abfsConfiguration.getFsConfiguredServiceType();
+  }
+
+  /**
+   * Get the AbfsClient based on the default service type.
+   * @return AbfsClient
+   */
+  public AbfsClient getClient() {
+    return getClient(defaultServiceType);
+  }
+
+  /**
+   * Get the AbfsClient based on the service type.
+   * @param serviceType AbfsServiceType
+   * @return AbfsClient
+   */
+  public AbfsClient getClient(AbfsServiceType serviceType) {
+    return serviceType == AbfsServiceType.DFS ? dfsAbfsClient : null;
+  }
+
+  /**
+   * Create the AbfsDfsClient using the url used to configure file system.
+   * If URL is for Blob endpoint, it will be converted to DFS endpoint.
+   * @param baseUrl URL
+   * @param creds SharedKeyCredentials
+   * @param abfsConfiguration AbfsConfiguration
+   * @param tokenProvider AccessTokenProvider
+   * @param sasTokenProvider SASTokenProvider
+   * @param encryptionContextProvider EncryptionContextProvider
+   * @param abfsClientContext AbfsClientContext
+   * @return AbfsDfsClient with DFS endpoint URL
+   * @throws IOException if URL conversion fails.
+   */
+  private AbfsDfsClient createDfsClient(final URL baseUrl,
+      final SharedKeyCredentials creds,
+      final AbfsConfiguration abfsConfiguration,
+      final AccessTokenProvider tokenProvider,
+      final SASTokenProvider sasTokenProvider,
+      final EncryptionContextProvider encryptionContextProvider,
+      final AbfsClientContext abfsClientContext) throws IOException {
+    URL dfsUrl = changeUrlFromBlobToDfs(baseUrl);
+    if (tokenProvider != null) {
+      LOG.debug("Creating AbfsDfsClient with access token provider using the URL: {}", dfsUrl);
+      return new AbfsDfsClient(dfsUrl, creds, abfsConfiguration,
+          tokenProvider, encryptionContextProvider,
+          abfsClientContext);
+    } else {
+      LOG.debug("Creating AbfsDfsClient with SAS token provider using the URL: {}", dfsUrl);
+      return new AbfsDfsClient(dfsUrl, creds, abfsConfiguration,
+          sasTokenProvider, encryptionContextProvider,
+          abfsClientContext);
+    }
+  }
+}

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

@@ -0,0 +1,1302 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ApiVersion;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+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.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
+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.Base64;
+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.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;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_BLOB_TYPE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_JSON;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_OCTET_STREAM;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BREAK_LEASE_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHECK_ACCESS;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMMA;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DEFAULT_LEASE_BREAK_PERIOD;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DIRECTORY;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FILE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FILESYSTEM;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FLUSH_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.GET_ACCESS_CONTROL;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.GET_STATUS;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_POST;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.RELEASE_LEASE_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.RENEW_LEASE_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SET_ACCESS_CONTROL;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SET_PROPERTIES_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.STAR;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_ASCII;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ACCEPT;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_MATCH;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_NONE_MATCH;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.RANGE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.USER_AGENT;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_EXISTING_RESOURCE_TYPE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_BREAK_PERIOD;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_DURATION;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ID;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_PROPERTIES;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_PROPOSED_LEASE_ID;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_RANGE_GET_CONTENT_MD5;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_RENAME_SOURCE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_FS_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_BLOBTYPE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_CLOSE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_CONTINUATION;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_DIRECTORY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_FLUSH;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_MAXRESULTS;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_PAGINATED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RECURSIVE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RESOURCE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RETAIN_UNCOMMITTED_DATA;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND;
+
+/**
+ * AbfsClient interacting with the DFS Endpoint.
+ */
+public class AbfsDfsClient extends AbfsClient {
+
+  public AbfsDfsClient(final URL baseUrl,
+      final SharedKeyCredentials sharedKeyCredentials,
+      final AbfsConfiguration abfsConfiguration,
+      final AccessTokenProvider tokenProvider,
+      final EncryptionContextProvider encryptionContextProvider,
+      final AbfsClientContext abfsClientContext) throws IOException {
+    super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider,
+        encryptionContextProvider, abfsClientContext);
+  }
+
+  public AbfsDfsClient(final URL baseUrl,
+      final SharedKeyCredentials sharedKeyCredentials,
+      final AbfsConfiguration abfsConfiguration,
+      final SASTokenProvider sasTokenProvider,
+      final EncryptionContextProvider encryptionContextProvider,
+      final AbfsClientContext abfsClientContext) throws IOException {
+    super(baseUrl, sharedKeyCredentials, abfsConfiguration, sasTokenProvider,
+        encryptionContextProvider, abfsClientContext);
+  }
+
+  /**
+   * Create request headers for Rest Operation using the default API version.
+   * @return default request headers.
+   */
+  @Override
+  public List<AbfsHttpHeader> createDefaultHeaders() {
+    return this.createDefaultHeaders(getxMsVersion());
+  }
+
+  /**
+   * Create request headers for Rest Operation using the specified API version.
+   * DFS Endpoint API responses are in JSON/Stream format.
+   * @param xMsVersion API version to be used.
+   * @return default request headers.
+   */
+  @Override
+  public List<AbfsHttpHeader> createDefaultHeaders(ApiVersion xMsVersion) {
+    List<AbfsHttpHeader> requestHeaders = createCommonHeaders(xMsVersion);
+    requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON
+        + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM));
+    return requestHeaders;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/filesystem/create">
+   *   Filesystem - Create</a>.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation createFilesystem(TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.CreateFileSystem,
+        HTTP_METHOD_PUT, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/filesystem/set-properties">
+   *   Filesystem - Set Properties</a>.
+   * @param properties list of metadata key-value pairs.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation setFilesystemProperties(final Hashtable<String, String> properties,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final String commaSeparatedProperties;
+    try {
+      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
+    } catch (CharacterCodingException ex) {
+      throw new InvalidAbfsRestOperationException(ex);
+    }
+
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to work around the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+        HTTP_METHOD_PATCH));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, commaSeparatedProperties));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.SetFileSystemProperties,
+        HTTP_METHOD_PUT, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/filesystem/get-properties">
+   *   Filesystem - Get Properties</a>.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   * */
+  @Override
+  public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.GetFileSystemProperties,
+        HTTP_METHOD_HEAD, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/filesystem/delete">
+   *   Filesystem - Delete</a>.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation deleteFilesystem(TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.DeleteFileSystem,
+        HTTP_METHOD_DELETE, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/list">
+   *   Filesystem - List</a>.
+   * List paths and their properties in the current filesystem.
+   * @param relativePath to return only blobs within this directory.
+   * @param recursive to return all blobs in the path, including those in subdirectories.
+   * @param listMaxResults maximum number of blobs to return.
+   * @param continuation marker to specify the continuation token.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation or response parsing fails.
+   */
+  @Override
+  public AbfsRestOperation listPath(final String relativePath,
+      final boolean recursive,
+      final int listMaxResults,
+      final String continuation,
+      TracingContext tracingContext) throws IOException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY,
+        getDirectoryQueryParameter(relativePath));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS,
+        String.valueOf(listMaxResults));
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN,
+        String.valueOf(getAbfsConfiguration().isUpnUsed()));
+    appendSASTokenToQuery(relativePath, SASTokenProvider.LIST_OPERATION,
+        abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.ListPaths,
+        HTTP_METHOD_GET, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/create">
+   *   Path - Create</a>.
+   * Create a path (file or directory) in the current filesystem.
+   * @param path to be created inside the filesystem.
+   * @param isFile to specify if the created path is file or directory.
+   * @param overwrite to specify if the path should be overwritten if it already exists.
+   * @param permissions to specify the permissions of the path.
+   * @param isAppendBlob to specify if the path to be created is an append blob.
+   * @param eTag to specify conditional headers.
+   * @param contextEncryptionAdapter to provide encryption context.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation createPath(final String path,
+      final boolean isFile,
+      final boolean overwrite,
+      final AzureBlobFileSystemStore.Permissions permissions,
+      final boolean isAppendBlob,
+      final String eTag,
+      final ContextEncryptionAdapter contextEncryptionAdapter,
+      final TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    if (isFile) {
+      addEncryptionKeyRequestHeaders(path, requestHeaders, true,
+          contextEncryptionAdapter, tracingContext);
+    }
+    if (!overwrite) {
+      requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR));
+    }
+
+    if (permissions.hasPermission()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS,
+          permissions.getPermission()));
+    }
+
+    if (permissions.hasUmask()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_UMASK,
+          permissions.getUmask()));
+    }
+
+    if (eTag != null && !eTag.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, isFile ? FILE : DIRECTORY);
+    if (isAppendBlob) {
+      abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOBTYPE, APPEND_BLOB_TYPE);
+    }
+
+    String operation = isFile
+        ? SASTokenProvider.CREATE_FILE_OPERATION
+        : SASTokenProvider.CREATE_DIRECTORY_OPERATION;
+    appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.CreatePath,
+        HTTP_METHOD_PUT, url, requestHeaders);
+    try {
+      op.execute(tracingContext);
+    } catch (AzureBlobFileSystemException ex) {
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw ex;
+      }
+      if (!isFile && op.getResult().getStatusCode() == HttpURLConnection.HTTP_CONFLICT) {
+        String existingResource =
+            op.getResult().getResponseHeader(X_MS_EXISTING_RESOURCE_TYPE);
+        if (existingResource != null && existingResource.equals(DIRECTORY)) {
+          return op; //don't throw ex on mkdirs for existing directory
+        }
+      }
+      throw ex;
+    }
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/lease">
+   *   Path - Lease</a>.
+   * Acquire lease on specified path.
+   * @param path on which lease has to be acquired.
+   * @param duration for which lease has to be acquired.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation acquireLease(final String path, final int duration,
+      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()));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.LeasePath,
+        HTTP_METHOD_POST, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/lease">
+   *   Path - Lease</a>.
+   * Renew lease on specified path.
+   * @param path on which lease has to be renewed.
+   * @param leaseId of the lease to be renewed.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation renewLease(final String path, final String leaseId,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RENEW_LEASE_ACTION));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.LeasePath,
+        HTTP_METHOD_POST, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/lease">
+   *   Path - Lease</a>.
+   * Release lease on specified path.
+   * @param path on which lease has to be released.
+   * @param leaseId of the lease to be released.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation releaseLease(final String path, final String leaseId,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RELEASE_LEASE_ACTION));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.LeasePath,
+        HTTP_METHOD_POST, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/lease">
+   *   Path - Lease</a>.
+   * Break lease on specified path.
+   * @param path on which lease has to be broke.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation breakLease(final String path,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, BREAK_LEASE_ACTION));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_BREAK_PERIOD,
+        DEFAULT_LEASE_BREAK_PERIOD));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.LeasePath,
+        HTTP_METHOD_POST, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/create">
+   *   Path - Create</a>.
+   * @param source path to source file
+   * @param destination destination of rename.
+   * @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
+   * @return executed rest operation containing response from server.
+   * @throws IOException if rest operation fails.
+   */
+  @Override
+  public AbfsClientRenameResult renamePath(
+      final String source,
+      final String destination,
+      final String continuation,
+      final TracingContext tracingContext,
+      String sourceEtag,
+      boolean isMetadataIncompleteState,
+      boolean isNamespaceEnabled) throws IOException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final boolean hasEtag = !isEmpty(sourceEtag);
+
+    boolean shouldAttemptRecovery = isRenameResilience() && isNamespaceEnabled;
+    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
+      // fetch the source etag to be used later in recovery
+      try {
+        final AbfsRestOperation srcStatusOp = getPathStatus(source,
+            false, tracingContext, null);
+        if (srcStatusOp.hasResult()) {
+          final AbfsHttpOperation result = srcStatusOp.getResult();
+          sourceEtag = extractEtagHeader(result);
+          // and update the directory status.
+          boolean isDir = checkIsDir(result);
+          shouldAttemptRecovery = !isDir;
+          LOG.debug(
+              "Retrieved etag of source for rename recovery: {}; isDir={}",
+              sourceEtag, isDir);
+        }
+      } catch (AbfsRestOperationException e) {
+        throw new AbfsRestOperationException(e.getStatusCode(),
+            SOURCE_PATH_NOT_FOUND.getErrorCode(),
+            e.getMessage(), e);
+      }
+
+    }
+
+    String encodedRenameSource = urlEncode(
+        FORWARD_SLASH + this.getFileSystem() + source);
+    if (getAuthType() == AuthType.SAS) {
+      final AbfsUriQueryBuilder srcQueryBuilder = new AbfsUriQueryBuilder();
+      appendSASTokenToQuery(source, SASTokenProvider.RENAME_SOURCE_OPERATION,
+          srcQueryBuilder);
+      encodedRenameSource += srcQueryBuilder.toString();
+    }
+
+    LOG.trace("Rename source queryparam added {}", encodedRenameSource);
+    requestHeaders.add(new AbfsHttpHeader(X_MS_RENAME_SOURCE, encodedRenameSource));
+    requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
+    appendSASTokenToQuery(destination,
+        SASTokenProvider.RENAME_DESTINATION_OPERATION, abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(destination,
+        abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = createRenameRestOperation(url, requestHeaders);
+    try {
+      incrementAbfsRenamePath();
+      op.execute(tracingContext);
+      // AbfsClientResult contains the AbfsOperation, If recovery happened or
+      // not, and the incompleteMetaDataState is true or false.
+      // If we successfully rename a path and isMetadataIncompleteState was
+      // true, then rename was recovered, else it didn't, this is why
+      // isMetadataIncompleteState is used for renameRecovery(as the 2nd param).
+      return new AbfsClientRenameResult(op, isMetadataIncompleteState,
+          isMetadataIncompleteState);
+    } catch (AzureBlobFileSystemException e) {
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw e;
+      }
+
+      // ref: HADOOP-18242. Rename failure occurring due to a rare case of
+      // tracking metadata being in incomplete state.
+      if (op.getResult().getStorageErrorCode()
+          .equals(RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode())
+          && !isMetadataIncompleteState) {
+        //Logging
+        ABFS_METADATA_INCOMPLETE_RENAME_FAILURE
+            .info(
+                "Rename Failure attempting to resolve tracking metadata state and retrying.");
+        // rename recovery should be attempted in this case also
+        shouldAttemptRecovery = true;
+        isMetadataIncompleteState = true;
+        String sourceEtagAfterFailure = sourceEtag;
+        if (isEmpty(sourceEtagAfterFailure)) {
+          // Doing a HEAD call resolves the incomplete metadata state and
+          // then we can retry the rename operation.
+          AbfsRestOperation sourceStatusOp = getPathStatus(source, false,
+              tracingContext, null);
+          isMetadataIncompleteState = true;
+          // Extract the sourceEtag, using the status Op, and set it
+          // for future rename recovery.
+          AbfsHttpOperation sourceStatusResult = sourceStatusOp.getResult();
+          sourceEtagAfterFailure = extractEtagHeader(sourceStatusResult);
+        }
+        renamePath(source, destination, continuation, tracingContext,
+            sourceEtagAfterFailure, isMetadataIncompleteState,
+            isNamespaceEnabled);
+      }
+      // if we get out of the condition without a successful rename, then
+      // it isn't metadata incomplete state issue.
+      isMetadataIncompleteState = false;
+
+      // setting default rename recovery success to false
+      boolean etagCheckSucceeded = false;
+      if (shouldAttemptRecovery) {
+        etagCheckSucceeded = renameIdempotencyCheckOp(
+            source,
+            sourceEtag, op, destination, tracingContext);
+      }
+      if (!etagCheckSucceeded) {
+        // idempotency did not return different result
+        // throw back the exception
+        throw e;
+      }
+      return new AbfsClientRenameResult(op, true, isMetadataIncompleteState);
+    }
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update">
+   *   Path - Update</a>.
+   * Uploads data to be appended to a file.
+   * @param path to which data has to be appended.
+   * @param buffer containing data to be appended.
+   * @param reqParams containing parameters for append operation like offset, length etc.
+   * @param cachedSasToken to be used for the authenticating operation.
+   * @param contextEncryptionAdapter to provide encryption context.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation append(final String path,
+      final byte[] buffer,
+      AppendRequestParameters reqParams,
+      final String cachedSasToken,
+      ContextEncryptionAdapter contextEncryptionAdapter,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addEncryptionKeyRequestHeaders(path, requestHeaders, false,
+        contextEncryptionAdapter, tracingContext);
+    if (reqParams.isExpectHeaderEnabled()) {
+      requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE));
+    }
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH));
+    if (reqParams.getLeaseId() != null) {
+      requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId()));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION,
+        Long.toString(reqParams.getPosition()));
+
+    if ((reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_MODE) || (
+        reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_CLOSE_MODE)) {
+      abfsUriQueryBuilder.addQuery(QUERY_PARAM_FLUSH, TRUE);
+      if (reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_CLOSE_MODE) {
+        abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, TRUE);
+      }
+    }
+
+    // Check if the retry is with "Expect: 100-continue" header being present in the previous request.
+    if (reqParams.isRetryDueToExpect()) {
+      String userAgentRetry = getUserAgent();
+      // Remove the specific marker related to "Expect: 100-continue" from the User-Agent string.
+      userAgentRetry = userAgentRetry.replace(HUNDRED_CONTINUE_USER_AGENT, EMPTY_STRING);
+      requestHeaders.removeIf(header -> header.getName().equalsIgnoreCase(USER_AGENT));
+      requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry));
+    }
+
+    // Add MD5 Hash of request content as request header if feature is enabled
+    if (isChecksumValidationEnabled()) {
+      addCheckSumHeaderForWrite(requestHeaders, reqParams, buffer);
+    }
+
+    // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
+    String sasTokenForReuse = appendSASTokenToQuery(path,
+        SASTokenProvider.WRITE_OPERATION,
+        abfsUriQueryBuilder, cachedSasToken);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.Append,
+        HTTP_METHOD_PUT, url, requestHeaders,
+        buffer, reqParams.getoffset(), reqParams.getLength(),
+        sasTokenForReuse);
+    try {
+      op.execute(tracingContext);
+    } catch (AbfsRestOperationException e) {
+      /*
+         If the http response code indicates a user error we retry
+         the same append request with expect header being disabled.
+         When "100-continue" header is enabled but a non Http 100 response comes,
+         the response message might not get set correctly by the server.
+         So, this handling is to avoid breaking of backward compatibility
+         if someone has taken dependency on the exception message,
+         which is created using the error string present in the response header.
+      */
+      int responseStatusCode = e.getStatusCode();
+      if (checkUserError(responseStatusCode)
+          && reqParams.isExpectHeaderEnabled()) {
+        LOG.debug(
+            "User error, retrying without 100 continue enabled for the given path {}",
+            path);
+        reqParams.setExpectHeaderEnabled(false);
+        reqParams.setRetryDueToExpect(true);
+        return this.append(path, buffer, reqParams, cachedSasToken,
+            contextEncryptionAdapter, tracingContext);
+      }
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw e;
+      }
+
+      if (isMd5ChecksumError(e)) {
+        throw new AbfsInvalidChecksumException(e);
+      }
+
+      if (reqParams.isAppendBlob()
+          && appendSuccessCheckOp(op, path,
+          (reqParams.getPosition() + reqParams.getLength()), tracingContext)) {
+        final AbfsRestOperation successOp = getAbfsRestOperation(
+            AbfsRestOperationType.Append,
+            HTTP_METHOD_PUT, url, requestHeaders,
+            buffer, reqParams.getoffset(), reqParams.getLength(),
+            sasTokenForReuse);
+        successOp.hardSetResult(HttpURLConnection.HTTP_OK);
+        return successOp;
+      }
+      throw e;
+    } catch (AzureBlobFileSystemException e) {
+      // Any server side issue will be returned as AbfsRestOperationException and will be handled above.
+      LOG.debug(
+          "Append request failed with non server issues for path: {}, offset: {}, position: {}",
+          path, reqParams.getoffset(), reqParams.getPosition());
+      throw e;
+    }
+
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update">
+   *   Path - Update</a>.
+   * Flush previously uploaded data to a file.
+   * @param path on which data has to be flushed.
+   * @param position to which data has to be flushed.
+   * @param retainUncommittedData whether to retain uncommitted data after flush.
+   * @param isClose specify if this is the last flush to the file.
+   * @param cachedSasToken to be used for the authenticating operation.
+   * @param leaseId if there is an active lease on the path.
+   * @param contextEncryptionAdapter to provide encryption context.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation flush(final String path,
+      final long position,
+      boolean retainUncommittedData,
+      boolean isClose,
+      final String cachedSasToken,
+      final String leaseId,
+      ContextEncryptionAdapter contextEncryptionAdapter,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addEncryptionKeyRequestHeaders(path, requestHeaders, false,
+        contextEncryptionAdapter, tracingContext);
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH));
+    if (leaseId != null) {
+      requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RETAIN_UNCOMMITTED_DATA,
+        String.valueOf(retainUncommittedData));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose));
+    // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
+    String sasTokenForReuse = appendSASTokenToQuery(path,
+        SASTokenProvider.WRITE_OPERATION,
+        abfsUriQueryBuilder, cachedSasToken);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.Flush,
+        HTTP_METHOD_PUT, url, requestHeaders,
+        sasTokenForReuse);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  @Override
+  public AbfsRestOperation flush(byte[] buffer,
+      final String path,
+      boolean isClose,
+      final String cachedSasToken,
+      final String leaseId,
+      final String eTag,
+      final TracingContext tracingContext) throws AzureBlobFileSystemException {
+    throw new UnsupportedOperationException(
+        "Flush with blockIds not supported on DFS Endpoint");
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update">
+   *   Path - Update</a>.
+   * Set the properties of a file or directory.
+   * @param path on which properties have to be set.
+   * @param properties list of metadata key-value pairs.
+   * @param tracingContext for tracing the server calls.
+   * @param contextEncryptionAdapter to provide encryption context.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation setPathProperties(final String path,
+      final Hashtable<String, String> properties,
+      final TracingContext tracingContext,
+      final ContextEncryptionAdapter contextEncryptionAdapter)
+      throws AzureBlobFileSystemException {
+    final String commaSeparatedProperties;
+    try {
+      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
+    } catch (CharacterCodingException ex) {
+      throw new InvalidAbfsRestOperationException(ex);
+    }
+
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addEncryptionKeyRequestHeaders(path, requestHeaders, false,
+        contextEncryptionAdapter, tracingContext);
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH));
+    requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, commaSeparatedProperties));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_PROPERTIES_ACTION);
+    appendSASTokenToQuery(path, SASTokenProvider.SET_PROPERTIES_OPERATION,
+        abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.SetPathProperties,
+        HTTP_METHOD_PUT, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/get-properties">
+   *   Path - Get Properties</a>.
+   * Get the properties of a file or directory.
+   * @param path of which properties have to be fetched.
+   * @param includeProperties to include user defined properties.
+   * @param tracingContext for tracing the server calls.
+   * @param contextEncryptionAdapter to provide encryption context.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation getPathStatus(final String path,
+      final boolean includeProperties,
+      final TracingContext tracingContext,
+      final ContextEncryptionAdapter contextEncryptionAdapter)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    String operation = SASTokenProvider.GET_PROPERTIES_OPERATION;
+    if (!includeProperties) {
+      // The default action (operation) is implicitly to get properties and this action requires read permission
+      // because it reads user defined properties.  If the action is getStatus or getAclStatus, then
+      // only traversal (execute) permission is required.
+      abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, GET_STATUS);
+      operation = SASTokenProvider.GET_STATUS_OPERATION;
+    } else {
+      addEncryptionKeyRequestHeaders(path, requestHeaders, false,
+          contextEncryptionAdapter, tracingContext);
+    }
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN,
+        String.valueOf(getAbfsConfiguration().isUpnUsed()));
+    appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.GetPathStatus,
+        HTTP_METHOD_HEAD, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/read">
+   *   Path - Read</a>.
+   * Read the contents of the file at specified path
+   * @param path of the file to be read.
+   * @param position in the file from where data has to be read.
+   * @param buffer to store the data read.
+   * @param bufferOffset offset in the buffer to start storing the data.
+   * @param bufferLength length of data to be read.
+   * @param eTag to specify conditional headers.
+   * @param cachedSasToken to be used for the authenticating operation.
+   * @param contextEncryptionAdapter to provide encryption context.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation read(final String path,
+      final long position,
+      final byte[] buffer,
+      final int bufferOffset,
+      final int bufferLength,
+      final String eTag,
+      String cachedSasToken,
+      ContextEncryptionAdapter contextEncryptionAdapter,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addEncryptionKeyRequestHeaders(path, requestHeaders, false,
+        contextEncryptionAdapter, tracingContext);
+    AbfsHttpHeader rangeHeader = new AbfsHttpHeader(RANGE,
+        String.format("bytes=%d-%d", position, position + bufferLength - 1));
+    requestHeaders.add(rangeHeader);
+    requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
+
+    // Add request header to fetch MD5 Hash of data returned by server.
+    if (isChecksumValidationEnabled(requestHeaders, rangeHeader, bufferLength)) {
+      requestHeaders.add(new AbfsHttpHeader(X_MS_RANGE_GET_CONTENT_MD5, TRUE));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
+    String sasTokenForReuse = appendSASTokenToQuery(path,
+        SASTokenProvider.READ_OPERATION,
+        abfsUriQueryBuilder, cachedSasToken);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.ReadFile,
+        HTTP_METHOD_GET, url, requestHeaders,
+        buffer, bufferOffset, bufferLength,
+        sasTokenForReuse);
+    op.execute(tracingContext);
+
+    // Verify the MD5 hash returned by server holds valid on the data received.
+    if (isChecksumValidationEnabled(requestHeaders, rangeHeader, bufferLength)) {
+      verifyCheckSumForRead(buffer, op.getResult(), bufferOffset);
+    }
+
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/delete">
+   *   Path - Delete</a>.
+   * 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.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation deletePath(final String path,
+      final boolean recursive,
+      final String continuation,
+      TracingContext tracingContext,
+      final boolean isNamespaceEnabled) throws AzureBlobFileSystemException {
+    /*
+     * If Pagination is enabled and current API version is old,
+     * use the minimum required version for pagination.
+     * If Pagination is enabled and current API version is later than minimum required
+     * version for pagination, use current version only as azure service is backward compatible.
+     * If pagination is disabled, use the current API version only.
+     */
+    final List<AbfsHttpHeader> requestHeaders = (isPaginatedDelete(recursive,
+        isNamespaceEnabled) && getxMsVersion().compareTo(
+        ApiVersion.AUG_03_2023) < 0)
+        ? createDefaultHeaders(ApiVersion.AUG_03_2023)
+        : createDefaultHeaders();
+    final AbfsUriQueryBuilder abfsUriQueryBuilder
+        = createDefaultUriQueryBuilder();
+
+    if (isPaginatedDelete(recursive, isNamespaceEnabled)) {
+      // Add paginated query parameter
+      abfsUriQueryBuilder.addQuery(QUERY_PARAM_PAGINATED, TRUE);
+    }
+
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE,
+        String.valueOf(recursive));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
+    String operation = recursive
+        ? SASTokenProvider.DELETE_RECURSIVE_OPERATION
+        : SASTokenProvider.DELETE_OPERATION;
+    appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+        AbfsRestOperationType.DeletePath, this,
+        HTTP_METHOD_DELETE, url, requestHeaders, getAbfsConfiguration());
+    try {
+      op.execute(tracingContext);
+    } catch (AzureBlobFileSystemException e) {
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw e;
+      }
+      final AbfsRestOperation idempotencyOp = deleteIdempotencyCheckOp(op);
+      if (idempotencyOp.getResult().getStatusCode()
+          == op.getResult().getStatusCode()) {
+        // idempotency did not return different result
+        // throw back the exception
+        throw e;
+      } else {
+        return idempotencyOp;
+      }
+    }
+
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update">
+   *   Path - Update</a>.
+   * @param path on which owner has to be set.
+   * @param owner to be set.
+   * @param group to be set.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation setOwner(final String path,
+      final String owner,
+      final String group,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH));
+    if (owner != null && !owner.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_OWNER, owner));
+    }
+    if (group != null && !group.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_GROUP, group));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_ACCESS_CONTROL);
+    appendSASTokenToQuery(path, SASTokenProvider.SET_OWNER_OPERATION,
+        abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.SetOwner,
+        HTTP_METHOD_PUT, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update">
+   *   Path - Update</a>.
+   * @param path on which permission has to be set.
+   * @param permission to be set.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation setPermission(final String path,
+      final String permission,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH));
+    requestHeaders.add(new AbfsHttpHeader(
+        HttpHeaderConfigurations.X_MS_PERMISSIONS, permission));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_ACCESS_CONTROL);
+    appendSASTokenToQuery(path, SASTokenProvider.SET_PERMISSION_OPERATION,
+        abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.SetPermissions,
+        HTTP_METHOD_PUT, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update">
+   *   Path - Update</a>.
+   * @param path on which ACL has to be set.
+   * @param aclSpecString to be set.
+   * @param eTag to specify conditional headers. Set only if etag matches.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation setAcl(final String path,
+      final String aclSpecString,
+      final String eTag,
+      TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+        HTTP_METHOD_PATCH));
+    requestHeaders.add(
+        new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ACL, aclSpecString));
+    if (eTag != null && !eTag.isEmpty()) {
+      requestHeaders.add(
+          new AbfsHttpHeader(IF_MATCH, eTag));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder
+        = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION,
+        SET_ACCESS_CONTROL);
+    appendSASTokenToQuery(path, SASTokenProvider.SET_ACL_OPERATION,
+        abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.SetAcl,
+        HTTP_METHOD_PUT, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/get-properties">
+   *   Path - Get Properties</a>.
+   * Retrieves the ACL properties of blob at specified path.
+   * @param path of which properties have to be fetched.
+   * @param useUPN whether to use UPN with rest operation.
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation getAclStatus(final String path,
+      final boolean useUPN,
+      TracingContext tracingContext) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, GET_ACCESS_CONTROL);
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN,
+        String.valueOf(useUPN));
+    appendSASTokenToQuery(path, SASTokenProvider.GET_ACL_OPERATION,
+        abfsUriQueryBuilder);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.GetAcl,
+        HTTP_METHOD_HEAD, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Get Rest Operation for API
+   * <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/get-properties">
+   *   Path - Get Properties</a>.
+   * @param path Path for which access check needs to be performed
+   * @param rwx The permission to be checked on the path
+   * @param tracingContext for tracing the server calls.
+   * @return executed rest operation containing response from server.
+   * @throws AzureBlobFileSystemException if rest operation fails.
+   */
+  @Override
+  public AbfsRestOperation checkAccess(String path,
+      String rwx,
+      TracingContext tracingContext)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, CHECK_ACCESS);
+    abfsUriQueryBuilder.addQuery(QUERY_FS_ACTION, rwx);
+    appendSASTokenToQuery(path, SASTokenProvider.CHECK_ACCESS_OPERATION,
+        abfsUriQueryBuilder);
+
+    URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    AbfsRestOperation op = getAbfsRestOperation(
+        AbfsRestOperationType.CheckAccess,
+        HTTP_METHOD_HEAD, url, requestHeaders);
+    op.execute(tracingContext);
+    return op;
+  }
+
+  /**
+   * Checks if the rest operation results indicate if the path is a directory.
+   * @param result executed rest operation containing response from server.
+   * @return True if the path is a directory, False otherwise.
+   */
+  @Override
+  public boolean checkIsDir(AbfsHttpOperation result) {
+    String resourceType = result.getResponseHeader(
+        HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+    return StringUtils.equalsIgnoreCase(resourceType, DIRECTORY);
+  }
+
+  /**
+   * Returns true if the status code lies in the range of user error.
+   * @param responseStatusCode http response status code.
+   * @return True or False.
+   */
+  @Override
+  public boolean checkUserError(int responseStatusCode) {
+    return (responseStatusCode >= HttpURLConnection.HTTP_BAD_REQUEST
+        && responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR);
+  }
+
+  private String convertXmsPropertiesToCommaSeparatedString(final Map<String,
+        String> properties) throws CharacterCodingException {
+    StringBuilder commaSeparatedProperties = new StringBuilder();
+
+    final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING_ASCII).newEncoder();
+
+    for (Map.Entry<String, String> propertyEntry : properties.entrySet()) {
+      String key = propertyEntry.getKey();
+      String value = propertyEntry.getValue();
+
+      Boolean canEncodeValue = encoder.canEncode(value);
+      if (!canEncodeValue) {
+        throw new CharacterCodingException();
+      }
+
+      String encodedPropertyValue = Base64.encode(encoder.encode(CharBuffer.wrap(value)).array());
+      commaSeparatedProperties.append(key)
+          .append(AbfsHttpConstants.EQUAL)
+          .append(encodedPropertyValue);
+
+      commaSeparatedProperties.append(AbfsHttpConstants.COMMA);
+    }
+
+    if (commaSeparatedProperties.length() != 0) {
+      commaSeparatedProperties.deleteCharAt(commaSeparatedProperties.length() - 1);
+    }
+
+    return commaSeparatedProperties.toString();
+  }
+}

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

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.azurebfs.utils;
 
 import java.io.UnsupportedEncodingException;
+import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.URLEncoder;
 import java.nio.charset.StandardCharsets;
@@ -30,11 +31,14 @@ import java.util.Set;
 import java.util.regex.Pattern;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.AND_MARK;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EQUAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DFS_DOMAIN_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_SAOID;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_SIGNATURE;
 import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_SKOID;
@@ -169,6 +173,38 @@ public final class UriUtils {
     return url.toString().replace(queryString, maskedQueryString);
   }
 
+  /**
+   * Changes Blob Endpoint URL to DFS Endpoint URL.
+   * If original url is not Blob Endpoint URL, it will return the original URL.
+   * @param url to be converted.
+   * @return updated URL
+   * @throws InvalidUriException in case of MalformedURLException.
+   */
+  public static URL changeUrlFromBlobToDfs(URL url) throws InvalidUriException {
+    try {
+      url = new URL(url.toString().replace(ABFS_BLOB_DOMAIN_NAME, ABFS_DFS_DOMAIN_NAME));
+    } catch (MalformedURLException ex) {
+      throw new InvalidUriException(url.toString());
+    }
+    return url;
+  }
+
+  /**
+   * Changes DFS Endpoint URL to Blob Endpoint URL.
+   * If original url is not DFS Endpoint URL, it will return the original URL.
+   * @param url to be converted.
+   * @return updated URL
+   * @throws InvalidUriException in case of MalformedURLException.
+   */
+  public static URL changeUrlFromDfsToBlob(URL url) throws InvalidUriException {
+    try {
+      url = new URL(url.toString().replace(ABFS_DFS_DOMAIN_NAME, ABFS_BLOB_DOMAIN_NAME));
+    } catch (MalformedURLException ex) {
+      throw new InvalidUriException(url.toString());
+    }
+    return url;
+  }
+
   private UriUtils() {
   }
 }

+ 82 - 0
hadoop-tools/hadoop-azure/src/site/markdown/fns_blob.md

@@ -0,0 +1,82 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+# ABFS Driver for Namespace Disabled Accounts (FNS: Flat Namespace)
+
+### Note: FNS-BLOB Support is being built and not yet ready for usage.
+
+## Background
+The ABFS driver is recommended to be used only with HNS Enabled ADLS Gen-2 accounts
+for big data analytics because of being more performant and scalable.
+
+However, to enable users of legacy WASB Driver to migrate to ABFS driver without
+needing them to upgrade their general purpose V2 accounts (HNS-Disabled), Support
+for FNS accounts is being added to ABFS driver.
+Refer to [WASB Deprication](./wasb.html) for more details.
+
+## Azure Service Endpoints Used by ABFS Driver
+Azure Services offers two set of endpoints for interacting with storage accounts:
+1. [Azure Blob Storage](https://learn.microsoft.com/en-us/rest/api/storageservices/blob-service-rest-api) referred as Blob Endpoint
+2. [Azure Data Lake Storage](https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/operation-groups) referred as DFS Endpoint
+
+The ABFS Driver by default is designed to work with DFS Endpoint only which primarily
+supports HNS Enabled Accounts only.
+
+To enable ABFS Driver to work with FNS Accounts, Support for Blob Endpoint is being added.
+This is because Azure services do not recommend using DFS Endpoint for FNS Accounts.
+ABFS Driver will only allow FNS Accounts to be accessed using Blob Endpoint.
+HNS Enabled accounts will still use DFS Endpoint which continues to be the
+recommended stack based on performance and feature capabilities.
+
+## Configuring ABFS Driver for FNS Accounts
+Following configurations will be introduced to configure ABFS Driver for FNS Accounts:
+1. Account Type: Must be set to `false` to indicate FNS Account
+    ```xml
+    <property>
+      <name>fs.azure.account.hns.enabled</name>
+      <value>false</value>
+    </property>
+    ```
+
+2. Account Url: It is the URL used to initialize the file system. It is either passed
+directly to file system or configured as default uri using "fs.DefaultFS" configuration.
+In both the cases the URL used must be the blob endpoint url of the account.
+    ```xml
+    <property>
+      <name>fs.defaultFS</name>
+      <value>https://ACCOUNT_NAME.blob.core.windows.net</value>
+    </property>
+    ```
+3. Service Type for FNS Accounts: This will allow an override to choose service
+type specially in cases where any local DNS resolution is set for the account and driver is
+unable to detect the intended endpoint from above configured URL. If this is set
+to blob for HNS Enabled Accounts, FS init will fail with InvalidConfiguration error.
+    ```xml
+   <property>
+        <name>fs.azure.fns.account.service.type</name>
+        <value>BLOB</value>
+    </property>
+    ```
+
+4. Service Type for Ingress Operations: This will allow an override to choose service
+type only for Ingress Related Operations like [Create](https://learn.microsoft.com/en-us/rest/api/storageservices/put-blob?tabs=microsoft-entra-id),
+[Append](https://learn.microsoft.com/en-us/rest/api/storageservices/put-block?tabs=microsoft-entra-id)
+and [Flush](https://learn.microsoft.com/en-us/rest/api/storageservices/put-block-list?tabs=microsoft-entra-id). All other operations will still use the
+configured service type.
+    ```xml
+   <property>
+        <name>fs.azure.fns.account.service.type</name>
+        <value>BLOB</value>
+    </property>
+    ```

+ 1 - 0
hadoop-tools/hadoop-azure/src/site/markdown/index.md

@@ -20,6 +20,7 @@ See also:
 
 * [WASB](./wasb.html)
 * [ABFS](./abfs.html)
+* [Namespace Disabled Accounts on ABFS](./fns_blob.html)
 * [Testing](./testing_azure.html)
 
 ## Introduction

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

@@ -339,8 +339,7 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest {
       case SET_ATTR:
         Hashtable<String, String> properties = new Hashtable<>();
         properties.put("key", "{ value: valueTest }");
-        return client.setPathProperties(path, fs.getAbfsStore()
-                .convertXmsPropertiesToCommaSeparatedString(properties),
+        return client.setPathProperties(path, properties,
             getTestTracingContext(fs, false),
             createEncryptionAdapterFromServerStoreContext(path,
                 getTestTracingContext(fs, false), client));

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

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.lang.reflect.Field;
 import java.util.List;
 
+import org.apache.hadoop.fs.azurebfs.enums.Trilean;
 import org.apache.hadoop.util.Lists;
 import org.junit.Assume;
 import org.junit.Test;
@@ -97,8 +98,12 @@ public class ITestAzureBlobFileSystemCheckAccess
         + getAccountName(), ClientCredsTokenProvider.class.getName());
     conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
         false);
-    conf.unset(FS_AZURE_ACCOUNT_IS_HNS_ENABLED);
+    // Since FS init now needs to know account type setting it before init to avoid that.
+    conf.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isHNSEnabled);
     this.testUserFs = FileSystem.newInstance(conf);
+    // Resetting the namespace enabled flag to unknown after file system init.
+    ((AzureBlobFileSystem) testUserFs).getAbfsStore().setNamespaceEnabled(
+        Trilean.UNKNOWN);
   }
 
   private void setTestFsConf(final String fsConfKey,
@@ -306,11 +311,11 @@ public class ITestAzureBlobFileSystemCheckAccess
   }
 
   private void checkPrerequisites() throws Exception {
-    setTestUserFs();
     Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
         isHNSEnabled);
     Assume.assumeTrue(FS_AZURE_ENABLE_CHECK_ACCESS + " is false",
         isCheckAccessEnabled);
+    setTestUserFs();
     checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID);
     checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET);
     checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID);

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

@@ -20,19 +20,32 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.io.FileNotFoundException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+
 import org.junit.Test;
 import org.mockito.Mockito;
 
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException;
 import org.apache.hadoop.fs.azurebfs.enums.Trilean;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 
+import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_HNS_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DFS_DOMAIN_NAME;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.mockito.ArgumentMatchers.any;
+
 /**
  * Test filesystem initialization and creation.
  */
@@ -73,11 +86,11 @@ public class ITestAzureBlobFileSystemInitAndCreate extends
     TracingContext tracingContext = getSampleTracingContext(fs, true);
     Mockito.doReturn(Mockito.mock(AbfsRestOperation.class))
         .when(client)
-        .getAclStatus(Mockito.anyString(), Mockito.any(TracingContext.class));
+        .getAclStatus(Mockito.anyString(), any(TracingContext.class));
     store.getIsNamespaceEnabled(tracingContext);
 
     Mockito.verify(client, Mockito.times(1))
-        .getAclStatus(Mockito.anyString(), Mockito.any(TracingContext.class));
+        .getAclStatus(Mockito.anyString(), any(TracingContext.class));
   }
 
   @Test
@@ -96,6 +109,31 @@ public class ITestAzureBlobFileSystemInitAndCreate extends
     store.getIsNamespaceEnabled(tracingContext);
 
     Mockito.verify(client, Mockito.times(0))
-        .getAclStatus(Mockito.anyString(), Mockito.any(TracingContext.class));
+        .getAclStatus(Mockito.anyString(), any(TracingContext.class));
+  }
+
+  // Todo: [FnsOverBlob] Remove this test case once Blob Endpoint Support is ready and enabled.
+  @Test
+  public void testFileSystemInitFailsWithBlobEndpoitUrl() throws Exception {
+    Configuration configuration = getRawConfiguration();
+    String defaultUri = configuration.get(FS_DEFAULT_NAME_KEY);
+    String blobUri = defaultUri.replace(ABFS_DFS_DOMAIN_NAME, ABFS_BLOB_DOMAIN_NAME);
+    intercept(InvalidConfigurationValueException.class,
+        "Blob Endpoint Support not yet available", () ->
+            FileSystem.newInstance(new Path(blobUri).toUri(), configuration));
+  }
+
+  @Test
+  public void testFileSystemInitFailsIfNotAbleToDetermineAccountType() throws Exception {
+    AzureBlobFileSystem fs = ((AzureBlobFileSystem) FileSystem.newInstance(
+        getRawConfiguration()));
+    AzureBlobFileSystem mockedFs = Mockito.spy(fs);
+    Mockito.doThrow(
+        new AbfsRestOperationException(HTTP_UNAVAILABLE, "Throttled",
+            "Throttled", null)).when(mockedFs).getIsNamespaceEnabled(any());
+
+    intercept(AzureBlobFileSystemException.class,
+        FS_AZURE_ACCOUNT_IS_HNS_ENABLED, () ->
+            mockedFs.initialize(fs.getUri(), getRawConfiguration()));
   }
 }

+ 10 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -140,12 +141,15 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
 
   @Test
   public void testFailedRequestWhenFSNotExist() throws Exception {
+    assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT);
     AbfsConfiguration config = this.getConfiguration();
     config.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, false);
     String testUri = this.getTestUrl();
     String nonExistingFsUrl = getAbfsScheme() + "://" + UUID.randomUUID()
             + testUri.substring(testUri.indexOf("@"));
+    config.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isUsingXNSAccount);
     AzureBlobFileSystem fs = this.getFileSystem(nonExistingFsUrl);
+    fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
 
     intercept(FileNotFoundException.class,
             "\"The specified filesystem does not exist.\", 404",
@@ -214,12 +218,14 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
 
   private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient()
       throws IOException {
-    final AzureBlobFileSystem abfs = this.getFileSystem();
-    final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore();
-    final AbfsClient mockClient = mock(AbfsClient.class);
+    final AzureBlobFileSystem abfs = Mockito.spy(this.getFileSystem());
+    final AzureBlobFileSystemStore abfsStore = Mockito.spy(abfs.getAbfsStore());
+    final AbfsClient mockClient = mock(AbfsDfsClient.class);
+    doReturn(abfsStore).when(abfs).getAbfsStore();
+    doReturn(mockClient).when(abfsStore).getClient();
+    doReturn(mockClient).when(abfsStore).getClient(any());
     doReturn(mock(AbfsRestOperation.class)).when(mockClient)
         .getAclStatus(anyString(), any(TracingContext.class));
-    abfsStore.setClient(mockClient);
     getIsNamespaceEnabled(abfs);
     return mockClient;
   }

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

@@ -160,7 +160,8 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
       boolean includeSSLProvider) throws IOException, URISyntaxException {
     AbfsCounters abfsCounters = Mockito.spy(new AbfsCountersImpl(new URI("abcd")));
     AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters).build();
-    AbfsClient client = new AbfsClient(new URL("https://azure.com"), null,
+    // Todo : [FnsOverBlob] Update to work with Blob Endpoint as well when Fns Over Blob is ready.
+    AbfsClient client = new AbfsDfsClient(new URL("https://azure.com"), null,
         config, (AccessTokenProvider) null, null, abfsClientContext);
     String sslProviderName = null;
     if (includeSSLProvider) {
@@ -363,7 +364,8 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
                                 .build();
 
     // Create test AbfsClient
-    AbfsClient testClient = new AbfsClient(
+    // Todo : [FnsOverBlob] Update to work with Blob Endpoint as well when Fns Over Blob is ready.
+    AbfsClient testClient = new AbfsDfsClient(
         baseAbfsClientInstance.getBaseUrl(),
         (currentAuthType == AuthType.SharedKey
             ? new SharedKeyCredentials(
@@ -391,7 +393,8 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
         (currentAuthType == AuthType.SharedKey)
         || (currentAuthType == AuthType.OAuth));
 
-    AbfsClient client = mock(AbfsClient.class);
+    // Todo : [FnsOverBlob] Update to work with Blob Endpoint as well when Fns Over Blob is ready.
+    AbfsClient client = mock(AbfsDfsClient.class);
     AbfsPerfTracker tracker = new AbfsPerfTracker(
         "test",
         abfsConfig.getAccountName(),

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác