diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 50cc57447f92b..6883f1ba7f38e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -84,6 +84,7 @@ public class AbfsConfiguration{ private final String accountName; 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) @@ -332,8 +333,19 @@ public AbfsConfiguration(final Configuration rawConfig, String accountName) } } + /** + * Returns the account type as per the user configuration. Gets the account + * specific value if it exists, then looks for an account agnostic value. + * If not configured driver makes additional getAcl call to determine + * the account type during file system initialization. + * @return TRUE/FALSE value if configured, UNKNOWN if not configured. + */ public Trilean getIsNamespaceEnabledAccount() { - return Trilean.getTrilean(isNamespaceEnabledAccount); + if (isNamespaceEnabled == null) { + isNamespaceEnabled = Trilean.getTrilean( + getString(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isNamespaceEnabledAccount)); + } + return isNamespaceEnabled; } /** @@ -1048,9 +1060,23 @@ void setMaxBackoffIntervalMilliseconds(int maxBackoffInterval) { 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. + * + * @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; } private String getTrimmedPasswordString(String key, String defaultValue) throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 750306c4a983f..03a9129eaa057 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -42,6 +42,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -106,6 +107,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*; 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; @@ -190,6 +192,15 @@ public void initialize(URI uri, Configuration configuration) tracingHeaderFormat = abfsConfiguration.getTracingHeaderFormat(); this.setWorkingDirectory(this.getHomeDirectory()); + TracingContext initFSTracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.INIT, tracingHeaderFormat, listener); + try { + getIsNamespaceEnabled(initFSTracingContext); + } 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); + } + if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index d86a3d9684617..e9a1ef9cd2fb9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -164,7 +164,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private final AbfsConfiguration abfsConfiguration; private final Set azureAtomicRenameDirSet; private Set azureInfiniteLeaseDirSet; - private Trilean isNamespaceEnabled; private final AuthType authType; private final UserGroupInformation userGroupInformation; private final IdentityTransformerInterface identityTransformer; @@ -207,8 +206,6 @@ public AzureBlobFileSystemStore( LOG.trace("AbfsConfiguration init complete"); - this.isNamespaceEnabled = abfsConfiguration.getIsNamespaceEnabledAccount(); - this.userGroupInformation = UserGroupInformation.getCurrentUser(); this.userName = userGroupInformation.getShortUserName(); LOG.trace("UGI init complete"); @@ -342,35 +339,67 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In return authorityParts; } + /** + * 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 {@link #setNamespaceEnabled(boolean)}. + * + * @param tracingContext tracing context + * @return true if namespace is enabled, false otherwise. + * @throws AzureBlobFileSystemException server errors. + */ public boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { try { - return this.isNamespaceEnabled.toBoolean(); + return isNamespaceEnabled(); } 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.ROOT_PATH, tracingContext); - perfInfo.registerResult(op.getResult()); - isNamespaceEnabled = Trilean.getTrilean(true); - perfInfo.registerSuccess(true); + return getNamespaceEnabledInformationFromServer(tracingContext); + } + + private synchronized boolean getNamespaceEnabledInformationFromServer( + final TracingContext tracingContext) throws AzureBlobFileSystemException { + if (abfsConfiguration.getIsNamespaceEnabledAccount() != Trilean.UNKNOWN) { + return isNamespaceEnabled(); + } + try { + LOG.debug("Get root ACL status"); + getClient().getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext); + // If getAcl succeeds, namespace is enabled. + 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 its account type. + // 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. + setNamespaceEnabled(true); + // Continue to throw exception as earlier. + LOG.debug("Failed to get ACL status with non 400. Inferring namespace enabled", ex); throw ex; } - - isNamespaceEnabled = Trilean.getTrilean(false); + // If getAcl fails with 400, namespace is disabled. + LOG.debug("Failed to get ACL status with 400. " + + "Inferring namespace disabled and ignoring error", ex); + setNamespaceEnabled(false); + } catch (AzureBlobFileSystemException ex) { + throw ex; } + return isNamespaceEnabled(); + } - return isNamespaceEnabled.toBoolean(); + /** + * @return true if namespace is enabled, false otherwise. + * @throws TrileanConversionException if namespaceEnabled information is UNKNOWN + */ + @VisibleForTesting + boolean isNamespaceEnabled() throws TrileanConversionException { + return abfsConfiguration.getIsNamespaceEnabledAccount().toBoolean(); } @VisibleForTesting @@ -1869,9 +1898,8 @@ void setClient(AbfsClient client) { this.client = client; } - @VisibleForTesting - void setNamespaceEnabled(Trilean isNamespaceEnabled){ - this.isNamespaceEnabled = isNamespaceEnabled; + void setNamespaceEnabled(boolean isNamespaceEnabled){ + abfsConfiguration.setIsNamespaceEnabledAccount(isNamespaceEnabled); } private void updateInfiniteLeaseDirs() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FSOperationType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FSOperationType.java index 6b6e98c9c7082..8c9c8af75b53d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FSOperationType.java +++ b/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; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 69ef0d01c7823..27125f523ecd7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -37,6 +37,8 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TrileanConversionException; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -1154,4 +1156,21 @@ public ListenableFuture submit(Runnable runnable) { public void addCallback(ListenableFuture future, FutureCallback callback) { Futures.addCallback(future, callback, executorService); } + + /** + * Checks if the namespace is enabled. + * + * @return True if the namespace is enabled, false otherwise. + * * @throws AzureBlobFileSystemException if the conversion fails. + */ + public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { + try { + return abfsConfiguration.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"); + } + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 01227691c3139..25623641a154b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -24,9 +24,11 @@ import org.junit.Assume; import org.junit.Test; import org.assertj.core.api.Assertions; +import org.mockito.Mockito; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.conf.Configuration; @@ -34,6 +36,9 @@ import org.apache.hadoop.fs.azurebfs.enums.Trilean; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.accountProperty; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; @@ -115,6 +120,8 @@ private AzureBlobFileSystem getNewFSWithHnsConf( Configuration rawConfig = new Configuration(); rawConfig.addResource(TEST_CONFIGURATION_FILE_NAME); rawConfig.set(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isNamespaceEnabledAccount); + rawConfig.set(accountProperty(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, + this.getAccountName()), isNamespaceEnabledAccount); rawConfig .setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, @@ -177,17 +184,209 @@ public void testEnsureGetAclCallIsMadeOnceWhenConfigIsNotPresent() 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)) .getAclStatus(anyString(), any(TracingContext.class)); } + + /** + * Tests the behavior of AbfsConfiguration when the namespace-enabled + * configuration is set based on account specific config. + * + * Expects the namespace value based on account specific config provided. + * + * @throws Exception if any error occurs during configuration setup or evaluation + */ + @Test + public void testAccountSpecificConfig() throws Exception { + 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 otherAccountName = "otherAccount.dfs.core.windows.net"; + String dummyAcountKey = "dummyKey"; + String defaultUri = this.getTestUrl().replace(this.getAccountName(), testAccountName); + String otherUri = this.getTestUrl().replace(this.getAccountName(), otherAccountName); + + // Set both account specific and account agnostic config for test account + rawConfig.set(accountProperty(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, testAccountName), FALSE_STR); + rawConfig.set(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, TRUE_STR); + rawConfig.set(accountProperty(FS_AZURE_ACCOUNT_KEY, testAccountName), dummyAcountKey); + rawConfig.set(accountProperty(FS_AZURE_ACCOUNT_KEY, otherAccountName), dummyAcountKey); + // Assert that account specific config takes precedence + rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri); + assertFileSystemInitWithExpectedHNSSettings(rawConfig, false); + // Assert that other account still uses account agnostic config + rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, otherUri); + assertFileSystemInitWithExpectedHNSSettings(rawConfig, true); + + // Set only the account specific config for test account + rawConfig.set(accountProperty(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, testAccountName), FALSE_STR); + rawConfig.unset(FS_AZURE_ACCOUNT_IS_HNS_ENABLED); + // Assert that only account specific config is enough for test account + rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri); + assertFileSystemInitWithExpectedHNSSettings(rawConfig, false); + + // Set only account agnostic config + rawConfig.set(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, FALSE_STR); + rawConfig.unset(accountProperty(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, testAccountName)); + rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri); + assertFileSystemInitWithExpectedHNSSettings(rawConfig, false); + + // Unset both account specific and account agnostic config + rawConfig.unset(FS_AZURE_ACCOUNT_IS_HNS_ENABLED); + rawConfig.unset(accountProperty(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, testAccountName)); + rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri); + rawConfig.set(AZURE_MAX_IO_RETRIES, "0"); + // Assert that file system init fails with UnknownHost exception as getAcl() is needed. + try { + assertFileSystemInitWithExpectedHNSSettings(rawConfig, false); + } catch (Exception e) { + Assertions.assertThat(e.getCause().getMessage()) + .describedAs("getAcl() to determine HNS Nature of account should" + + "fail with Unknown Host Exception").contains("UnknownHostException"); + } + } + + /** + * 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(); + 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 ensureGetAclCallIsNeverMadeForValidConf(String validConf) throws Exception { this.getFileSystem().getAbfsStore() - .setNamespaceEnabled(Trilean.getTrilean(validConf)); + .getAbfsConfiguration() + .setIsNamespaceEnabledAccountForTesting(Trilean.getTrilean(validConf)); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); verify(mockClient, never()) @@ -195,7 +394,9 @@ private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) } 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)) @@ -214,4 +415,15 @@ private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() return mockClient; } + private void assertFileSystemInitWithExpectedHNSSettings( + Configuration configuration, boolean expectedIsHnsEnabledValue) throws IOException { + try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration)) { + Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs( + "getIsNamespaceEnabled should return true when the " + + "account specific config is not set").isEqualTo(expectedIsHnsEnabledValue); + } catch (Exception e) { + throw e; + } + } + } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 006004850d0df..be8fad984e3ed 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -187,7 +187,8 @@ public void testExternalOps() throws Exception { 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())); @@ -195,7 +196,8 @@ public void testExternalOps() throws Exception { 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); } }