Browse Source

HADOOP-19595. ABFS: AbfsConfiguration should store account type information (HNS or FNS) (#7765)

Contributed by Manish Bhatt
Manish Bhatt 2 days ago
parent
commit
11cbda4f55

+ 23 - 4
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java

@@ -95,6 +95,7 @@ public class AbfsConfiguration{
   private final AbfsServiceType fsConfiguredServiceType;
   private final boolean isSecure;
   private static final Logger LOG = LoggerFactory.getLogger(AbfsConfiguration.class);
+  private Trilean isNamespaceEnabled = null;
 
   @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_IS_HNS_ENABLED,
       DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED)
@@ -525,8 +526,11 @@ public class AbfsConfiguration{
    * @return TRUE/FALSE value if configured, UNKNOWN if not configured.
    */
   public Trilean getIsNamespaceEnabledAccount() {
-    return Trilean.getTrilean(
-        getString(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isNamespaceEnabledAccount));
+    if (isNamespaceEnabled == null) {
+      isNamespaceEnabled = Trilean.getTrilean(
+          getString(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isNamespaceEnabledAccount));
+    }
+    return isNamespaceEnabled;
   }
 
   /**
@@ -1524,9 +1528,24 @@ public class AbfsConfiguration{
     this.maxBackoffInterval = maxBackoffInterval;
   }
 
+  /**
+   * Sets the namespace enabled account flag.
+   *
+   * @param isNamespaceEnabledAccount boolean value indicating if the account is namespace enabled.
+   */
+  void setIsNamespaceEnabledAccount(boolean isNamespaceEnabledAccount) {
+    this.isNamespaceEnabled = Trilean.getTrilean(isNamespaceEnabledAccount);
+  }
+
+  /**
+   * Sets the namespace enabled account flag for testing purposes.
+   * Use this method only for testing scenarios.
+   *
+   * @param isNamespaceEnabledAccount Trilean value indicating if the account is namespace enabled.
+   */
   @VisibleForTesting
-  void setIsNamespaceEnabledAccount(String isNamespaceEnabledAccount) {
-    this.isNamespaceEnabledAccount = isNamespaceEnabledAccount;
+  void setIsNamespaceEnabledAccountForTesting(Trilean isNamespaceEnabledAccount) {
+    this.isNamespaceEnabled = isNamespaceEnabledAccount;
   }
 
   /**

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

@@ -229,6 +229,8 @@ public class AzureBlobFileSystem extends FileSystem
      * HNS Account Cannot have Blob Endpoint URI.
      */
     try {
+      // This will update namespaceEnable based on getAcl in case config is not set.
+      // This Information will be stored in abfsConfiguration class.
       abfsConfiguration.validateConfiguredServiceType(
           tryGetIsNamespaceEnabled(initFSTracingContext));
     } catch (InvalidConfigurationValueException ex) {
@@ -296,7 +298,6 @@ public class AzureBlobFileSystem extends FileSystem
         }
       }
     }
-    getAbfsStore().updateClientWithNamespaceInfo(new TracingContext(initFSTracingContext));
 
     LOG.trace("Initiate check for delegation token manager");
     if (UserGroupInformation.isSecurityEnabled()) {

+ 21 - 27
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java

@@ -186,7 +186,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
 
   private final AbfsConfiguration abfsConfiguration;
   private Set<String> azureInfiniteLeaseDirSet;
-  private volatile Trilean isNamespaceEnabled;
   private final AuthType authType;
   private final UserGroupInformation userGroupInformation;
   private final IdentityTransformerInterface identityTransformer;
@@ -234,8 +233,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
 
     LOG.trace("AbfsConfiguration init complete");
 
-    this.isNamespaceEnabled = abfsConfiguration.getIsNamespaceEnabledAccount();
-
     this.userGroupInformation = UserGroupInformation.getCurrentUser();
     this.userName = userGroupInformation.getShortUserName();
     LOG.trace("UGI init complete");
@@ -287,18 +284,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
         "abfs-bounded");
   }
 
