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 72b3c257e76f2..9aaf17144729e 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 @@ -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 AbfsConfiguration(final Configuration rawConfig, String accountName) * @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 @@ 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. + * 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; } /** 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 d8074be5d17a1..348444fd1f1cc 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 @@ -229,6 +229,8 @@ public void initialize(URI uri, Configuration configuration) * 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 void initialize(URI uri, Configuration configuration) } } } - getAbfsStore().updateClientWithNamespaceInfo(new TracingContext(initFSTracingContext)); LOG.trace("Initiate check for delegation token manager"); if (UserGroupInformation.isSecurityEnabled()) { 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 e3df803480562..d93c1a3dc65e3 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 @@ -186,7 +186,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private final AbfsConfiguration abfsConfiguration; private Set azureInfiniteLeaseDirSet; - private volatile Trilean isNamespaceEnabled; private final AuthType authType; private final UserGroupInformation userGroupInformation; private final IdentityTransformerInterface identityTransformer; @@ -234,8 +233,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"); @@ -287,18 +284,6 @@ public AzureBlobFileSystemStore( "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 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In } /** - * 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 boolean getIsNamespaceEnabled(TracingContext tracingContext) 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 @@ private synchronized boolean getNamespaceEnabledInformationFromServer( // 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 @@ private synchronized boolean getNamespaceEnabledInformationFromServer( */ @VisibleForTesting boolean isNamespaceEnabled() throws TrileanConversionException { - return this.isNamespaceEnabled.toBoolean(); + return getAbfsConfiguration().getIsNamespaceEnabledAccount().toBoolean(); } @VisibleForTesting @@ -2026,9 +2021,8 @@ DataBlocks.BlockFactory getBlockFactory() { return blockFactory; } - @VisibleForTesting - void setNamespaceEnabled(Trilean isNamespaceEnabled){ - this.isNamespaceEnabled = isNamespaceEnabled; + void setNamespaceEnabled(boolean isNamespaceEnabled){ + getAbfsConfiguration().setIsNamespaceEnabledAccount(isNamespaceEnabled); } @VisibleForTesting 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 8c13b972f7f1d..09a2f1549a742 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 @@ -66,9 +66,11 @@ 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 @@ protected List createCommonHeaders(ApiVersion xMsVersion) { 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 @@ String initializeUserAgent(final AbfsConfiguration abfsConfiguration, 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 @@ protected String getUserAgent() { /** * 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() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java index 2b1fa87756396..577d99560ded2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java @@ -1563,9 +1563,11 @@ private Hashtable parseCommaSeparatedXmsProperties(String xMsPro * @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 requestHeaders) { + public String addClientTransactionIdToHeader(List requestHeaders) + throws AzureBlobFileSystemException { String clientTransactionId = null; // Set client transaction ID if the namespace and client transaction ID config are enabled. if (getIsNamespaceEnabled() && getAbfsConfiguration().getIsClientTransactionIdEnabled()) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java index 71c77ce82c8e2..c1ea6f3c22817 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java @@ -102,8 +102,8 @@ private void setTestUserFs() throws Exception { 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, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java index 35ce615ba738c..153edab897a11 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java @@ -33,7 +33,6 @@ 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 void testGetAclCallOnHnsConfigAbsence() throws Exception { 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)) 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 0e69a09f8002b..dd32643063a3a 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 @@ -30,6 +30,7 @@ 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 void testFailedRequestWhenFSNotExist() throws Exception { + 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 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)) @@ -217,7 +220,8 @@ private void ensureGetAclCallIsMadeOnceForInvalidConf(String invalidConf) 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 @@ 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)) @@ -262,7 +267,7 @@ private void ensureGetAclDetermineHnsStatusAccuratelyInternal(int statusCode, 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 @@ private void ensureGetAclDetermineHnsStatusAccuratelyInternal(int statusCode, 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 void testAccountSpecificConfig() throws Exception { } } + /** + * 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)) { 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 7fb672920cd67..3a9365e208506 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 @@ -200,7 +200,7 @@ 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())); @@ -208,7 +208,7 @@ 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); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java index 9af8f0f5a6f34..848f686f8eb6a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java @@ -39,6 +39,7 @@ 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 static void mockGetRenameBlobHandler(AbfsBlobClient blobClient, * @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 headers = addClientTransactionId.getArgument(0); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java index c859be5fd0daa..159d8e1379dbd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java +++ b/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.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 void testExistingPathCorrectlyRejected() throws Exception { */ @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);