Sfoglia il codice sorgente

HADOOP-17002. ABFS: Adding config to determine if the account is HNS enabled or not

Contributed by Bilahari T H.
bilaharith 5 anni fa
parent
commit
30ef8d0f1a

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

@@ -47,6 +47,7 @@ import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValida
 import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
 import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
 import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
 import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
 import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
 import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.enums.Trilean;
 import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee;
 import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee;
 import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
 import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
@@ -81,6 +82,10 @@ public class AbfsConfiguration{
   private final boolean isSecure;
   private final boolean isSecure;
   private static final Logger LOG = LoggerFactory.getLogger(AbfsConfiguration.class);
   private static final Logger LOG = LoggerFactory.getLogger(AbfsConfiguration.class);
 
 
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_IS_HNS_ENABLED,
+      DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED)
+  private String isNamespaceEnabledAccount;
+
   @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_BUFFER_SIZE,
   @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_BUFFER_SIZE,
       MinValue = MIN_BUFFER_SIZE,
       MinValue = MIN_BUFFER_SIZE,
       MaxValue = MAX_BUFFER_SIZE,
       MaxValue = MAX_BUFFER_SIZE,
@@ -232,6 +237,10 @@ public class AbfsConfiguration{
     }
     }
   }
   }
 
 
+  public Trilean getIsNamespaceEnabledAccount() {
+    return Trilean.getTrilean(isNamespaceEnabledAccount);
+  }
+
   /**
   /**
    * Gets the Azure Storage account name corresponding to this instance of configuration.
    * Gets the Azure Storage account name corresponding to this instance of configuration.
    * @return the Azure Storage account name
    * @return the Azure Storage account name
@@ -746,6 +755,11 @@ public class AbfsConfiguration{
     this.maxIoRetries = maxIoRetries;
     this.maxIoRetries = maxIoRetries;
   }
   }
 
 
+  @VisibleForTesting
+  void setIsNamespaceEnabledAccount(String isNamespaceEnabledAccount) {
+    this.isNamespaceEnabledAccount = isNamespaceEnabledAccount;
+  }
+
   private String getTrimmedPasswordString(String key, String defaultValue) throws IOException {
   private String getTrimmedPasswordString(String key, String defaultValue) throws IOException {
     String value = getPasswordString(key);
     String value = getPasswordString(key);
     if (StringUtils.isBlank(value)) {
     if (StringUtils.isBlank(value)) {

+ 40 - 19
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java

@@ -73,6 +73,8 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException;
+import org.apache.hadoop.fs.azurebfs.enums.Trilean;
 import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
 import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
 import org.apache.hadoop.fs.azurebfs.extensions.ExtensionHelper;
 import org.apache.hadoop.fs.azurebfs.extensions.ExtensionHelper;
 import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
@@ -133,8 +135,7 @@ public class AzureBlobFileSystemStore implements Closeable {
 
 
   private final AbfsConfiguration abfsConfiguration;
   private final AbfsConfiguration abfsConfiguration;
   private final Set<String> azureAtomicRenameDirSet;
   private final Set<String> azureAtomicRenameDirSet;
-  private boolean isNamespaceEnabledSet;
-  private boolean isNamespaceEnabled;
+  private Trilean isNamespaceEnabled;
   private final AuthType authType;
   private final AuthType authType;
   private final UserGroupInformation userGroupInformation;
   private final UserGroupInformation userGroupInformation;
   private final IdentityTransformer identityTransformer;
   private final IdentityTransformer identityTransformer;
@@ -155,6 +156,8 @@ public class AzureBlobFileSystemStore implements Closeable {
 
 
     LOG.trace("AbfsConfiguration init complete");
     LOG.trace("AbfsConfiguration init complete");
 
 
+    this.isNamespaceEnabled = abfsConfiguration.getIsNamespaceEnabledAccount();
+
     this.userGroupInformation = UserGroupInformation.getCurrentUser();
     this.userGroupInformation = UserGroupInformation.getCurrentUser();
     this.userName = userGroupInformation.getShortUserName();
     this.userName = userGroupInformation.getShortUserName();
     LOG.trace("UGI init complete");
     LOG.trace("UGI init complete");
@@ -234,26 +237,33 @@ public class AzureBlobFileSystemStore implements Closeable {
   }
   }
 
 
   public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException {
   public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException {
-    if (!isNamespaceEnabledSet) {
+    try {
+      return this.isNamespaceEnabled.toBoolean();
+    } catch (TrileanConversionException e) {
+      LOG.debug("isNamespaceEnabled is UNKNOWN; fall back and determine through"
+          + " getAcl server call", e);
+    }
 
 
-      LOG.debug("Get root ACL status");
-      try (AbfsPerfInfo perfInfo = startTracking("getIsNamespaceEnabled", "getAclStatus")) {
-        AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + AbfsHttpConstants.ROOT_PATH);
-        perfInfo.registerResult(op.getResult());
-        isNamespaceEnabled = true;
-        perfInfo.registerSuccess(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 its account type.
-        if (HttpURLConnection.HTTP_BAD_REQUEST != ex.getStatusCode()) {
-          throw ex;
-        }
-        isNamespaceEnabled = false;
+    LOG.debug("Get root ACL status");
+    try (AbfsPerfInfo perfInfo = startTracking("getIsNamespaceEnabled",
+        "getAclStatus")) {
+      AbfsRestOperation op = client.getAclStatus(
+          AbfsHttpConstants.FORWARD_SLASH + AbfsHttpConstants.ROOT_PATH);
+      perfInfo.registerResult(op.getResult());
+      isNamespaceEnabled = Trilean.getTrilean(true);
+      perfInfo.registerSuccess(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 its account type.
+      if (HttpURLConnection.HTTP_BAD_REQUEST != ex.getStatusCode()) {
+        throw ex;
       }
       }
-      isNamespaceEnabledSet = true;
+
+      isNamespaceEnabled = Trilean.getTrilean(false);
     }
     }
 
 
-    return isNamespaceEnabled;
+    return isNamespaceEnabled.toBoolean();
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
@@ -1406,4 +1416,15 @@ public class AzureBlobFileSystemStore implements Closeable {
   AbfsClient getClient() {
   AbfsClient getClient() {
     return this.client;
     return this.client;
   }
   }
-}
+
+  @VisibleForTesting
+  void setClient(AbfsClient client) {
+    this.client = client;
+  }
+
+  @VisibleForTesting
+  void setNamespaceEnabled(Trilean isNamespaceEnabled){
+    this.isNamespaceEnabled = isNamespaceEnabled;
+  }
+
+}

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

@@ -27,6 +27,13 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public final class ConfigurationKeys {
 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.
+   */
+  public static final String FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "fs.azure.account.hns.enabled";
   public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key";
   public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key";
   public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)";
   public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)";
   public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode";
   public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode";

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