-  /**
-   * Updates the client with the namespace information.
-   *
-   * @param tracingContext the tracing context to be used for the operation
-   * @throws AzureBlobFileSystemException if an error occurs while updating the client
-   */
-  public void updateClientWithNamespaceInfo(TracingContext tracingContext)
-      throws AzureBlobFileSystemException {
-    boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
-    AbfsClient.setIsNamespaceEnabled(isNamespaceEnabled);
-  }
-
   /**
    * Checks if the given key in Azure Storage should be stored as a page
    * blob instead of block blob.
@@ -384,12 +369,12 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
   }
 
   /**
-   * Resolves namespace information of the filesystem from the state of {@link #isNamespaceEnabled}.
+   * Resolves namespace information of the filesystem from the state of {@link #isNamespaceEnabled()}.
    * if the state is UNKNOWN, it will be determined by making a GET_ACL request
    * to the root of the filesystem. GET_ACL call is synchronized to ensure a single
    * call is made to determine the namespace information in case multiple threads are
    * calling this method at the same time. The resolution of namespace information
-   * would be stored back as state of {@link #isNamespaceEnabled}.
+   * would be stored back as {@link #setNamespaceEnabled(boolean)}.
    *
    * @param tracingContext tracing context
    * @return true if namespace is enabled, false otherwise.
@@ -407,22 +392,32 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     return getNamespaceEnabledInformationFromServer(tracingContext);
   }
 
+  /**
+   * In case the namespace configuration is not set or invalid, this method will
+   * make a call to the server to determine if namespace is enabled or not.
+   * This method is synchronized to ensure that only one thread
+   * is making the call to the server to determine the namespace
+   *
+   * @param tracingContext tracing context
+   * @return true if namespace is enabled, false otherwise.
+   * @throws AzureBlobFileSystemException server errors.
+   */
   private synchronized boolean getNamespaceEnabledInformationFromServer(
       final TracingContext tracingContext) throws AzureBlobFileSystemException {
-    if (isNamespaceEnabled != Trilean.UNKNOWN) {
-      return isNamespaceEnabled.toBoolean();
+    if (getAbfsConfiguration().getIsNamespaceEnabledAccount() != Trilean.UNKNOWN) {
+      return isNamespaceEnabled();
     }
     try {
       LOG.debug("Get root ACL status");
       getClient(AbfsServiceType.DFS).getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext);
       // If getAcl succeeds, namespace is enabled.
-      isNamespaceEnabled = Trilean.getTrilean(true);
+      setNamespaceEnabled(true);
     } catch (AbfsRestOperationException ex) {
       // Get ACL status is a HEAD request, its response doesn't contain errorCode
       // So can only rely on its status code to determine account type.
       if (HttpURLConnection.HTTP_BAD_REQUEST != ex.getStatusCode()) {
         // If getAcl fails with anything other than 400, namespace is enabled.
-        isNamespaceEnabled = Trilean.getTrilean(true);
+        setNamespaceEnabled(true);
         // Continue to throw exception as earlier.
         LOG.debug("Failed to get ACL status with non 400. Inferring namespace enabled", ex);
         throw ex;
@@ -430,11 +425,11 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       // If getAcl fails with 400, namespace is disabled.
       LOG.debug("Failed to get ACL status with 400. "
           + "Inferring namespace disabled and ignoring error", ex);
-      isNamespaceEnabled = Trilean.getTrilean(false);
+      setNamespaceEnabled(false);
     } catch (AzureBlobFileSystemException ex) {
       throw ex;
     }
-    return isNamespaceEnabled.toBoolean();
+    return isNamespaceEnabled();
   }
 
   /**
@@ -443,7 +438,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
    */
   @VisibleForTesting
   boolean isNamespaceEnabled() throws TrileanConversionException {
-    return this.isNamespaceEnabled.toBoolean();
+    return getAbfsConfiguration().getIsNamespaceEnabledAccount().toBoolean();
   }
 
   @VisibleForTesting
@@ -2026,9 +2021,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
     return blockFactory;
   }
 