@@ -30,6 +30,9 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_ST
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public final class FileSystemConfigurations {
 public final class FileSystemConfigurations {
+
+  public static final String DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "";
+
   public static final String USER_HOME_DIRECTORY_PREFIX = "/user";
   public static final String USER_HOME_DIRECTORY_PREFIX = "/user";
 
 
   // Retry parameter defaults.
   // Retry parameter defaults.

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

@@ -0,0 +1,36 @@
+/**
+ * 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.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when tried to convert Trilean.UNKNOWN to boolean. Only Trilean.TRUE
+ * and Trilean.FALSE can be converted to boolean.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public final class TrileanConversionException
+    extends AzureBlobFileSystemException {
+  public TrileanConversionException() {
+    super("Cannot convert Trilean.UNKNOWN to boolean");
+  }
+
+}

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

@@ -0,0 +1,80 @@
+/**
+ * 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.enums;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException;
+
+/**
+ * Enum to represent 3 values, TRUE, FALSE and UNKNOWN. Can be used where
+ * boolean is not enough to hold the information.
+ */
+public enum Trilean {
+
+  FALSE, TRUE, UNKNOWN;
+
+  private static final String TRUE_STR = "true";
+  private static final String FALSE_STR = "false";
+
+  /**
+   * Converts boolean to Trilean.
+   *
+   * @param isTrue the boolean to convert.
+   * @return the corresponding Trilean for the passed boolean isTrue.
+   */
+  public static Trilean getTrilean(final boolean isTrue) {
+    if (isTrue) {
+      return Trilean.TRUE;
+    }
+
+    return Trilean.FALSE;
+  }
+
+  /**
+   * Converts String to Trilean.
+   *
+   * @param str the string to convert.
+   * @return the corresponding Trilean for the passed string str.
+   */
+  public static Trilean getTrilean(String str) {
+    if (TRUE_STR.equalsIgnoreCase(str)) {
+      return Trilean.TRUE;
+    }
+
+    if (FALSE_STR.equalsIgnoreCase(str)) {
+      return Trilean.FALSE;
+    }
+
+    return Trilean.UNKNOWN;
+  }
+
+  /**
+   * Converts the Trilean enum to boolean.
+   *
+   * @return the corresponding boolean.
+   * @throws TrileanConversionException when tried to convert Trilean.UNKNOWN.
+   */
+  public boolean toBoolean() throws TrileanConversionException {
+    if (this == Trilean.UNKNOWN) {
+      throw new TrileanConversionException();
+    }
+
+    return Boolean.valueOf(this.name());
+  }
+
+}

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.enums;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 5 - 0
hadoop-tools/hadoop-azure/src/site/markdown/abfs.md

@@ -663,6 +663,11 @@ Hflush() being the only documented API that can provide persistent data
 transfer, Flush() also attempting to persist buffered data will lead to
 transfer, Flush() also attempting to persist buffered data will lead to
 performance issues.
 performance issues.
 
 
+### <a name="hnscheckconfigoptions"></a> HNS Check Options
+Config `fs.azure.account.hns.enabled` provides an option to specify whether
+ the storage account is HNS enabled or not. In case the config is not provided,
+  a server call is made to check the same.
+
 ### <a name="flushconfigoptions"></a> Access Options
 ### <a name="flushconfigoptions"></a> Access Options
 Config `fs.azure.enable.check.access` needs to be set true to enable
 Config `fs.azure.enable.check.access` needs to be set true to enable
  the AzureBlobFileSystem.access().
  the AzureBlobFileSystem.access().

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

@@ -21,15 +21,31 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.UUID;
 import java.util.UUID;
 
 
+import org.apache.hadoop.fs.azurebfs.enums.Trilean;
 import org.junit.Assume;
 import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
+import org.assertj.core.api.Assertions;
 
 
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 
 
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_HNS_ENABLED;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -39,6 +55,9 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
  */
  */
 public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
 public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
 
 
+  private static final String TRUE_STR = "true";
+  private static final String FALSE_STR = "false";
+
   private boolean isUsingXNSAccount;
   private boolean isUsingXNSAccount;
   public ITestGetNameSpaceEnabled() throws Exception {
   public ITestGetNameSpaceEnabled() throws Exception {
     isUsingXNSAccount = getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
     isUsingXNSAccount = getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
@@ -57,7 +76,57 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
     Assume.assumeFalse("Skip this test because the account being used for test is a XNS account",
     Assume.assumeFalse("Skip this test because the account being used for test is a XNS account",
             isUsingXNSAccount);
             isUsingXNSAccount);
     assertFalse("Expecting getIsNamespaceEnabled() return false",
     assertFalse("Expecting getIsNamespaceEnabled() return false",
-            getFileSystem().getIsNamespaceEnabled());
+        getFileSystem().getIsNamespaceEnabled());
+  }
+
+  @Test
+  public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception {
+    AzureBlobFileSystem fs = getNewFSWithHnsConf(TRUE_STR);
+    Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs(
+        "getIsNamespaceEnabled should return true when the "
+            + "config is set as true").isTrue();
+    fs.getAbfsStore().deleteFilesystem();
+    unsetAndAssert();
+  }
+
+  @Test
+  public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception {
+    AzureBlobFileSystem fs = getNewFSWithHnsConf(FALSE_STR);
+    Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs(
+        "getIsNamespaceEnabled should return false when the "
+            + "config is set as false").isFalse();
+    fs.getAbfsStore().deleteFilesystem();
+    unsetAndAssert();
+  }
+
+  private void unsetAndAssert() throws Exception {
+    AzureBlobFileSystem fs = getNewFSWithHnsConf(
+        DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED);
+    boolean expectedValue = this.getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs(
+        "getIsNamespaceEnabled should return the value "
+            + "configured for fs.azure.test.namespace.enabled")
+        .isEqualTo(expectedValue);
+    fs.getAbfsStore().deleteFilesystem();
+  }
+
+  private AzureBlobFileSystem getNewFSWithHnsConf(
+      String isNamespaceEnabledAccount) throws Exception {
+    Configuration rawConfig = new Configuration();
+    rawConfig.addResource(TEST_CONFIGURATION_FILE_NAME);
+    rawConfig.set(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isNamespaceEnabledAccount);
+    rawConfig
+        .setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+        getNonExistingUrl());
+    return (AzureBlobFileSystem) FileSystem.get(rawConfig);
+  }
+
+  private String getNonExistingUrl() {
+    String testUri = this.getTestUrl();
+    return getAbfsScheme() + "://" + UUID.randomUUID() + testUri
+        .substring(testUri.indexOf("@"));
   }
   }
 
 
   @Test
   @Test
@@ -95,4 +164,72 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
               fs.getIsNamespaceEnabled();
               fs.getIsNamespaceEnabled();
             });
             });
   }
   }