-  @VisibleForTesting
-  void setNamespaceEnabled(Trilean isNamespaceEnabled){
-    this.isNamespaceEnabled = isNamespaceEnabled;
+  void setNamespaceEnabled(boolean isNamespaceEnabled){
+    getAbfsConfiguration().setIsNamespaceEnabledAccount(isNamespaceEnabled);
   }
 
   @VisibleForTesting

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

@@ -66,9 +66,11 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumExc
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
@@ -196,7 +198,6 @@ public abstract class AbfsClient implements Closeable {
   private KeepAliveCache keepAliveCache;
 
   private AbfsApacheHttpClient abfsApacheHttpClient;
-  private static boolean isNamespaceEnabled = false;
 
   /**
    * logging the rename failure if metadata is in an incomplete state.
@@ -442,7 +443,7 @@ public abstract class AbfsClient implements Closeable {
     requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion.toString()));
     requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET, UTF_8));
     requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, EMPTY_STRING));
-    requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgent));
+    requestHeaders.add(new AbfsHttpHeader(USER_AGENT, getUserAgent()));
     return requestHeaders;
   }
 
@@ -1322,8 +1323,9 @@ public abstract class AbfsClient implements Closeable {
     sb.append(abfsConfiguration.getClusterType());
 
     // Add a unique identifier in FNS-Blob user agent string
-    if (!getIsNamespaceEnabled()
-        && abfsConfiguration.getFsConfiguredServiceType() == BLOB) {
+    // Current filesystem init restricts HNS-Blob combination
+    // so namespace check not required.
+    if (abfsConfiguration.getFsConfiguredServiceType() == BLOB) {
       sb.append(SEMICOLON)
           .append(SINGLE_WHITE_SPACE)
           .append(FNS_BLOB_USER_AGENT_IDENTIFIER);
@@ -1724,20 +1726,20 @@ public abstract class AbfsClient implements Closeable {
 
   /**
    * Checks if the namespace is enabled.
+   * Filesystem init will fail if namespace is not correctly configured,
+   * so instead of swallowing the exception, we should throw the exception
+   * in case namespace is not configured correctly.
    *
    * @return True if the namespace is enabled, false otherwise.
+   * @throws AzureBlobFileSystemException if the conversion fails.
    */
-  public static boolean getIsNamespaceEnabled() {
-    return isNamespaceEnabled;
-  }
-
-  /**
-   * Sets the namespace enabled status.
-   *
-   * @param namespaceEnabled True to enable the namespace, false to disable it.
-   */
-  public static void setIsNamespaceEnabled(final boolean namespaceEnabled) {
-    isNamespaceEnabled = namespaceEnabled;
+  public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException {
+    try {
+      return getAbfsConfiguration().getIsNamespaceEnabledAccount().toBoolean();
+    } catch (TrileanConversionException ex) {
+      LOG.error("Failed to convert namespace enabled account property to boolean", ex);
+      throw new InvalidConfigurationValueException("Failed to determine account type", ex);
+    }
   }
 
   protected boolean isRenameResilience() {

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

@@ -1563,9 +1563,11 @@ public class AbfsDfsClient extends AbfsClient {
    * @param requestHeaders list of headers to be sent with the request
    *
    * @return client transaction id
+   * @throws AzureBlobFileSystemException if an error occurs while generating the client transaction id
    */
   @VisibleForTesting
-  public String addClientTransactionIdToHeader(List<AbfsHttpHeader> requestHeaders) {
+  public String addClientTransactionIdToHeader(List<AbfsHttpHeader> requestHeaders)
+      throws AzureBlobFileSystemException {
     String clientTransactionId = null;
     // Set client transaction ID if the namespace and client transaction ID config are enabled.
     if (getIsNamespaceEnabled() && getAbfsConfiguration().getIsClientTransactionIdEnabled()) {

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

@@ -102,8 +102,8 @@ public class ITestAzureBlobFileSystemCheckAccess
     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);
+    ((AzureBlobFileSystem) testUserFs).getAbfsStore()
+        .getAbfsConfiguration().setIsNamespaceEnabledAccountForTesting(Trilean.UNKNOWN);
   }
 
   private void setTestFsConf(final String fsConfKey,

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

@@ -33,7 +33,6 @@ 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;
@@ -83,11 +82,7 @@ public class ITestAzureBlobFileSystemInitAndCreate extends
     AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
     AbfsClient client = Mockito.spy(fs.getAbfsStore().getClient(AbfsServiceType.DFS));
     Mockito.doReturn(client).when(store).getClient(AbfsServiceType.DFS);
-
-    Mockito.doThrow(TrileanConversionException.class)
-        .when(store)
-        .isNamespaceEnabled();
-    store.setNamespaceEnabled(Trilean.UNKNOWN);
+    store.getAbfsConfiguration().setIsNamespaceEnabledAccountForTesting(Trilean.UNKNOWN);
 
     TracingContext tracingContext = getSampleTracingContext(fs, true);
     Mockito.doReturn(Mockito.mock(AbfsRestOperation.class))

+ 142 - 5
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.constants.AbfsServiceType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
@@ -159,7 +160,8 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
             + testUri.substring(testUri.indexOf("@"));
     config.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isUsingXNSAccount);
     AzureBlobFileSystem fs = this.getFileSystem(nonExistingFsUrl);
-    fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
+    fs.getAbfsStore().getAbfsConfiguration()
+        .setIsNamespaceEnabledAccountForTesting(Trilean.UNKNOWN);
 
     FileNotFoundException ex = intercept(FileNotFoundException.class, ()-> {
       fs.getFileStatus(new Path("/")); // Run a dummy FS call
@@ -207,7 +209,8 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
   private void ensureGetAclCallIsMadeOnceForInvalidConf(String invalidConf)
       throws Exception {
     this.getFileSystem().getAbfsStore()
-        .setNamespaceEnabled(Trilean.getTrilean(invalidConf));
+        .getAbfsConfiguration()
+        .setIsNamespaceEnabledAccountForTesting(Trilean.getTrilean(invalidConf));
     AbfsClient mockClient =
         callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
     verify(mockClient, times(1))
@@ -217,7 +220,8 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
   private void ensureGetAclCallIsNeverMadeForValidConf(String validConf)
       throws Exception {
     this.getFileSystem().getAbfsStore()
-        .setNamespaceEnabled(Trilean.getTrilean(validConf));
+        .getAbfsConfiguration()
+        .setIsNamespaceEnabledAccountForTesting(Trilean.getTrilean(validConf));
     AbfsClient mockClient =
         callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
     verify(mockClient, never())
@@ -225,7 +229,8 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
   }
 
   private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException {
-    this.getFileSystem().getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
+    this.getFileSystem().getAbfsStore()
+        .getAbfsConfiguration().setIsNamespaceEnabledAccountForTesting(Trilean.UNKNOWN);
     AbfsClient mockClient =
         callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
     verify(mockClient, times(1))
@@ -262,7 +267,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
       boolean expectedValue, boolean isExceptionExpected) throws Exception {
     AzureBlobFileSystemStore store = Mockito.spy(getFileSystem().getAbfsStore());
     AbfsClient mockClient = mock(AbfsClient.class);
-    store.setNamespaceEnabled(Trilean.UNKNOWN);
+    store.getAbfsConfiguration().setIsNamespaceEnabledAccountForTesting(Trilean.UNKNOWN);
     doReturn(mockClient).when(store).getClient(AbfsServiceType.DFS);
     AbfsRestOperationException ex = new AbfsRestOperationException(
         statusCode, null, Integer.toString(statusCode), null);
@@ -282,6 +287,9 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
     boolean isHnsEnabled = store.getIsNamespaceEnabled(
         getTestTracingContext(getFileSystem(), false));
     Assertions.assertThat(isHnsEnabled).isEqualTo(expectedValue);
+    Assertions.assertThat(store.getClient().getIsNamespaceEnabled())
+        .describedAs("ABFS Client should return same isNameSpace value as store")
+        .isEqualTo(expectedValue);
 
     // GetAcl() should be called only once to determine the HNS status.
     Mockito.verify(mockClient, times(1))
@@ -341,6 +349,135 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
     }
   }
 
+  /**
+   * Tests the behavior of AbfsConfiguration when the namespace-enabled
+   * configuration set based on config provided.
+   *
+   * Expects the namespace value based on config provided.
+   *
+   * @throws Exception if any error occurs during configuration setup or evaluation
+   */
+  @Test
+  public void testNameSpaceConfig() throws Exception {
+    Configuration configuration = getConfigurationWithoutHnsConfig();
+    AzureBlobFileSystem abfs = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
+    AbfsConfiguration abfsConfig = new AbfsConfiguration(configuration, "bogusAccountName");
+
+    // Test that the namespace value when config is not set
+    Assertions.assertThat(abfsConfig.getIsNamespaceEnabledAccount())
+        .describedAs("Namespace enabled should be unknown in case config is not set")
+        .isEqualTo(Trilean.UNKNOWN);
+
+    // In case no namespace config is present, file system init calls getAcl() to determine account type.
+    Assertions.assertThat(abfs.getIsNamespaceEnabled(getTestTracingContext(abfs, false)))
+        .describedAs("getIsNamespaceEnabled should return account type based on getAcl() call")
+        .isEqualTo(abfs.getAbfsClient().getIsNamespaceEnabled());
+
+    // In case no namespace config is present, file system init calls getAcl() to determine account type.
+    Assertions.assertThat(abfs.getAbfsStore().getAbfsConfiguration().getIsNamespaceEnabledAccount())
+        .describedAs("getIsNamespaceEnabled() should return updated account type based on getAcl() call")
+        .isNotEqualTo(Trilean.UNKNOWN);
+
+    configuration.set(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, TRUE_STR);
+    abfs = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
+    abfsConfig = new AbfsConfiguration(configuration, "bogusAccountName");
+
+    // Test that the namespace enabled config is set correctly
+    Assertions.assertThat(abfsConfig.getIsNamespaceEnabledAccount())
+        .describedAs("Namespace enabled should be true in case config is set to true")
+        .isEqualTo(Trilean.TRUE);
+
+    // In case namespace config is present, same value will be return.
+    Assertions.assertThat(abfs.getIsNamespaceEnabled(getTestTracingContext(abfs, false)))
+        .describedAs("getIsNamespaceEnabled() should return true when config is set to true")
+        .isEqualTo(true);
+
+    // In case namespace config is present, same value will be return.
+    Assertions.assertThat(abfs.getAbfsClient().getIsNamespaceEnabled())
+        .describedAs("Client's getIsNamespaceEnabled() should return true when config is set to true")
+        .isEqualTo(true);
+
+    configuration.set(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, FALSE_STR);
+    abfs = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
+    abfsConfig = new AbfsConfiguration(configuration, "bogusAccountName");
+
+    // Test that the namespace enabled config is set correctly
+    Assertions.assertThat(abfsConfig.getIsNamespaceEnabledAccount())
+        .describedAs("Namespace enabled should be false in case config is set to false")
+        .isEqualTo(Trilean.FALSE);
+
+    // In case namespace config is present, same value will be return.
+    Assertions.assertThat(abfs.getIsNamespaceEnabled(getTestTracingContext(abfs, false)))
+        .describedAs("getIsNamespaceEnabled() should return false when config is set to false")
+        .isEqualTo(false);
+
+    // In case namespace config is present, same value will be return.
+    Assertions.assertThat(abfs.getAbfsClient().getIsNamespaceEnabled())
+        .describedAs("Client's getIsNamespaceEnabled() should return false when config is set to false")
+        .isEqualTo(false);
+  }
+
+  /**
+   * Tests to check that the namespace configuration is set correctly
+   * during the initialization of the AzureBlobFileSystem.
+   *
+   *
+   * @throws Exception if any error occurs during configuration setup or evaluation
+   */
+  @Test
+  public void testFsInitShouldSetNamespaceConfig() throws Exception {
+    // Mock the AzureBlobFileSystem and its dependencies
+    AzureBlobFileSystem mockFileSystem = Mockito.spy((AzureBlobFileSystem)
+        FileSystem.newInstance(getConfigurationWithoutHnsConfig()));
+    AzureBlobFileSystemStore mockStore = Mockito.spy(mockFileSystem.getAbfsStore());
+    AbfsClient abfsClient = Mockito.spy(mockStore.getClient());
+    Mockito.doReturn(abfsClient).when(mockStore).getClient();
+    Mockito.doReturn(abfsClient).when(mockStore).getClient(any());
+    abfsClient.getIsNamespaceEnabled();
+    // Verify that getAclStatus is called once during initialization
+    Mockito.verify(abfsClient, times(0))
+        .getAclStatus(anyString(), any(TracingContext.class));
+
+    mockStore.getAbfsConfiguration().setIsNamespaceEnabledAccountForTesting(Trilean.UNKNOWN);
+    // In case someone wrongly configured the namespace in between the processing,
+    // abfsclient.getIsNamespaceEnabled() should throw an exception.
+    String errorMessage = intercept(InvalidConfigurationValueException.class, () -> {
+      abfsClient.getIsNamespaceEnabled();
+    }).getMessage();
+    Assertions.assertThat(errorMessage)
+        .describedAs("Client should throw exception when namespace is unknown")
+        .contains("Failed to determine account type");
+
+    // In case of unknown namespace, store's getIsNamespaceEnabled should call getAclStatus
+    // to determine the namespace status.
+    mockStore.getIsNamespaceEnabled(getTestTracingContext(mockFileSystem, false));
+    Mockito.verify(abfsClient, times(1))
+        .getAclStatus(anyString(), any(TracingContext.class));
+
+    abfsClient.getIsNamespaceEnabled();
+    // Verify that client's getNamespaceEnabled will not call getAclStatus again
+    Mockito.verify(abfsClient, times(1))
+        .getAclStatus(anyString(), any(TracingContext.class));
+  }
+
+  /**
+   * Returns the configuration without the HNS config set.
+   *
+   * @return Configuration without HNS config
+   */
+  private Configuration getConfigurationWithoutHnsConfig() {
+    Configuration rawConfig = new Configuration();
+    rawConfig.addResource(TEST_CONFIGURATION_FILE_NAME);
+    rawConfig.unset(FS_AZURE_ACCOUNT_IS_HNS_ENABLED);
+    rawConfig.unset(accountProperty(FS_AZURE_ACCOUNT_IS_HNS_ENABLED,
+        this.getAccountName()));
+    String testAccountName = "testAccount.dfs.core.windows.net";
+    String defaultUri = this.getTestUrl().replace(this.getAccountName(), testAccountName);
+    // Assert that account specific config takes precedence
+    rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri);
+    return rawConfig;
+  }
+
   private void assertFileSystemInitWithExpectedHNSSettings(
       Configuration configuration, boolean expectedIsHnsEnabledValue) throws IOException {
     try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration)) {

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

@@ -200,7 +200,7 @@ public class TestTracingContext extends AbstractAbfsIntegrationTest {
         0));
 
     // unset namespaceEnabled to call getAcl -> trigger tracing header validator
-    fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
+    fs.getAbfsStore().getAbfsConfiguration().setIsNamespaceEnabledAccountForTesting(Trilean.UNKNOWN);
     fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS);
 
     Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
@@ -208,7 +208,7 @@ public class TestTracingContext extends AbstractAbfsIntegrationTest {
     Assume.assumeTrue(getAuthType() == AuthType.OAuth);
 
     fs.setListenerOperation(FSOperationType.ACCESS);
-    fs.getAbfsStore().setNamespaceEnabled(Trilean.TRUE);
+    fs.getAbfsStore().getAbfsConfiguration().setIsNamespaceEnabledAccountForTesting(Trilean.TRUE);
     fs.access(new Path("/"), FsAction.READ);
   }
 

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

@@ -39,6 +39,7 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.util.functional.FunctionRaisingIOE;
 
@@ -370,7 +371,7 @@ public final class AbfsClientTestUtil {
    * @param clientTransactionId An array to hold the generated transaction ID.
    */
   public static void mockAddClientTransactionIdToHeader(AbfsDfsClient abfsDfsClient,
-      String[] clientTransactionId) {
+      String[] clientTransactionId) throws AzureBlobFileSystemException {
     Mockito.doAnswer(addClientTransactionId -> {
       clientTransactionId[0] = UUID.randomUUID().toString();
       List<AbfsHttpHeader> headers = addClientTransactionId.getArgument(0);

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

@@ -40,7 +40,6 @@ import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
 import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
-import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
@@ -76,11 +75,8 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestAbfsRenameRetryRecovery.class);
 
-  private boolean isNamespaceEnabled;
-
   public TestAbfsRenameRetryRecovery() throws Exception {
-    isNamespaceEnabled = getConfiguration()
-            .getBoolean(TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    // do nothing
   }
 
   /**
@@ -461,10 +457,10 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
    */
   @Test
   public void testRenameRecoveryUnsupportedForFlatNamespace() throws Exception {
-    Assume.assumeTrue(!isNamespaceEnabled);
     // In DFS endpoint, renamePath is O(1) API call and idempotency issue can happen.
     // For blob endpoint, client orchestrates the rename operation.
     assumeDfsServiceType();
+    assumeHnsDisabled();
     AzureBlobFileSystem fs = getFileSystem();
     AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
     TracingContext testTracingContext = getTestTracingContext(fs, false);