-}
+
+  @Test
+  public void testEnsureGetAclCallIsMadeOnceWhenConfigIsInvalid()
+      throws Exception {
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+    ensureGetAclCallIsMadeOnceForInvalidConf(" ");
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+    ensureGetAclCallIsMadeOnceForInvalidConf("Invalid conf");
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+  }
+
+  @Test
+  public void testEnsureGetAclCallIsNeverMadeWhenConfigIsValid()
+      throws Exception {
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+    ensureGetAclCallIsNeverMadeForValidConf(FALSE_STR.toLowerCase());
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+    ensureGetAclCallIsNeverMadeForValidConf(FALSE_STR.toUpperCase());
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+    ensureGetAclCallIsNeverMadeForValidConf(TRUE_STR.toLowerCase());
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+    ensureGetAclCallIsNeverMadeForValidConf(TRUE_STR.toUpperCase());
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+  }
+
+  @Test
+  public void testEnsureGetAclCallIsMadeOnceWhenConfigIsNotPresent()
+      throws IOException {
+    unsetConfAndEnsureGetAclCallIsMadeOnce();
+  }
+
+  private void ensureGetAclCallIsMadeOnceForInvalidConf(String invalidConf)
+      throws Exception {
+    this.getFileSystem().getAbfsStore()
+        .setNamespaceEnabled(Trilean.getTrilean(invalidConf));
+    AbfsClient mockClient =
+        callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
+    verify(mockClient, times(1)).getAclStatus(anyString());
+  }
+
+  private void ensureGetAclCallIsNeverMadeForValidConf(String validConf)
+      throws Exception {
+    this.getFileSystem().getAbfsStore()
+        .setNamespaceEnabled(Trilean.getTrilean(validConf));
+    AbfsClient mockClient =
+        callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
+    verify(mockClient, never()).getAclStatus(anyString());
+  }
+
+  private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException {
+    this.getFileSystem().getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
+    AbfsClient mockClient =
+        callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
+    verify(mockClient, times(1)).getAclStatus(anyString());
+  }
+
+  private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient()
+      throws IOException {
+    final AzureBlobFileSystem abfs = this.getFileSystem();
+    final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore();
+    final AbfsClient mockClient = mock(AbfsClient.class);
+    doReturn(mock(AbfsRestOperation.class)).when(mockClient)
+        .getAclStatus(anyString());
+    abfsStore.setClient(mockClient);
+    abfs.getIsNamespaceEnabled();
+    return mockClient;
+  }
+
+}

+ 92 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TrileanTests.java

@@ -0,0 +1,92 @@
+/**
+ * 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;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException;
+import org.apache.hadoop.fs.azurebfs.enums.Trilean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+
+/**
+ * Tests for the enum Trilean.
+ */
+public class TrileanTests {
+
+  private static final String TRUE_STR = "true";
+  private static final String FALSE_STR = "false";
+
+  @Test
+  public void testGetTrileanForBoolean() {
+    assertThat(Trilean.getTrilean(true)).describedAs(
+        "getTrilean should return Trilean.TRUE when true is passed")
+        .isEqualTo(Trilean.TRUE);
+    assertThat(Trilean.getTrilean(false)).describedAs(
+        "getTrilean should return Trilean.FALSE when false is passed")
+        .isEqualTo(Trilean.FALSE);
+  }
+
+  @Test
+  public void testGetTrileanForString() {
+    assertThat(Trilean.getTrilean(TRUE_STR.toLowerCase())).describedAs(
+        "getTrilean should return Trilean.TRUE when true is passed")
+        .isEqualTo(Trilean.TRUE);
+    assertThat(Trilean.getTrilean(TRUE_STR.toUpperCase())).describedAs(
+        "getTrilean should return Trilean.TRUE when TRUE is passed")
+        .isEqualTo(Trilean.TRUE);
+
+    assertThat(Trilean.getTrilean(FALSE_STR.toLowerCase())).describedAs(
+        "getTrilean should return Trilean.FALSE when false is passed")
+        .isEqualTo(Trilean.FALSE);
+    assertThat(Trilean.getTrilean(FALSE_STR.toUpperCase())).describedAs(
+        "getTrilean should return Trilean.FALSE when FALSE is passed")
+        .isEqualTo(Trilean.FALSE);
+
+    testInvalidString(null);
+    testInvalidString(" ");
+    testInvalidString("invalid");
+    testInvalidString("truee");
+    testInvalidString("falsee");
+  }
+
+  private void testInvalidString(String invalidString) {
+    assertThat(Trilean.getTrilean(invalidString)).describedAs(
+        "getTrilean should return Trilean.UNKNOWN for anything not true/false")
+        .isEqualTo(Trilean.UNKNOWN);
+  }
+
+  @Test
+  public void testToBoolean() throws TrileanConversionException {
+    assertThat(Trilean.TRUE.toBoolean())
+        .describedAs("toBoolean should return true for Trilean.TRUE").isTrue();
+    assertThat(Trilean.FALSE.toBoolean())
+        .describedAs("toBoolean should return false for Trilean.FALSE")
+        .isFalse();
+
+    assertThat(catchThrowable(() -> Trilean.UNKNOWN.toBoolean())).describedAs(
+        "toBoolean on Trilean.UNKNOWN results in TrileanConversionException")
+        .isInstanceOf(TrileanConversionException.class).describedAs(
+        "Exception message should be: catchThrowable(()->Trilean.UNKNOWN"
+            + ".toBoolean())")
+        .hasMessage("Cannot convert Trilean.UNKNOWN to boolean");
+  }
+
+}