From 05b52e40f1bc99edc039fc0d2ab5f83d1ceb0da9 Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Fri, 24 Oct 2025 02:31:26 -0700 Subject: [PATCH 1/9] user-bound SAS --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 45 ++++- .../fs/azurebfs/AzureBlobFileSystemStore.java | 10 +- .../azurebfs/services/AbfsRestOperation.java | 5 + .../hadoop/fs/azurebfs/services/AuthType.java | 3 +- .../hadoop-azure/src/site/markdown/index.md | 40 ++++ ...reBlobFileSystemUserboundSASWithOAuth.java | 189 ++++++++++++++++++ .../fs/azurebfs/TestAccountConfiguration.java | 7 + .../constants/TestConfigurationKeys.java | 3 + .../MockDelegationSASTokenProvider.java | 4 +- .../utils/DelegationSASGenerator.java | 17 +- 10 files changed, 317 insertions(+), 6 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserboundSASWithOAuth.java 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 7c355671cf8b2..50bd760e534b4 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 @@ -1279,7 +1279,7 @@ public boolean shouldTrackLatency() { public AccessTokenProvider getTokenProvider() throws TokenAccessProviderException { AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); - if (authType == AuthType.OAuth) { + if (authType == AuthType.OAuth || authType == AuthType.UserboundSASWithOAuth) { try { Class tokenProviderClass = getTokenProviderClass(authType, @@ -1474,6 +1474,49 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio } } + /** + * Returns the SASTokenProvider implementation to be used to generate user-bound SAS token.
+ * Custom implementation of {@link SASTokenProvider} under th config + * "fs.azure.sas.token.provider.type" needs to be provided.
+ * @return sasTokenProvider object based on configurations provided + * @throws AzureBlobFileSystemException + */ + public SASTokenProvider getSASTokenProviderForUserBoundSAS() throws AzureBlobFileSystemException { + AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); + if (authType != AuthType.UserboundSASWithOAuth) { + throw new SASTokenProviderException(String.format( + "Invalid auth type: %s is being used, expecting user-bound SAS.", authType)); + } + + try { + Class customSasTokenProviderImplementation = + getTokenProviderClass(authType, FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, + null, SASTokenProvider.class); + + if (customSasTokenProviderImplementation == null) { + throw new SASTokenProviderException(String.format( + "\"%s\" must be set for user-bound SAS auth type.", + FS_AZURE_SAS_TOKEN_PROVIDER_TYPE)); + } + + SASTokenProvider sasTokenProvider = ReflectionUtils.newInstance( + customSasTokenProviderImplementation, rawConfig); + if (sasTokenProvider == null) { + throw new SASTokenProviderException(String.format( + "Failed to initialize %s", customSasTokenProviderImplementation)); + } + LOG.trace("Initializing {}", customSasTokenProviderImplementation.getName()); + sasTokenProvider.initialize(rawConfig, accountName); + LOG.trace("{} init complete", customSasTokenProviderImplementation.getName()); + return sasTokenProvider; + } catch (SASTokenProviderException e) { + throw e; + } catch (Exception e) { + throw new SASTokenProviderException( + "Unable to load user-bound SAS token provider class: " + e, e); + } + } + public EncryptionContextProvider createEncryptionContextProvider() { try { String configKey = FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; 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 d51559de18ea0..f3463688b168f 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 @@ -1741,7 +1741,15 @@ private void initializeClient(URI uri, String fileSystemName, } else if (authType == AuthType.SAS) { LOG.trace("Fetching SAS Token Provider"); sasTokenProvider = abfsConfiguration.getSASTokenProvider(); - } else { + } else if(authType == AuthType.UserboundSASWithOAuth){ + LOG.trace("Fetching SAS and OAuth Token Provider for user bound SAS"); + AzureADAuthenticator.init(abfsConfiguration); + tokenProvider = abfsConfiguration.getTokenProvider(); + ExtensionHelper.bind(tokenProvider, uri, + abfsConfiguration.getRawConfiguration()); + sasTokenProvider = abfsConfiguration.getSASTokenProviderForUserBoundSAS(); + } + else { LOG.trace("Fetching token provider"); tokenProvider = abfsConfiguration.getTokenProvider(); ExtensionHelper.bind(tokenProvider, uri, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index c019fcbc3d3a7..6df4425f653cb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -570,6 +570,11 @@ public void signRequest(final AbfsHttpOperation httpOperation, int bytesToSign) // do nothing; the SAS token should already be appended to the query string httpOperation.setMaskForSAS(); //mask sig/oid from url for logs break; + case UserboundSASWithOAuth: + httpOperation.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION, + client.getAccessToken()); + httpOperation.setMaskForSAS(); //mask sig/oid from url for logs + break; case SharedKey: default: // sign the HTTP request diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java index 03ffece350e6b..142b266fb4664 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java @@ -24,5 +24,6 @@ public enum AuthType { SharedKey, OAuth, Custom, - SAS + SAS, + UserboundSASWithOAuth } diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/index.md b/hadoop-tools/hadoop-azure/src/site/markdown/index.md index e52555ef76f9d..91141e26e98f8 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/index.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/index.md @@ -303,6 +303,7 @@ driven by them. 3. Deployed in-Azure with the Azure VMs providing OAuth 2.0 tokens to the application, "Managed Instance". 4. Using Shared Access Signature (SAS) tokens provided by a custom implementation of the SASTokenProvider interface. 5. By directly configuring a fixed Shared Access Signature (SAS) token in the account configuration settings files. +6. Using user-bound SAS auth type, which is requires OAuth 2.0 setup (point 2 above) and SAS setup (point 4 above) Note: SAS Based Authentication should be used only with HNS Enabled accounts. @@ -783,6 +784,45 @@ requests. User can specify them as fixed SAS Token to be used across all the req - fs.azure.sas.fixed.token.ACCOUNT_NAME - fs.azure.sas.fixed.token +### User-bound user delegation SAS +- **Description**: The user-bound SAS auth type allows to track the usage of the SAS token generated- something + that was not possible in user-delegation SAS authentication type. Reach out to us at 'askabfs@microsoft.com' for more information. + To use this authentication type, both custom SAS token provider class (that implements org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider) as + well as OAuth 2.0 provider type need to be specified. + - Refer to 'Shared Access Signature (SAS) Token Provider' section above for user-delegation SAS token provider class details and example class implementation. + - There are multiple identity configurations for OAuth settings. Listing the main ones below: + - Client Credentials + - Custom token provider + - Managed Identity + - Workload Identity + Refer to respective OAuth 2.0 sections above to correctly chose the OAuth provider type + + +- **Configuration**: To use this method with ABFS Driver, specify the following properties in your `core-site.xml` file: + + 1. Authentication Type: + ```xml + + fs.azure.account.auth.type + UserboundSASWithOAuth + + ``` + 2. OAuth 2.0 Provider Type: + ```xml + + fs.azure.account.oauth.provider.type + org.apache.hadoop.fs.azurebfs.oauth2.ADD_CHOSEN_OAUTH_IDENTITY_CONFIGURATION + + ``` + 3. Custom SAS Token Provider Class: + ```xml + + fs.azure.sas.token.provider.type + CUSTOM_SAS_TOKEN_PROVIDER_CLASS + + ``` + + ## Technical notes ### Proxy setup diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserboundSASWithOAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserboundSASWithOAuth.java new file mode 100644 index 0000000000000..4449d6faa5e67 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserboundSASWithOAuth.java @@ -0,0 +1,189 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.AccessDeniedException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Hashtable; +import java.util.List; +import java.util.UUID; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; +import org.apache.hadoop.fs.azurebfs.extensions.MockUserBoundSASTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.AclEntryScope; +import org.apache.hadoop.fs.permission.AclStatus; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.AccessControlException; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_FILE_ALREADY_EXISTS; +import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; +import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; +import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; +import static org.apache.hadoop.fs.permission.AclEntryType.GROUP; +import static org.apache.hadoop.fs.permission.AclEntryType.USER; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assumptions.assumeThat; + +import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; + +/** + * Test Perform Authorization Check operation for UserboundSASWithOAuth auth type + */ +public class ITestAzureBlobFileSystemUserboundSASWithOAuth extends AbstractAbfsIntegrationTest { + private static final String TEST_GROUP = UUID.randomUUID().toString(); + + private static final Logger LOG = + LoggerFactory.getLogger(ITestAzureBlobFileSystemUserboundSASWithOAuth.class); + + private boolean isHNSEnabled; + + public ITestAzureBlobFileSystemUserboundSASWithOAuth() throws Exception { + // These tests rely on specific settings in azure-auth-keys.xml: + String sasProvider = getRawConfiguration().get( + FS_AZURE_SAS_TOKEN_PROVIDER_TYPE); + assumeThat( + MockUserBoundSASTokenProvider.class.getCanonicalName()).isEqualTo( + sasProvider); + assumeThat(getRawConfiguration().get( + TestConfigurationKeys.FS_AZURE_TEST_APP_ID)).isNotNull(); + assumeThat(getRawConfiguration().get( + TestConfigurationKeys.FS_AZURE_TEST_APP_SECRET)).isNotNull(); + assumeThat(getRawConfiguration().get( + TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID)).isNotNull(); + assumeThat(getRawConfiguration().get( + TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID)).isNotNull(); + //todo: check if this would still be relevant for user bound SAS testing +// The test uses shared key to create a random filesystem and then creates another +// instance of this filesystem using SAS+OAuth authorization. + //assumeThat(this.getAuthType()).isEqualTo(AuthType.SharedKey); + } + + @BeforeEach + @Override + public void setup() throws Exception { + isHNSEnabled = this.getConfiguration().getBoolean( + TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); + if (!isHNSEnabled) { + assumeBlobServiceType(); + } + createFilesystemForSASTests(); + super.setup(); + } + + @Test + // FileSystemProperties are not supported by delegation SAS (hence user-bound SAS too) and should throw exception + public void testSetFileSystemProperties() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Hashtable + properties = new Hashtable<>(); + properties.put("FileSystemProperties", "true"); + TracingContext tracingContext = getTestTracingContext(fs, true); + assertThrows(IOException.class, () -> fs.getAbfsStore() + .setFilesystemProperties(properties, tracingContext)); + assertThrows(IOException.class, + () -> fs.getAbfsStore().getFilesystemProperties(tracingContext)); + } + + + @Test + public void testSignatureMaskOnExceptionMessage() throws Exception { + intercept(IOException.class, "sig=XXXX", + () -> getFileSystem().getAbfsClient() + .renamePath("testABC/test.xt", "testABC/abc.txt", + null, getTestTracingContext(getFileSystem(), false), + null, false)); + } + + + @Test + public void testSASQuesMarkPrefix() throws Exception { + AbfsConfiguration testConfig = this.getConfiguration(); + // the SAS Token Provider is changed + testConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, "org.apache.hadoop.fs.azurebfs.extensions.MockWithPrefixSASTokenProvider"); + + AzureBlobFileSystem testFs = (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()); + Path testFile = new Path("/testSASPrefixQuesMark"); + + // the creation of this filesystem should work correctly even when a SAS Token is generated with a ? prefix + testFs.create(testFile).close(); + } + + @Test + // Verify OAuth token provider and user-bound SAS provider are both configured and usable + //CURRENTLY ONLY WORKING WITH THE REMOVED (BUT OPTIONAL) UDK PARAM + public void testOAuthTokenProviderAndSASTokenFlow() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + + // Verify AbfsConfiguration has an OAuth token provider configured + AbfsConfiguration config = fs.getAbfsStore().getAbfsConfiguration(); + config.set("fs.azure.account.auth.type", "UserboundSASWithOAuth"); + + AccessTokenProvider tokenProvider = config.getTokenProvider(); + assertNotNull(tokenProvider, "AccessTokenProvider must be configured for UserboundSASWithOAuth"); + + // Acquire an OAuth token and assert it is non-empty + AzureADToken token = tokenProvider.getToken(); + assertNotNull(token, "OAuth token must not be null"); + assertNotNull(token.getAccessToken(), "OAuth access token must not be null"); + assertFalse(token.getAccessToken().isEmpty(), "OAuth access token must not be empty"); + + // Verify SASTokenProvider for user-bound SAS is present and usable + SASTokenProvider sasProvider = config.getSASTokenProviderForUserBoundSAS(); + assertNotNull(sasProvider, "SASTokenProvider for user-bound SAS must be configured"); + assertTrue(sasProvider instanceof MockUserBoundSASTokenProvider, + "Expected MockUserBoundSASTokenProvider to be used for tests"); + + // Request a SAS token and assert we get a non-empty result + String sasToken = sasProvider.getSASToken(getAccountName(), getFileSystemName(), "/", SASTokenProvider.GET_PROPERTIES_OPERATION); + assertNotNull(sasToken, "SAS token must not be null"); + assertFalse(sasToken.isEmpty(), "SAS token must not be empty"); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java index 4a60bfad29151..429422fc30549 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java @@ -518,6 +518,7 @@ public void setAuthConfig(AbfsConfiguration abfsConf, String providerClassKey = ""; String providerClassValue = ""; + //todo: add user-bound auth type handling tests switch (authType) { case OAuth: providerClassKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME @@ -548,6 +549,12 @@ public void setAuthConfig(AbfsConfiguration abfsConf, providerClassValue = TEST_SAS_PROVIDER_CLASS_CONFIG_1; break; + case UserboundSASWithOAuth: +// providerClassKey = FS_AZURE_SAS_TOKEN_PROVIDER_TYPE +// + (isAccountSetting ? accountNameSuffix : ""); +// providerClassValue = TEST_SAS_PROVIDER_CLASS_CONFIG_1; + break; + default: // set nothing } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java index ebccae55c0a93..e60086f2298a7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java @@ -55,6 +55,9 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID = "fs.azure.test.app.service.principal.object.id"; + public static final String FS_AZURE_END_USER_TENANT_ID = "fs.azure.test.end.user.tenant.id"; + public static final String FS_AZURE_END_USER_OBJECT_ID = "fs.azure.test.end.user.object.id"; + public static final String FS_AZURE_TEST_APP_ID = "fs.azure.test.app.id"; public static final String FS_AZURE_TEST_APP_SECRET = "fs.azure.test.app.secret"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java index 36c38e80b79ce..bbc9222eecc3a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; import org.apache.hadoop.security.AccessControlException; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CONNECTION_TIMEOUT; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_READ_TIMEOUT; @@ -65,8 +66,7 @@ public void initialize(Configuration configuration, String accountName) throws I String skv = SASGenerator.AuthenticationVersion.Dec19.toString(); byte[] key = getUserDelegationKey(accountName, appID, appSecret, sktid, skt, ske, skv); - - generator = new DelegationSASGenerator(key, skoid, sktid, skt, ske, skv); + generator = new DelegationSASGenerator(key, skoid, sktid, skt, ske, skv, EMPTY_STRING, EMPTY_STRING); } // Invokes the AAD v2.0 authentication endpoint with a client credentials grant to get an diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java index eec0d86f0b6bb..66e33cd143f6b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java @@ -19,11 +19,13 @@ package org.apache.hadoop.fs.azurebfs.utils; import java.time.Instant; +import java.util.Objects; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; import org.apache.hadoop.fs.azurebfs.services.AbfsUriQueryBuilder; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH; /** @@ -36,14 +38,18 @@ public class DelegationSASGenerator extends SASGenerator { private final String ske; private final String sks = "b"; private final String skv; + private final String skdutid; + private final String sduoid; - public DelegationSASGenerator(byte[] userDelegationKey, String skoid, String sktid, String skt, String ske, String skv) { + public DelegationSASGenerator(byte[] userDelegationKey, String skoid, String sktid, String skt, String ske, String skv, String skdutid, String sduoid) { super(userDelegationKey); this.skoid = skoid; this.sktid = sktid; this.skt = skt; this.ske = ske; this.skv = skv; + this.skdutid = skdutid; + this.sduoid = sduoid; } public String getDelegationSAS(String accountName, String containerName, String path, String operation, @@ -117,6 +123,15 @@ public String getDelegationSAS(String accountName, String containerName, String qb.addQuery("ske", ske); qb.addQuery("sks", sks); qb.addQuery("skv", skv); + + //skdutid and sduoid are required for user bound SAS only + if(!Objects.equals(skdutid, EMPTY_STRING)){ + qb.addQuery("skdutid", skdutid); + } + if(!Objects.equals(sduoid, EMPTY_STRING)){ + qb.addQuery("sduoid", sduoid); + } + if (saoid != null) { qb.addQuery("saoid", saoid); } From 4c181df7561b2e07c5b9276a3d8fa340c6b7a906 Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Mon, 27 Oct 2025 20:32:30 -0700 Subject: [PATCH 2/9] UBS code changes --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 5 ++++ .../azurebfs/constants/AbfsHttpConstants.java | 5 ++-- .../fs/azurebfs/services/AbfsClient.java | 17 ++++++++++- .../azurebfs/services/AbfsClientHandler.java | 29 ++++++++++++++++++- .../fs/azurebfs/services/AbfsDfsClient.java | 11 +++++++ .../utils/DelegationSASGenerator.java | 15 +++++++++- .../fs/azurebfs/utils/SASGenerator.java | 3 +- 7 files changed, 79 insertions(+), 6 deletions(-) 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 f3463688b168f..03f6a687e8b61 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 @@ -1778,6 +1778,11 @@ private void initializeClient(URI uri, String fileSystemName, } LOG.trace("Initializing AbfsClient for {}", baseUrl); + if(tokenProvider != null && sasTokenProvider != null){ + this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration, + tokenProvider, sasTokenProvider, encryptionContextProvider, + populateAbfsClientContext()); + } else if (tokenProvider != null) { this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration, tokenProvider, encryptionContextProvider, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index fe4991c9582d5..5b836cc7452e3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -187,7 +187,8 @@ public enum ApiVersion { DEC_12_2019("2019-12-12"), APR_10_2021("2021-04-10"), AUG_03_2023("2023-08-03"), - NOV_04_2024("2024-11-04"); + NOV_04_2024("2024-11-04"), + JULY_05_2025("2025-07-05"); private final String xMsApiVersion; @@ -201,7 +202,7 @@ public String toString() { } public static ApiVersion getCurrentVersion() { - return NOV_04_2024; + return JULY_05_2025; } } 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 71da8f9bda96e..6470f7b45900c 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 @@ -363,6 +363,21 @@ public AbfsClient(final URL baseUrl, this.sasTokenProvider = sasTokenProvider; } + public AbfsClient(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final AccessTokenProvider tokenProvider, + final SASTokenProvider sasTokenProvider, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext, + final AbfsServiceType abfsServiceType) + throws IOException { + this(baseUrl, sharedKeyCredentials, abfsConfiguration, + encryptionContextProvider, abfsClientContext, abfsServiceType); + this.sasTokenProvider = sasTokenProvider; + this.tokenProvider = tokenProvider; + } + @Override public void close() throws IOException { if (isMetricCollectionEnabled && runningTimerTask != null) { @@ -1157,7 +1172,7 @@ protected String appendSASTokenToQuery(String path, String cachedSasToken) throws SASTokenProviderException { String sasToken = null; - if (this.authType == AuthType.SAS) { + if (this.authType == AuthType.SAS || this.authType == AuthType.UserboundSASWithOAuth) { try { LOG.trace("Fetch SAS token for {} on {}", operation, path); if (cachedSasToken == null) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java index a7bf5699dc208..996182dad245c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java @@ -80,6 +80,25 @@ public AbfsClientHandler(final URL baseUrl, abfsClientContext); } + public AbfsClientHandler(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final AccessTokenProvider tokenProvider, + final SASTokenProvider sasTokenProvider, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext) throws IOException { + // This will initialize the default and ingress service types. + // This is needed before creating the clients so that we can do cache warmup + // only for default client. + initServiceType(abfsConfiguration); + this.dfsAbfsClient = createDfsClient(baseUrl, sharedKeyCredentials, + abfsConfiguration, tokenProvider, sasTokenProvider, encryptionContextProvider, + abfsClientContext); + this.blobAbfsClient = createBlobClient(baseUrl, sharedKeyCredentials, + abfsConfiguration, tokenProvider, sasTokenProvider, encryptionContextProvider, + abfsClientContext); + } + /** * Initialize the default service type based on the user configuration. * @param abfsConfiguration set by user. @@ -154,7 +173,15 @@ private AbfsDfsClient createDfsClient(final URL baseUrl, final EncryptionContextProvider encryptionContextProvider, final AbfsClientContext abfsClientContext) throws IOException { URL dfsUrl = changeUrlFromBlobToDfs(baseUrl); - if (tokenProvider != null) { + if (tokenProvider != null && sasTokenProvider != null) { + LOG.debug( + "Creating AbfsDfsClient with both access token provider and SAS token provider using the URL: {}", + dfsUrl); + return new AbfsDfsClient(dfsUrl, creds, abfsConfiguration, + tokenProvider, sasTokenProvider, encryptionContextProvider, + abfsClientContext); + } + else if (tokenProvider != null) { LOG.debug("Creating AbfsDfsClient with access token provider using the URL: {}", dfsUrl); return new AbfsDfsClient(dfsUrl, creds, abfsConfiguration, tokenProvider, encryptionContextProvider, 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 f574f4704ab5c..2bf89f1ca4357 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 @@ -174,6 +174,17 @@ public AbfsDfsClient(final URL baseUrl, encryptionContextProvider, abfsClientContext, AbfsServiceType.DFS); } + public AbfsDfsClient(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final AccessTokenProvider tokenProvider, + final SASTokenProvider sasTokenProvider, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext) throws IOException { + super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, sasTokenProvider, + encryptionContextProvider, abfsClientContext, AbfsServiceType.DFS); + } + /** * Create request headers for Rest Operation using the default API version. * @return default request headers. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java index 66e33cd143f6b..8bee328dded32 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java @@ -55,7 +55,9 @@ public DelegationSASGenerator(byte[] userDelegationKey, String skoid, String skt public String getDelegationSAS(String accountName, String containerName, String path, String operation, String saoid, String suoid, String scid) { - final String sv = AuthenticationVersion.Feb20.toString(); + final String sv = AuthenticationVersion.July5.toString(); + //todo: this will be removed later. Keeping for now + //final String sv = AuthenticationVersion.Feb20.toString(); final String st = ISO_8601_FORMATTER.format(Instant.now().minus(FIVE_MINUTES)); final String se = ISO_8601_FORMATTER.format(Instant.now().plus(ONE_DAY)); String sr = "b"; @@ -198,6 +200,16 @@ private String computeSignatureForSAS(String sp, String st, String se, String sv } sb.append("\n"); + if (skdutid != null) { + sb.append(skdutid); + } + sb.append("\n"); + if (sduoid != null) { + sb.append(sduoid); + } + sb.append("\n"); + + sb.append("\n"); // sip sb.append("\n"); // spr sb.append(sv); @@ -212,6 +224,7 @@ private String computeSignatureForSAS(String sp, String st, String se, String sv String stringToSign = sb.toString(); LOG.debug("Delegation SAS stringToSign: " + stringToSign.replace("\n", ".")); + System.out.println("Delegation SAS stringToSign: " + stringToSign.replace("\n", ".")); return computeHmac256(stringToSign); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java index a80ddac5ed36f..c51001c9f1da4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java @@ -41,7 +41,8 @@ public abstract class SASGenerator { public enum AuthenticationVersion { Nov18("2018-11-09"), Dec19("2019-12-12"), - Feb20("2020-02-10"); + Feb20("2020-02-10"), + July5("2025-07-05"); private final String ver; From 0411d9b9e4e19b048168bea7cc81e5b1e5ffb992 Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Mon, 27 Oct 2025 21:03:07 -0700 Subject: [PATCH 3/9] UDS compatible --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 4 ++-- .../azurebfs/utils/DelegationSASGenerator.java | 16 ++++++++++------ 2 files changed, 12 insertions(+), 8 deletions(-) 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 03f6a687e8b61..b90cacea0c07a 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 @@ -1782,8 +1782,8 @@ private void initializeClient(URI uri, String fileSystemName, this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration, tokenProvider, sasTokenProvider, encryptionContextProvider, populateAbfsClientContext()); - } else - if (tokenProvider != null) { + } + else if (tokenProvider != null) { this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration, tokenProvider, encryptionContextProvider, populateAbfsClientContext()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java index 8bee328dded32..a7dae2b1b732e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java @@ -55,9 +55,9 @@ public DelegationSASGenerator(byte[] userDelegationKey, String skoid, String skt public String getDelegationSAS(String accountName, String containerName, String path, String operation, String saoid, String suoid, String scid) { - final String sv = AuthenticationVersion.July5.toString(); + //final String sv = AuthenticationVersion.July5.toString(); //todo: this will be removed later. Keeping for now - //final String sv = AuthenticationVersion.Feb20.toString(); + final String sv = AuthenticationVersion.Feb20.toString(); final String st = ISO_8601_FORMATTER.format(Instant.now().minus(FIVE_MINUTES)); final String se = ISO_8601_FORMATTER.format(Instant.now().plus(ONE_DAY)); String sr = "b"; @@ -200,14 +200,18 @@ private String computeSignatureForSAS(String sp, String st, String se, String sv } sb.append("\n"); - if (skdutid != null) { + // skdutid, sduoid are sent as empty strings for user-delegation SAS + // They are only required for user-bound SAS so added the escape sequences + // also inside if checks only + if (!Objects.equals(skdutid, EMPTY_STRING)) { sb.append(skdutid); + sb.append("\n"); } - sb.append("\n"); - if (sduoid != null) { + + if (!Objects.equals(sduoid, EMPTY_STRING)) { sb.append(sduoid); + sb.append("\n"); } - sb.append("\n"); sb.append("\n"); // sip From a33f6820ef3babc8c8e9210469d1151949b40581 Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Mon, 27 Oct 2025 22:23:32 -0700 Subject: [PATCH 4/9] missed class added --- .../MockUserBoundSASTokenProvider.java | 166 ++++++++++++++++++ .../utils/DelegationSASGenerator.java | 4 +- 2 files changed, 168 insertions(+), 2 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java new file mode 100644 index 0000000000000..cabbddde1100e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java @@ -0,0 +1,166 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.extensions; + +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; +import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; +import org.apache.hadoop.fs.azurebfs.services.AbfsJdkHttpOperation; +import org.apache.hadoop.fs.azurebfs.utils.Base64; +import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator; +import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; +import org.apache.hadoop.security.AccessControlException; + +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CONNECTION_TIMEOUT; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_READ_TIMEOUT; + +/** + * A mock user-bound SAS token provider implementation. + */ + +public class MockUserBoundSASTokenProvider implements SASTokenProvider { + + private DelegationSASGenerator generator; + + public static final String TEST_OWNER = "325f1619-4205-432f-9fce-3fd594325ce5"; + public static final String CORRELATION_ID = "66ff4ffc-ff17-417e-a2a9-45db8c5b0b5c"; + public static final String NO_AGENT_PATH = "NoAgentPath"; + + @Override + public void initialize(Configuration configuration, String accountName) throws IOException { + String appID = configuration.get(TestConfigurationKeys.FS_AZURE_TEST_APP_ID); + String appSecret = configuration.get(TestConfigurationKeys.FS_AZURE_TEST_APP_SECRET); + String sktid = configuration.get(TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID); + String skoid = configuration.get(TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID); + String skt = SASGenerator.ISO_8601_FORMATTER.format(Instant.now().minus(SASGenerator.FIVE_MINUTES)); + String ske = SASGenerator.ISO_8601_FORMATTER.format(Instant.now().plus(SASGenerator.ONE_DAY)); + String skv = SASGenerator.AuthenticationVersion.July5.toString(); + + String skdutid = configuration.get(TestConfigurationKeys.FS_AZURE_END_USER_TENANT_ID); + String sduoid = configuration.get(TestConfigurationKeys.FS_AZURE_END_USER_OBJECT_ID); + + byte[] key = getUserDelegationKey(accountName, appID, appSecret, sktid, skt, ske, skv, skdutid); + + generator = new DelegationSASGenerator(key, skoid, sktid, skt, ske, skv, skdutid, sduoid); + } + + // Invokes the AAD v2.0 authentication endpoint with a client credentials grant to get an + // access token. See https://docs.microsoft.com/en-us/azure/active-directory/develop/v2-oauth2-client-creds-grant-flow. + private String getAuthorizationHeader(String accountName, String appID, String appSecret, String sktid) throws IOException { + String authEndPoint = String.format("https://login.microsoftonline.com/%s/oauth2/v2.0/token", sktid); + ClientCredsTokenProvider provider = new ClientCredsTokenProvider(authEndPoint, appID, appSecret); + return "Bearer " + provider.getToken().getAccessToken(); + } + + private byte[] getUserDelegationKey(String accountName, String appID, String appSecret, + String sktid, String skt, String ske, String skv, String skdutid) throws IOException { + + String method = "POST"; + String account = accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)); + + final StringBuilder sb = new StringBuilder(128); + sb.append("https://"); + sb.append(account); + sb.append(".blob.core.windows.net/?restype=service&comp=userdelegationkey"); + + URL url; + try { + url = new URL(sb.toString()); + } catch (MalformedURLException ex) { + throw new InvalidUriException(sb.toString()); + } + + List requestHeaders = new ArrayList(); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_VERSION, skv)); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, "application/x-www-form-urlencoded")); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.AUTHORIZATION, getAuthorizationHeader(account, appID, appSecret, sktid))); + + final StringBuilder requestBody = new StringBuilder(512); + requestBody.append(""); + requestBody.append(skt); + requestBody.append(""); + requestBody.append(ske); + requestBody.append(""); + requestBody.append(skdutid); + requestBody.append(""); + +// requestBody.append(""); +// requestBody.append(skt); +// requestBody.append(""); +// requestBody.append(ske); +// requestBody.append(""); + + AbfsJdkHttpOperation op = new AbfsJdkHttpOperation(url, method, requestHeaders, + Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT), null); + + byte[] requestBuffer = requestBody.toString().getBytes(StandardCharsets.UTF_8.toString()); + op.sendPayload(requestBuffer, 0, requestBuffer.length); + + byte[] responseBuffer = new byte[4 * 1024]; + op.processResponse(responseBuffer, 0, responseBuffer.length); //GETTING NULL HERE [400- wrong XML] + + String responseBody = new String(responseBuffer, 0, (int) op.getBytesReceived(), StandardCharsets.UTF_8); + int beginIndex = responseBody.indexOf("") + "".length(); + int endIndex = responseBody.indexOf(""); + String value = responseBody.substring(beginIndex, endIndex); + return Base64.decode(value); + } + + /** + * Invokes the authorizer to obtain a SAS token. + * + * @param accountName the name of the storage account. + * @param fileSystem the name of the fileSystem. + * @param path the file or directory path. + * @param operation the operation to be performed on the path. + * @return a SAS token to perform the request operation. + * @throws IOException if there is a network error. + * @throws AccessControlException if access is denied. + */ + @Override + public String getSASToken(String accountName, String fileSystem, String path, + String operation) throws IOException, AccessControlException { + // Except for the special case where we test without an agent, + // the user for these tests is always TEST_OWNER. The check access operation + // requires suoid to check permissions for the user and will throw if the + // user does not have access and otherwise succeed. + String saoid = null; + String suoid = null; + if (path == null || !path.endsWith(NO_AGENT_PATH)) { + saoid = (operation == SASTokenProvider.CHECK_ACCESS_OPERATION) ? null : TEST_OWNER; + suoid = (operation == SASTokenProvider.CHECK_ACCESS_OPERATION) ? TEST_OWNER : null; + } + + return generator.getDelegationSAS(accountName, fileSystem, path, operation, + saoid, suoid, CORRELATION_ID); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java index a7dae2b1b732e..ef1229ae444b8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java @@ -55,9 +55,9 @@ public DelegationSASGenerator(byte[] userDelegationKey, String skoid, String skt public String getDelegationSAS(String accountName, String containerName, String path, String operation, String saoid, String suoid, String scid) { - //final String sv = AuthenticationVersion.July5.toString(); + final String sv = AuthenticationVersion.July5.toString(); //todo: this will be removed later. Keeping for now - final String sv = AuthenticationVersion.Feb20.toString(); + //final String sv = AuthenticationVersion.Feb20.toString(); final String st = ISO_8601_FORMATTER.format(Instant.now().minus(FIVE_MINUTES)); final String se = ISO_8601_FORMATTER.format(Instant.now().plus(ONE_DAY)); String sr = "b"; From c9a994e031fa84cf240b6277e3fc54d9bcbfaf56 Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Thu, 30 Oct 2025 04:42:12 -0700 Subject: [PATCH 5/9] sas version change --- .../MockDelegationSASTokenProvider.java | 6 ++--- .../MockUserBoundSASTokenProvider.java | 12 +++------- ...DelegationSASGenerator_Version_July5.java} | 24 ++++++++++++------- 3 files changed, 21 insertions(+), 21 deletions(-) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/{DelegationSASGenerator.java => DelegationSASGenerator_Version_July5.java} (88%) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java index bbc9222eecc3a..5032bdb7c01b3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java @@ -36,7 +36,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; import org.apache.hadoop.fs.azurebfs.services.AbfsJdkHttpOperation; import org.apache.hadoop.fs.azurebfs.utils.Base64; -import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator; +import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator_Version_July5; import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; import org.apache.hadoop.security.AccessControlException; @@ -49,7 +49,7 @@ */ public class MockDelegationSASTokenProvider implements SASTokenProvider { - private DelegationSASGenerator generator; + private DelegationSASGenerator_Version_July5 generator; public static final String TEST_OWNER = "325f1619-4205-432f-9fce-3fd594325ce5"; public static final String CORRELATION_ID = "66ff4ffc-ff17-417e-a2a9-45db8c5b0b5c"; @@ -66,7 +66,7 @@ public void initialize(Configuration configuration, String accountName) throws I String skv = SASGenerator.AuthenticationVersion.Dec19.toString(); byte[] key = getUserDelegationKey(accountName, appID, appSecret, sktid, skt, ske, skv); - generator = new DelegationSASGenerator(key, skoid, sktid, skt, ske, skv, EMPTY_STRING, EMPTY_STRING); + generator = new DelegationSASGenerator_Version_July5(key, skoid, sktid, skt, ske, skv, EMPTY_STRING, EMPTY_STRING); } // Invokes the AAD v2.0 authentication endpoint with a client credentials grant to get an diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java index cabbddde1100e..ffb087c63800a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java @@ -36,7 +36,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; import org.apache.hadoop.fs.azurebfs.services.AbfsJdkHttpOperation; import org.apache.hadoop.fs.azurebfs.utils.Base64; -import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator; +import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator_Version_July5; import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; import org.apache.hadoop.security.AccessControlException; @@ -49,7 +49,7 @@ public class MockUserBoundSASTokenProvider implements SASTokenProvider { - private DelegationSASGenerator generator; + private DelegationSASGenerator_Version_July5 generator; public static final String TEST_OWNER = "325f1619-4205-432f-9fce-3fd594325ce5"; public static final String CORRELATION_ID = "66ff4ffc-ff17-417e-a2a9-45db8c5b0b5c"; @@ -70,7 +70,7 @@ public void initialize(Configuration configuration, String accountName) throws I byte[] key = getUserDelegationKey(accountName, appID, appSecret, sktid, skt, ske, skv, skdutid); - generator = new DelegationSASGenerator(key, skoid, sktid, skt, ske, skv, skdutid, sduoid); + generator = new DelegationSASGenerator_Version_July5(key, skoid, sktid, skt, ske, skv, skdutid, sduoid); } // Invokes the AAD v2.0 authentication endpoint with a client credentials grant to get an @@ -113,12 +113,6 @@ private byte[] getUserDelegationKey(String accountName, String appID, String app requestBody.append(skdutid); requestBody.append(""); -// requestBody.append(""); -// requestBody.append(skt); -// requestBody.append(""); -// requestBody.append(ske); -// requestBody.append(""); - AbfsJdkHttpOperation op = new AbfsJdkHttpOperation(url, method, requestHeaders, Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT), null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java similarity index 88% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java index ef1229ae444b8..469781c0a4a87 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java @@ -31,7 +31,7 @@ /** * Test Delegation SAS generator. */ -public class DelegationSASGenerator extends SASGenerator { +public class DelegationSASGenerator_Version_July5 extends SASGenerator { private final String skoid; private final String sktid; private final String skt; @@ -41,7 +41,7 @@ public class DelegationSASGenerator extends SASGenerator { private final String skdutid; private final String sduoid; - public DelegationSASGenerator(byte[] userDelegationKey, String skoid, String sktid, String skt, String ske, String skv, String skdutid, String sduoid) { + public DelegationSASGenerator_Version_July5(byte[] userDelegationKey, String skoid, String sktid, String skt, String ske, String skv, String skdutid, String sduoid) { super(userDelegationKey); this.skoid = skoid; this.sktid = sktid; @@ -55,9 +55,13 @@ public DelegationSASGenerator(byte[] userDelegationKey, String skoid, String skt public String getDelegationSAS(String accountName, String containerName, String path, String operation, String saoid, String suoid, String scid) { + // The params for signature computation (particularly the string-to-sign) are different based on the SAS version (sv) + // They might need to be changed if using a different version + //Ref: https://learn.microsoft.com/en-us/rest/api/storageservices/create-user-delegation-sas + + // SAS version (sv) used here is 2025-07-05 final String sv = AuthenticationVersion.July5.toString(); - //todo: this will be removed later. Keeping for now - //final String sv = AuthenticationVersion.Feb20.toString(); + final String st = ISO_8601_FORMATTER.format(Instant.now().minus(FIVE_MINUTES)); final String se = ISO_8601_FORMATTER.format(Instant.now().plus(ONE_DAY)); String sr = "b"; @@ -117,6 +121,7 @@ public String getDelegationSAS(String accountName, String containerName, String String signature = computeSignatureForSAS(sp, st, se, sv, sr, accountName, containerName, path, saoid, suoid, scid); + // String signature = "testttsstst"; AbfsUriQueryBuilder qb = new AbfsUriQueryBuilder(); qb.addQuery("skoid", skoid); @@ -201,17 +206,16 @@ private String computeSignatureForSAS(String sp, String st, String se, String sv sb.append("\n"); // skdutid, sduoid are sent as empty strings for user-delegation SAS - // They are only required for user-bound SAS so added the escape sequences - // also inside if checks only + // They are only required for user-bound SAS if (!Objects.equals(skdutid, EMPTY_STRING)) { sb.append(skdutid); - sb.append("\n"); } + sb.append("\n"); if (!Objects.equals(sduoid, EMPTY_STRING)) { sb.append(sduoid); - sb.append("\n"); } + sb.append("\n"); sb.append("\n"); // sip @@ -220,11 +224,13 @@ private String computeSignatureForSAS(String sp, String st, String se, String sv sb.append("\n"); sb.append(sr); sb.append("\n"); + sb.append("\n"); // - For optional : signedSnapshotTime + sb.append("\n"); // - For optional :signedEncryptionScope sb.append("\n"); // - For optional : rscc - ResponseCacheControl sb.append("\n"); // - For optional : rscd - ResponseContentDisposition sb.append("\n"); // - For optional : rsce - ResponseContentEncoding sb.append("\n"); // - For optional : rscl - ResponseContentLanguage - sb.append("\n"); // - For optional : rsct - ResponseContentType + //No escape sequence required for optional param rsct - ResponseContentType String stringToSign = sb.toString(); LOG.debug("Delegation SAS stringToSign: " + stringToSign.replace("\n", ".")); From 0a168e9a4dcbb98fc41ef4caa8802615a5b4ae68 Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Mon, 3 Nov 2025 19:59:18 -0800 Subject: [PATCH 6/9] tests --- .../hadoop/fs/azurebfs/IntegrationTests.java | 378 ++++++++++++++++++ 1 file changed, 378 insertions(+) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/IntegrationTests.java diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/IntegrationTests.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/IntegrationTests.java new file mode 100644 index 0000000000000..eaecceb4170fa --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/IntegrationTests.java @@ -0,0 +1,378 @@ +package org.apache.hadoop.fs.azurebfs; + +import java.lang.reflect.Field; +import java.nio.charset.StandardCharsets; +import java.nio.file.AccessDeniedException; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.UUID; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider; +import org.apache.hadoop.fs.azurebfs.extensions.MockUserBoundSASTokenProvider; +import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AuthType; + +import static java.time.temporal.ChronoUnit.DAYS; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_END_USER_OBJECT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class IntegrationTests extends AbstractAbfsIntegrationTest { + + private static Path testPath = new Path("/test.txt"); + private static final String TEST_OBJECT_ID = "123456789"; + + protected IntegrationTests() throws Exception { + //todo: add later +// String sasProvider = getRawConfiguration().get(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE); +// assumeThat(MockUserBoundSASTokenProvider.class.getCanonicalName()).isEqualTo(sasProvider); + + assumeThat(this.getAuthType()).isEqualTo(AuthType.SharedKey); + } + + @BeforeEach + @Override + public void setup() throws Exception { + Boolean isHNSEnabled = this.getConfiguration().getBoolean( + TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); + if (!isHNSEnabled) { + assumeBlobServiceType(); + } + + //todo: thisi is changed + createFilesystemForUserBoundSASTests(); + super.setup(); + } + // TEST FOR SAS- HOW DOES CREATE CONTAINER PASS!!! + + + // Common helper to inject a mock token provider into AbfsClient + private void injectMockTokenProvider(AzureBlobFileSystem fs, AccessTokenProvider mockProvider) throws Exception { + Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField("abfsStore"); + abfsStoreField.setAccessible(true); + AzureBlobFileSystemStore store = (AzureBlobFileSystemStore) abfsStoreField.get(fs); + + Field abfsClientField = AzureBlobFileSystemStore.class.getDeclaredField("client"); + abfsClientField.setAccessible(true); + AbfsClient client = (AbfsClient) abfsClientField.get(store); + + Field tokenProviderField = AbfsClient.class.getDeclaredField("tokenProvider"); + tokenProviderField.setAccessible(true); + tokenProviderField.set(client, mockProvider); + } + + private void addOAuthConfigs(AzureBlobFileSystem fs, Configuration testConfig, String accountName) { + AbfsConfiguration abfsConfig = fs.getAbfsStore().getAbfsConfiguration(); + + testConfig.set(FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID + "." +accountName, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); + testConfig.set(FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); + + testConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + "." + accountName, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID)); + testConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID)); + + testConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET+ "." + accountName, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET)); + testConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET)); + + // Set a different SDUOID + testConfig.set(FS_AZURE_END_USER_OBJECT_ID, abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); +// testConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, +// "org.apache.hadoop.fs.azurebfs.extensions.MockUserBoundSASTokenProvider"); + } + + + @Test + public void testShouldFailWhenSduoidMismatchesServicePrincipalId() + //WIHOUT ABSTRACT INTEGRATION CLASS + throws Exception { + // Arrange + //final AzureBlobFileSystem fs = this.getFileSystem(); + AzureBlobFileSystem fs = getFileSystem(); + //AzureBlobFileSystem fs = createFs(); + AzureBlobFileSystem testFs = new AzureBlobFileSystem(); + + String accountName = getAccountName(); + +// Configuration testConfig = getConfiguration() + Configuration testConfig = new Configuration(getRawConfiguration()); + addOAuthConfigs(fs, testConfig, accountName); + + testConfig.set(FS_AZURE_END_USER_OBJECT_ID, TEST_OBJECT_ID); + testFs.initialize(fs.getUri(), testConfig); + intercept(AccessDeniedException.class, + ()-> { + testFs.create(testPath); + }); + } + + @Test + public void testReadAndWrite() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystem testFs = new AzureBlobFileSystem(); + + String accountName = getAccountName(); + +// Configuration testConfig = getConfiguration() + Configuration testConfig = new Configuration(getRawConfiguration()); + // addOAuthConfigs(fs, testConfig, accountName); + testFs.initialize(fs.getUri(), testConfig); + + Path reqPath = new Path(UUID.randomUUID().toString()); + + final String msg1 = "purple"; + final String msg2 = "yellow"; + int expectedFileLength = msg1.length() * 2; + + byte[] readBuffer = new byte[1024]; + + // create file with content "purplepurple" + try (FSDataOutputStream stream = testFs.create(reqPath)) { + stream.writeBytes(msg1); + stream.hflush(); + stream.writeBytes(msg1); + } + + // open file and verify content is "purplepurple" + try (FSDataInputStream stream = testFs.open(reqPath)) { + int bytesRead = stream.read(readBuffer, 0, readBuffer.length); + assertEquals(expectedFileLength, bytesRead); + String fileContent = new String(readBuffer, 0, bytesRead, StandardCharsets.UTF_8); + assertEquals(msg1 + msg1, fileContent); + } + + // overwrite file with content "yellowyellow" + try (FSDataOutputStream stream = testFs.create(reqPath)) { + stream.writeBytes(msg2); + stream.hflush(); + stream.writeBytes(msg2); + } + + // open file and verify content is "yellowyellow" + try (FSDataInputStream stream = testFs.open(reqPath)) { + int bytesRead = stream.read(readBuffer, 0, readBuffer.length); + assertEquals(expectedFileLength, bytesRead); + String fileContent = new String(readBuffer, 0, bytesRead, StandardCharsets.UTF_8); + assertEquals(msg2 + msg2, fileContent); + } + + // append to file so final content is "yellowyellowpurplepurple" + try (FSDataOutputStream stream = testFs.append(reqPath)) { + stream.writeBytes(msg1); + stream.hflush(); + stream.writeBytes(msg1); + } + + // open file and verify content is "yellowyellowpurplepurple" + try (FSDataInputStream stream = testFs.open(reqPath)) { + int bytesRead = stream.read(readBuffer, 0, readBuffer.length); + assertEquals(2 * expectedFileLength, bytesRead); + String fileContent = new String(readBuffer, 0, bytesRead, StandardCharsets.UTF_8); + assertEquals(msg2 + msg2 + msg1 + msg1, fileContent); + } + } + + + @Test + // Verify OAuth token provider and user-bound SAS provider are both configured and usable + public void testOAuthTokenProviderAndSASTokenFlow() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystem testFs = new AzureBlobFileSystem(); + + String accountName = getAccountName(); + Configuration testConfig = new Configuration(getRawConfiguration()); + addOAuthConfigs(fs, testConfig, accountName); + testFs.initialize(fs.getUri(), testConfig); + + // Verify AbfsConfiguration has an OAuth token provider configured + AbfsConfiguration abfsConfiguration = testFs.getAbfsStore().getAbfsConfiguration(); + + AccessTokenProvider tokenProvider = abfsConfiguration.getTokenProvider(); + assertNotNull(tokenProvider, "AccessTokenProvider must be configured for UserboundSASWithOAuth"); + + // Acquire an OAuth token and assert it is non-empty + AzureADToken token = tokenProvider.getToken(); + assertNotNull(token, "OAuth token must not be null"); + assertNotNull(token.getAccessToken(), "OAuth access token must not be null"); + assertFalse(token.getAccessToken().isEmpty(), "OAuth access token must not be empty"); + + // Verify SASTokenProvider for user-bound SAS is present and usable + SASTokenProvider sasProvider = abfsConfiguration.getSASTokenProviderForUserBoundSAS(); + assertNotNull(sasProvider, "SASTokenProvider for user-bound SAS must be configured"); + assertTrue(sasProvider instanceof MockUserBoundSASTokenProvider, + "Expected MockUserBoundSASTokenProvider to be used for tests"); + + // Request a SAS token and assert we get a non-empty result + String sasToken = sasProvider.getSASToken("abfsdrivercanaryhns.dfs.core.windows.net", "userbound", "/", SASTokenProvider.GET_PROPERTIES_OPERATION); + assertNotNull(sasToken, "SAS token must not be null"); + assertFalse(sasToken.isEmpty(), "SAS token must not be empty"); + } + + @Test + public void testOpenFile() throws Exception { + AzureBlobFileSystem fs = getFileSystem(getRawConfiguration()); //dont change + AzureBlobFileSystem testFs = new AzureBlobFileSystem(); + + AbfsConfiguration abfsConfig = fs.getAbfsStore().getAbfsConfiguration(); + + String accountName = getAccountName(); + + Configuration testConfig = new Configuration(getRawConfiguration()); //dont change + // Configuration testConfig = fs.getAbfsStore().getAbfsConfiguration().getRawConfiguration(); + + addOAuthConfigs(fs, testConfig, accountName); + testFs.initialize(fs.getUri(), testConfig); + +// System.out.print(testFs.getAbfsStore().getAbfsConfiguration()); +// System.out.print(abfsConfig); + + testFs.create(testPath).close(); + testFs.open(testPath); + testFs.getFileStatus(testPath); + } + + + @Test + public void testReadWriteFailsWithInvalidOAuthToken() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystem testFs = new AzureBlobFileSystem(); + + String accountName = getAccountName(); + + Configuration testConfig = new Configuration(getRawConfiguration()); + addOAuthConfigs(fs, testConfig, accountName); + + testFs.initialize(fs.getUri(), testConfig); + + // Create mock token provider with invalid token + AccessTokenProvider mockProvider = Mockito.mock(AccessTokenProvider.class); + AzureADToken mockToken = Mockito.mock(AzureADToken.class); + Mockito.when(mockToken.getAccessToken()).thenReturn("1234=abcd"); // Invalid token + Mockito.when(mockProvider.getToken()).thenReturn(mockToken); + + // Inject mock provider into AbfsClient + injectMockTokenProvider(testFs, mockProvider); + + intercept(AccessDeniedException.class, () -> {testFs.create(testPath); }); + } + + @Test + public void testReadWriteFailsWithInvalidSASToken() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystem testFs = new AzureBlobFileSystem(); + + String accountName = getAccountName(); + + Configuration testConfig = new Configuration(getRawConfiguration()); + addOAuthConfigs(fs, testConfig, accountName); + testConfig.unset("fs.azure.sas.token.provider.type"); + testConfig.set("fs.azure.sas.token.provider.type", "org.apache.hadoop.fs.azurebfs.extensions.MockInvalidSASTokenProvider"); + + testFs.initialize(fs.getUri(), testConfig); + + intercept(AccessDeniedException.class, () -> {testFs.create(testPath); }); + } + + @Test + public void testOperationWithValidAndExpiredSASToken() throws Exception { + // Set up the Configuration and FileSystem + AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystem testFs = new AzureBlobFileSystem(); + + Configuration testConfig = new Configuration(getRawConfiguration()); + addOAuthConfigs(fs, testConfig, getAccountName()); + + testFs.initialize(fs.getUri(), testConfig); + + // Get a real SAS token from the configured provider + AbfsConfiguration abfsConfig = testFs.getAbfsStore().getAbfsConfiguration(); + SASTokenProvider realSasProvider = abfsConfig.getSASTokenProviderForUserBoundSAS(); + assertNotNull(realSasProvider, "SASTokenProvider for user-bound SAS must be configured"); + String validSasToken = realSasProvider.getSASToken( + getAccountName(), + testFs.toString(), + String.valueOf(testPath), + SASTokenProvider.GET_PROPERTIES_OPERATION); + assertNotNull(validSasToken, "SAS token must not be null"); + assertFalse(validSasToken.isEmpty(), "SAS token must not be empty"); + + // 1. Operation should work with valid SAS token + // (No exception expected) + org.apache.hadoop.fs.Path path = testPath; + testFs.create(path); // Should succeed + + // 2. Now, modify the ske/se fields to be expired and inject a mock provider + String expiredDate = OffsetDateTime.now(ZoneOffset.UTC) + .minusDays(1) + .format(DateTimeFormatter.ISO_DATE_TIME); + String expiredSasToken = java.util.Arrays.stream(validSasToken.split("&")) + .map(kv -> { + String[] pair = kv.split("=", 2); + if (pair[0].equals("ske") || pair[0].equals("se")) { + return pair[0] + "=" + expiredDate; + } else { + return kv; + } + }) + .collect(java.util.stream.Collectors.joining("&")); + + // Create a mock SASTokenProvider that returns the expired SAS token + SASTokenProvider mockSasProvider = org.mockito.Mockito.mock( + SASTokenProvider.class); + org.mockito.Mockito.when( + mockSasProvider.getSASToken(org.mockito.Mockito.anyString(), + org.mockito.Mockito.anyString(), org.mockito.Mockito.anyString(), + org.mockito.Mockito.anyString())) + .thenReturn(expiredSasToken); + + // Inject the mock provider into the AbfsClient + injectMockSASTokenProvider(testFs, mockSasProvider); + + // Try a file operation and expect failure due to expired SAS token + intercept(AccessDeniedException.class, () -> {testFs.getFileStatus(path);}); + } + + // Helper to inject a mock SASTokenProvider into the AbfsClient + private void injectMockSASTokenProvider(AzureBlobFileSystem fs, SASTokenProvider provider) throws Exception { + Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField("abfsStore"); + abfsStoreField.setAccessible(true); + AzureBlobFileSystemStore store = (AzureBlobFileSystemStore) abfsStoreField.get(fs); + + Field abfsClientField = AzureBlobFileSystemStore.class.getDeclaredField("client"); + abfsClientField.setAccessible(true); + AbfsClient client = (AbfsClient) abfsClientField.get(store); + + // Use AbfsClient.class to get the field, not client.getClass() + java.lang.reflect.Field sasProviderField = AbfsClient.class.getDeclaredField("sasTokenProvider"); + sasProviderField.setAccessible(true); + sasProviderField.set(client, provider); + } +} From 5c9cfd5612187791d927ced5d7985d3b13f1db5e Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Fri, 7 Nov 2025 00:45:36 -0800 Subject: [PATCH 7/9] comments --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 37 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 33 +- .../azurebfs/constants/AbfsHttpConstants.java | 5 +- .../fs/azurebfs/services/AbfsBlobClient.java | 18 +- .../fs/azurebfs/services/AbfsClient.java | 38 +- .../azurebfs/services/AbfsClientHandler.java | 60 ++- .../fs/azurebfs/services/AbfsDfsClient.java | 38 +- .../hadoop/fs/azurebfs/services/AuthType.java | 2 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 19 + .../ITestAzureBlobFileSystemUserBoundSAS.java | 408 ++++++++++++++++++ ...reBlobFileSystemUserboundSASWithOAuth.java | 189 -------- .../hadoop/fs/azurebfs/IntegrationTests.java | 378 ---------------- .../fs/azurebfs/TestAccountConfiguration.java | 7 - .../constants/TestConfigurationKeys.java | 4 +- .../MockInvalidSASTokenProvider.java | 53 +++ .../MockUserBoundSASTokenProvider.java | 98 +++-- .../fs/azurebfs/services/ITestAbfsClient.java | 75 ++++ .../DelegationSASGenerator_Version_July5.java | 53 ++- .../fs/azurebfs/utils/SASGenerator.java | 2 +- 19 files changed, 809 insertions(+), 708 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserBoundSAS.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserboundSASWithOAuth.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/IntegrationTests.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockInvalidSASTokenProvider.java 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 50bd760e534b4..4f6ca58a9b661 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 @@ -1112,9 +1112,11 @@ public int getNumLeaseThreads() { } public boolean getCreateRemoteFileSystemDuringInitialization() { - // we do not support creating the filesystem when AuthType is SAS + // we do not support creating the filesystem when AuthType is SAS or UserboundSASWithOAuth return this.createRemoteFileSystemDuringInitialization - && this.getAuthType(this.accountName) != AuthType.SAS; + && this.getAuthType(this.accountName) != AuthType.SAS + && this.getAuthType(this.accountName) + != AuthType.UserboundSASWithOAuth; } public boolean getSkipUserGroupMetadataDuringInitialization() { @@ -1481,12 +1483,7 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio * @return sasTokenProvider object based on configurations provided * @throws AzureBlobFileSystemException */ - public SASTokenProvider getSASTokenProviderForUserBoundSAS() throws AzureBlobFileSystemException { - AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); - if (authType != AuthType.UserboundSASWithOAuth) { - throw new SASTokenProviderException(String.format( - "Invalid auth type: %s is being used, expecting user-bound SAS.", authType)); - } + public SASTokenProvider getUserBoundSASTokenProvider(AuthType authType) throws AzureBlobFileSystemException { try { Class customSasTokenProviderImplementation = @@ -1517,6 +1514,30 @@ public SASTokenProvider getSASTokenProviderForUserBoundSAS() throws AzureBlobFil } } + /** + * Returns both the AccessTokenProvider and the SASTokenProvider + * when auth type is UserboundSASWithOAuth. + * + * @return Object[] where: + * [0] = AccessTokenProvider + * [1] = SASTokenProvider + * @throws AzureBlobFileSystemException if provider initialization fails + */ + public Object[] getUserBoundSASBothTokenProviders() + throws AzureBlobFileSystemException { + AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, + AuthType.SharedKey); + if (authType != AuthType.UserboundSASWithOAuth) { + throw new SASTokenProviderException(String.format( + "Invalid auth type: %s is being used, expecting user-bound SAS.", + authType)); + } + + AccessTokenProvider tokenProvider = getTokenProvider(); + SASTokenProvider sasTokenProvider = getUserBoundSASTokenProvider(authType); + return new Object[]{tokenProvider, sasTokenProvider}; + } + public EncryptionContextProvider createEncryptionContextProvider() { try { String configKey = FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; 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 b90cacea0c07a..4d6f3c9fe9337 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 @@ -1741,19 +1741,20 @@ private void initializeClient(URI uri, String fileSystemName, } else if (authType == AuthType.SAS) { LOG.trace("Fetching SAS Token Provider"); sasTokenProvider = abfsConfiguration.getSASTokenProvider(); - } else if(authType == AuthType.UserboundSASWithOAuth){ + } else if (authType == AuthType.UserboundSASWithOAuth) { LOG.trace("Fetching SAS and OAuth Token Provider for user bound SAS"); AzureADAuthenticator.init(abfsConfiguration); - tokenProvider = abfsConfiguration.getTokenProvider(); + Object[] providers + = abfsConfiguration.getUserBoundSASBothTokenProviders(); + tokenProvider = (AccessTokenProvider) providers[0]; + sasTokenProvider = (SASTokenProvider) providers[1]; ExtensionHelper.bind(tokenProvider, uri, abfsConfiguration.getRawConfiguration()); - sasTokenProvider = abfsConfiguration.getSASTokenProviderForUserBoundSAS(); - } - else { + } else { LOG.trace("Fetching token provider"); tokenProvider = abfsConfiguration.getTokenProvider(); ExtensionHelper.bind(tokenProvider, uri, - abfsConfiguration.getRawConfiguration()); + abfsConfiguration.getRawConfiguration()); } // Encryption setup @@ -1777,21 +1778,11 @@ private void initializeClient(URI uri, String fileSystemName, } } - LOG.trace("Initializing AbfsClient for {}", baseUrl); - if(tokenProvider != null && sasTokenProvider != null){ - this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration, - tokenProvider, sasTokenProvider, encryptionContextProvider, - populateAbfsClientContext()); - } - else if (tokenProvider != null) { - this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration, - tokenProvider, encryptionContextProvider, - populateAbfsClientContext()); - } else { - this.clientHandler = new AbfsClientHandler(baseUrl, creds, abfsConfiguration, - sasTokenProvider, encryptionContextProvider, - populateAbfsClientContext()); - } + LOG.trace("Initializing AbfsClientHandler for {}", baseUrl); + this.clientHandler = new AbfsClientHandler(baseUrl, creds, + abfsConfiguration, + tokenProvider, sasTokenProvider, encryptionContextProvider, + populateAbfsClientContext()); this.setClient(getClientHandler().getClient()); LOG.trace("AbfsClient init complete"); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 5b836cc7452e3..64e2182974334 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -147,6 +147,7 @@ public final class AbfsHttpConstants { public static final String APPLICATION_JSON = "application/json"; public static final String APPLICATION_OCTET_STREAM = "application/octet-stream"; public static final String APPLICATION_XML = "application/xml"; + public static final String APPLICATION_X_WWW_FORM_URLENCODED = "application/x-www-form-urlencoded"; public static final String XMS_PROPERTIES_ENCODING_ASCII = "ISO-8859-1"; public static final String XMS_PROPERTIES_ENCODING_UNICODE = "UTF-8"; @@ -188,7 +189,7 @@ public enum ApiVersion { APR_10_2021("2021-04-10"), AUG_03_2023("2023-08-03"), NOV_04_2024("2024-11-04"), - JULY_05_2025("2025-07-05"); + JUL_05_2025("2025-07-05"); private final String xMsApiVersion; @@ -202,7 +203,7 @@ public String toString() { } public static ApiVersion getCurrentVersion() { - return JULY_05_2025; + return NOV_04_2024; } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java index d6ae0427b23b9..c62976dcffa39 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java @@ -188,7 +188,7 @@ public AbfsBlobClient(final URL baseUrl, final AccessTokenProvider tokenProvider, final EncryptionContextProvider encryptionContextProvider, final AbfsClientContext abfsClientContext) throws IOException { - super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, + super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, null, encryptionContextProvider, abfsClientContext, AbfsServiceType.BLOB); this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAzureAtomicRenameDirs() @@ -201,7 +201,21 @@ public AbfsBlobClient(final URL baseUrl, final SASTokenProvider sasTokenProvider, final EncryptionContextProvider encryptionContextProvider, final AbfsClientContext abfsClientContext) throws IOException { - super(baseUrl, sharedKeyCredentials, abfsConfiguration, sasTokenProvider, + super(baseUrl, sharedKeyCredentials, abfsConfiguration, null, sasTokenProvider, + encryptionContextProvider, abfsClientContext, AbfsServiceType.BLOB); + this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( + abfsConfiguration.getAzureAtomicRenameDirs() + .split(AbfsHttpConstants.COMMA))); + } + + public AbfsBlobClient(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final AccessTokenProvider tokenProvider, + final SASTokenProvider sasTokenProvider, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext) throws IOException { + super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, sasTokenProvider, encryptionContextProvider, abfsClientContext, AbfsServiceType.BLOB); this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAzureAtomicRenameDirs() 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 6470f7b45900c..5f304ad3ff0ea 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 @@ -337,32 +337,20 @@ private AbfsClient(final URL baseUrl, LOG.trace("primaryUserGroup is {}", this.primaryUserGroup); } - public AbfsClient(final URL baseUrl, - final SharedKeyCredentials sharedKeyCredentials, - final AbfsConfiguration abfsConfiguration, - final AccessTokenProvider tokenProvider, - final EncryptionContextProvider encryptionContextProvider, - final AbfsClientContext abfsClientContext, - final AbfsServiceType abfsServiceType) - throws IOException { - this(baseUrl, sharedKeyCredentials, abfsConfiguration, - encryptionContextProvider, abfsClientContext, abfsServiceType); - this.tokenProvider = tokenProvider; - } - - public AbfsClient(final URL baseUrl, - final SharedKeyCredentials sharedKeyCredentials, - final AbfsConfiguration abfsConfiguration, - final SASTokenProvider sasTokenProvider, - final EncryptionContextProvider encryptionContextProvider, - final AbfsClientContext abfsClientContext, - final AbfsServiceType abfsServiceType) - throws IOException { - this(baseUrl, sharedKeyCredentials, abfsConfiguration, - encryptionContextProvider, abfsClientContext, abfsServiceType); - this.sasTokenProvider = sasTokenProvider; - } + /** + * Constructs an AbfsClient instance with all authentication and configuration options. + * + * @param baseUrl The base URL for the ABFS endpoint. + * @param sharedKeyCredentials Shared key credentials for authentication. + * @param abfsConfiguration The ABFS configuration. + * @param tokenProvider The access token provider for OAuth authentication. + * @param sasTokenProvider The SAS token provider for SAS authentication. + * @param encryptionContextProvider The encryption context provider. + * @param abfsClientContext The client context + * @param abfsServiceType The ABFS service type (e.g., Blob, DFS). + * @throws IOException if initialization fails. + */ public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java index 996182dad245c..ce1106666d305 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java @@ -47,39 +47,22 @@ public class AbfsClientHandler implements Closeable { private final AbfsDfsClient dfsAbfsClient; private final AbfsBlobClient blobAbfsClient; - public AbfsClientHandler(final URL baseUrl, - final SharedKeyCredentials sharedKeyCredentials, - final AbfsConfiguration abfsConfiguration, - final AccessTokenProvider tokenProvider, - final EncryptionContextProvider encryptionContextProvider, - final AbfsClientContext abfsClientContext) throws IOException { - this.dfsAbfsClient = createDfsClient(baseUrl, sharedKeyCredentials, - abfsConfiguration, tokenProvider, null, encryptionContextProvider, - abfsClientContext); - this.blobAbfsClient = createBlobClient(baseUrl, sharedKeyCredentials, - abfsConfiguration, tokenProvider, null, encryptionContextProvider, - abfsClientContext); - initServiceType(abfsConfiguration); - } - - public AbfsClientHandler(final URL baseUrl, - final SharedKeyCredentials sharedKeyCredentials, - final AbfsConfiguration abfsConfiguration, - final SASTokenProvider sasTokenProvider, - final EncryptionContextProvider encryptionContextProvider, - final AbfsClientContext abfsClientContext) throws IOException { - // This will initialize the default and ingress service types. - // This is needed before creating the clients so that we can do cache warmup - // only for default client. - initServiceType(abfsConfiguration); - this.dfsAbfsClient = createDfsClient(baseUrl, sharedKeyCredentials, - abfsConfiguration, null, sasTokenProvider, encryptionContextProvider, - abfsClientContext); - this.blobAbfsClient = createBlobClient(baseUrl, sharedKeyCredentials, - abfsConfiguration, null, sasTokenProvider, encryptionContextProvider, - abfsClientContext); - } + /** + * Constructs an AbfsClientHandler instance. + * + * Initializes the default and ingress service types from the provided configuration, + * then creates both DFS and Blob clients using the given params + * + * @param baseUrl the base URL for the file system. + * @param sharedKeyCredentials credentials for shared key authentication. + * @param abfsConfiguration the ABFS configuration. + * @param tokenProvider the access token provider, may be null. + * @param sasTokenProvider the SAS token provider, may be null. + * @param encryptionContextProvider the encryption context provider + * @param abfsClientContext the ABFS client context. + * @throws IOException if client creation or URL conversion fails. + */ public AbfsClientHandler(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, @@ -215,12 +198,21 @@ private AbfsBlobClient createBlobClient(final URL baseUrl, final EncryptionContextProvider encryptionContextProvider, final AbfsClientContext abfsClientContext) throws IOException { URL blobUrl = changeUrlFromDfsToBlob(baseUrl); - if (tokenProvider != null) { + if (tokenProvider != null && sasTokenProvider != null) { + LOG.debug( + "Creating AbfsBlobClient with both access token provider and SAS token provider using the URL: {}", + blobUrl); + return new AbfsBlobClient(blobUrl, creds, abfsConfiguration, + tokenProvider, sasTokenProvider, encryptionContextProvider, + abfsClientContext); + } + else if (tokenProvider != null) { LOG.debug("Creating AbfsBlobClient with access token provider using the URL: {}", blobUrl); return new AbfsBlobClient(blobUrl, creds, abfsConfiguration, tokenProvider, encryptionContextProvider, abfsClientContext); - } else { + } + else { LOG.debug("Creating AbfsBlobClient with SAS token provider using the URL: {}", blobUrl); return new AbfsBlobClient(blobUrl, creds, abfsConfiguration, sasTokenProvider, encryptionContextProvider, 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 2bf89f1ca4357..fa5282a6bf818 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 @@ -154,26 +154,60 @@ */ public class AbfsDfsClient extends AbfsClient { + /** + * Constructs an AbfsDfsClient using OAuth token provider. + * + * @param baseUrl the base URL for the DFS endpoint + * @param sharedKeyCredentials the shared key credentials + * @param abfsConfiguration the ABFS configuration + * @param tokenProvider the OAuth access token provider + * @param encryptionContextProvider the encryption context provider + * @param abfsClientContext the ABFS client context + * @throws IOException if an I/O error occurs + */ public AbfsDfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, final AccessTokenProvider tokenProvider, final EncryptionContextProvider encryptionContextProvider, final AbfsClientContext abfsClientContext) throws IOException { - super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, + super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, null, encryptionContextProvider, abfsClientContext, AbfsServiceType.DFS); } + /** + * Constructs an AbfsDfsClient using SAS token provider. + * + * @param baseUrl the base URL for the DFS endpoint + * @param sharedKeyCredentials the shared key credentials + * @param abfsConfiguration the ABFS configuration + * @param sasTokenProvider the SAS token provider + * @param encryptionContextProvider the encryption context provider + * @param abfsClientContext the ABFS client context + * @throws IOException if an I/O error occurs + */ public AbfsDfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, final SASTokenProvider sasTokenProvider, final EncryptionContextProvider encryptionContextProvider, final AbfsClientContext abfsClientContext) throws IOException { - super(baseUrl, sharedKeyCredentials, abfsConfiguration, sasTokenProvider, + super(baseUrl, sharedKeyCredentials, abfsConfiguration, null, sasTokenProvider, encryptionContextProvider, abfsClientContext, AbfsServiceType.DFS); } + /** + * Constructs an AbfsDfsClient using both OAuth and SAS token providers. + * + * @param baseUrl the base URL for the DFS endpoint + * @param sharedKeyCredentials the shared key credentials + * @param abfsConfiguration the ABFS configuration + * @param tokenProvider the OAuth access token provider + * @param sasTokenProvider the SAS token provider + * @param encryptionContextProvider the encryption context provider + * @param abfsClientContext the ABFS client context + * @throws IOException if an I/O error occurs + */ public AbfsDfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java index 142b266fb4664..cda8f686453a7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java @@ -25,5 +25,5 @@ public enum AuthType { OAuth, Custom, SAS, - UserboundSASWithOAuth + UserboundSASWithOAuth } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index c35b76e1a7368..d9d56fa7a90da 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -199,6 +199,8 @@ public void setup() throws Exception { // Only live account without namespace support can run ABFS&WASB // compatibility tests if (!isIPAddress && (abfsConfig.getAuthType(accountName) != AuthType.SAS) + && (abfsConfig.getAuthType(accountName) + != AuthType.UserboundSASWithOAuth) && !abfs.getIsNamespaceEnabled(getTestTracingContext( getFileSystem(), false))) { final URI wasbUri = new URI( @@ -325,6 +327,20 @@ protected void createFilesystemWithTestFileForSASTests(Path testPath) throws Exc } } + /** + * Create a filesystem for user bound SAS tests using the SharedKey authentication. + * + * @throws Exception + */ + protected void createFilesystemForUserBoundSASTests() throws Exception{ + try (AzureBlobFileSystem tempFs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig)){ + ContractTestUtils.assertPathExists(tempFs, "This path should exist", + new Path("/")); + abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.UserboundSASWithOAuth.name()); + usingFilesystemForSASTests = true; + } + } + public AzureBlobFileSystem getFileSystem() throws IOException { return abfs; } @@ -587,6 +603,9 @@ protected void assumeValidAuthConfigsPresent() { assumeThat(currentAuthType). as("SAS Based Authentication Not Allowed For Integration Tests"). isNotEqualTo(AuthType.SAS); + assumeThat(currentAuthType). + as("User-bound SAS Based Authentication Not Allowed For Integration Tests"). + isNotEqualTo(AuthType.UserboundSASWithOAuth); if (currentAuthType == AuthType.SharedKey) { assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_ACCOUNT_KEY); } else { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserBoundSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserBoundSAS.java new file mode 100644 index 0000000000000..6a8b2783971e6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserBoundSAS.java @@ -0,0 +1,408 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.AccessDeniedException; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; +import org.apache.hadoop.fs.azurebfs.extensions.MockInvalidSASTokenProvider; +import org.apache.hadoop.fs.azurebfs.extensions.MockUserBoundSASTokenProvider; +import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AuthType; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_END_USER_OBJECT_ID; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assumptions.assumeThat; + +/** + * Integration tests for AzureBlobFileSystem using User-Bound SAS and OAuth. + * Covers scenarios for token provider configuration, SAS token validity, and basic file operations. + */ +public class ITestAzureBlobFileSystemUserBoundSAS extends AbstractAbfsIntegrationTest { + + private static Path testPath = new Path("/test.txt"); + + private static final String TEST_OBJECT_ID = "123456789"; + + private static final String InvalidOAuthToken = "InvalidOAuthTokenValue"; + + /** + * Constructor. Ensures tests run with SharedKey authentication. + * @throws Exception if auth type is not SharedKey + */ + protected ITestAzureBlobFileSystemUserBoundSAS() throws Exception { + assumeThat(this.getAuthType()).isEqualTo(AuthType.SharedKey); + } + + /** + * Sets up the test environment and configures the AbfsConfiguration for user-bound SAS tests. + * @throws Exception if setup fails + */ + @BeforeEach + @Override + public void setup() throws Exception { + AbfsConfiguration abfsConfig = this.getConfiguration(); + String accountName = getAccountName(); + + Boolean isHNSEnabled = abfsConfig.getBoolean( + TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); + + if (!isHNSEnabled) { + assumeBlobServiceType(); + } + + createFilesystemForUserBoundSASTests(); + super.setup(); + + // Set all required configs on the raw configuration + abfsConfig.set( + FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID + "." + accountName, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); + abfsConfig.set(FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); + abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + "." + accountName, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID)); + abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID)); + abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + "." + accountName, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET)); + abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET)); + abfsConfig.set(FS_AZURE_TEST_END_USER_OBJECT_ID, + abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); + abfsConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, + MockUserBoundSASTokenProvider.class.getName()); + } + + + /** + * Injects a mock AccessTokenProvider into the AbfsClient of the given filesystem. + * @param fs AzureBlobFileSystem instance + * @param mockProvider AccessTokenProvider to inject + * @throws Exception if reflection fails + */ + private void injectMockTokenProvider(AzureBlobFileSystem fs, + AccessTokenProvider mockProvider) throws Exception { + Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField( + "abfsStore"); + abfsStoreField.setAccessible(true); + AzureBlobFileSystemStore store + = (AzureBlobFileSystemStore) abfsStoreField.get(fs); + + Field abfsClientField = AzureBlobFileSystemStore.class.getDeclaredField( + "client"); + abfsClientField.setAccessible(true); + AbfsClient client = (AbfsClient) abfsClientField.get(store); + + Field tokenProviderField = AbfsClient.class.getDeclaredField( + "tokenProvider"); + tokenProviderField.setAccessible(true); + tokenProviderField.set(client, mockProvider); + } + + /** + * Helper to create a new AzureBlobFileSystem instance for tests. + * @return AzureBlobFileSystem instance + * @throws RuntimeException if creation fails + */ + private AzureBlobFileSystem createTestFileSystem() throws RuntimeException { + try { + return (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Test that file creation fails when the end user object ID does not match the service principal object ID. + * @throws Exception if test fails + */ + @Test + public void testShouldFailWhenSduoidMismatchesServicePrincipalId() + throws Exception { + this.getConfiguration().set(FS_AZURE_TEST_END_USER_OBJECT_ID, TEST_OBJECT_ID); + AzureBlobFileSystem testFs = createTestFileSystem(); + intercept(AccessDeniedException.class, + () -> { + testFs.create(testPath); + }); + } + + /** + * Verifies that both OAuth token provider and user-bound SAS token provider are configured and usable. + * @throws Exception if test fails + */ + @Test + public void testOAuthTokenProviderAndSASTokenFlow() throws Exception { + AzureBlobFileSystem testFs = createTestFileSystem(); + + AbfsConfiguration abfsConfiguration = testFs.getAbfsStore() + .getAbfsConfiguration(); + + // Verify AbfsConfiguration has an OAuth token provider configured + AccessTokenProvider tokenProvider = abfsConfiguration.getTokenProvider(); + assertNotNull(tokenProvider, + "AccessTokenProvider must be configured for UserboundSASWithOAuth"); + + // Acquire an OAuth token and assert it is non-empty + AzureADToken token = tokenProvider.getToken(); + assertNotNull(token, "OAuth token must not be null"); + assertNotNull(token.getAccessToken(), + "OAuth access token must not be null"); + assertFalse(token.getAccessToken().isEmpty(), + "OAuth access token must not be empty"); + + // Verify AbfsConfiguration has an SASTokenProvider configured + SASTokenProvider sasProvider + = abfsConfiguration.getUserBoundSASTokenProvider(AuthType.UserboundSASWithOAuth); + assertNotNull(sasProvider, + "SASTokenProvider for user-bound SAS must be configured"); + assertInstanceOf(MockUserBoundSASTokenProvider.class, sasProvider, + "Expected MockUserBoundSASTokenProvider to be used for tests"); + + // Request a SAS token and assert we get a non-empty result + String sasToken = sasProvider.getSASToken( + "abfsdrivercanaryhns.dfs.core.windows.net", "userbound", "/", + SASTokenProvider.GET_PROPERTIES_OPERATION); + assertNotNull(sasToken, "SAS token must not be null"); + assertFalse(sasToken.isEmpty(), "SAS token must not be empty"); + } + + /* + * Tests listing and deleting files under an implicit directory + */ + @Test + public void testOperationsForImplicitPaths() throws Exception { + AzureBlobFileSystem fs = createTestFileSystem(); + assumeBlobServiceType(); + + AbfsBlobClient client = (AbfsBlobClient) getFileSystem().getAbfsClient(); + + Path file1 = new Path("/testDir/dir1/file1"); + Path file2 = new Path("/testDir/dir1/file2"); + Path implicitDir = file1.getParent(); + + createAzCopyFolder(implicitDir); + createAzCopyFile(file1); + createAzCopyFile(file2); + + AbfsRestOperation listOp = client.listPath( + implicitDir.toString(), + false, + 2, + null, + getTestTracingContext(fs, false), + null).getOp(); + + List listedEntries = + listOp.getResult().getListResultSchema().paths(); + + assertNotNull(listedEntries, "List result should not be null"); + assertEquals(2, listedEntries.size(), "Expected exactly two files under implicit directory"); + + client.deletePath( + implicitDir.toString(), + true, + "", + getTestTracingContext(fs, false)); + + assertFalse(fs.exists(file1), "File1 should not exist after deletion"); + assertFalse(fs.exists(file2), "File2 should not exist after deletion"); + assertFalse(fs.exists(implicitDir), "Implicit directory should be deleted"); + } + + + /** + * Tests basic file operations (create, open, write, read, list, delete) using user-bound SAS. + * @throws Exception if test fails + */ + @Test + public void testBasicOperations() throws Exception { + AzureBlobFileSystem testFs = createTestFileSystem(); + + // 1. Create file + testFs.create(testPath).close(); + + // 2. Open file + testFs.open(testPath).close(); + + // 3. Get file status + testFs.getFileStatus(testPath); + + // 4. Write to file (overwrite) + try (FSDataOutputStream out = testFs.create(testPath, true)) { + out.writeUTF("hello"); + } + + // 5. Read from file + try (FSDataInputStream in = testFs.open(testPath)) { + String content = in.readUTF(); + assertEquals("hello", content); + } + + // 6. List parent directory + FileStatus[] files = testFs.listStatus(testPath.getParent()); + assertTrue(files.length > 0); + + // 7. Check file existence + assertTrue(testFs.exists(testPath)); + + // 9. Delete file + assertTrue(testFs.delete(testPath, false)); + assertFalse(testFs.exists(testPath)); + } + + /** + * Test that file creation fails when an invalid OAuth token is used. + * @throws Exception if test fails + */ + @Test + public void testCreateFailsWithInvalidOAuthToken() throws Exception { + AzureBlobFileSystem testFs = createTestFileSystem(); + + // Create mock token provider with invalid token + AccessTokenProvider mockProvider = Mockito.mock(AccessTokenProvider.class); + AzureADToken mockToken = Mockito.mock(AzureADToken.class); + Mockito.when(mockToken.getAccessToken()).thenReturn(InvalidOAuthToken); + Mockito.when(mockProvider.getToken()).thenReturn(mockToken); + + // Inject mock provider into AbfsClient + injectMockTokenProvider(testFs, mockProvider); + + intercept(AccessDeniedException.class, () -> {testFs.create(testPath);}); + } + + /** + * Test that file creation fails when an invalid SAS token is used. + * @throws Exception if test fails + */ + @Test + public void testGPSFailsWithInvalidSASToken() throws Exception { + AbfsConfiguration abfsConfig = this.getConfiguration(); + abfsConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, + MockInvalidSASTokenProvider.class.getName()); + AzureBlobFileSystem invalidSASTokenFs = createTestFileSystem(); + intercept(AccessDeniedException.class, + () -> {invalidSASTokenFs.create(testPath);}); + } + + + /** + * Test file operations with a valid and then expired SAS token. + * Verifies that operations succeed with a valid token and fail with an expired token. + * @throws Exception if test fails + */ + @Test + public void testOperationWithValidAndExpiredSASToken() throws Exception { + AzureBlobFileSystem testFs = createTestFileSystem(); + + // Get a real SAS token from the configured provider + AbfsConfiguration abfsConfig = testFs.getAbfsStore().getAbfsConfiguration(); + SASTokenProvider realSasProvider + = abfsConfig.getUserBoundSASTokenProvider(AuthType.UserboundSASWithOAuth); + assertNotNull(realSasProvider, + "SASTokenProvider for user-bound SAS must be configured"); + String validSasToken = realSasProvider.getSASToken( + getAccountName(), + testFs.toString(), + String.valueOf(testPath), + SASTokenProvider.CREATE_FILE_OPERATION); + assertNotNull(validSasToken, "SAS token must not be null"); + assertFalse(validSasToken.isEmpty(), "SAS token must not be empty"); + + // Operation should work with valid SAS token + testFs.create(testPath); // Should succeed + + // Modify the ske/se fields to be expired and inject a mock provider + String expiredDate = OffsetDateTime.now(ZoneOffset.UTC) + .minusDays(1) + .format(DateTimeFormatter.ISO_DATE_TIME); + String expiredSasToken = Arrays.stream(validSasToken.split("&")) + .map(kv -> { + String[] pair = kv.split("=", 2); + if (pair[0].equals("ske") || pair[0].equals("se")) { + return pair[0] + "=" + expiredDate; + } else { + return kv; + } + }) + .collect(Collectors.joining("&")); + + // Create a mock SASTokenProvider that returns the expired SAS token + SASTokenProvider mockSasProvider = Mockito.mock( + SASTokenProvider.class); + Mockito.when( + mockSasProvider.getSASToken(Mockito.anyString(), + Mockito.anyString(), Mockito.anyString(), + Mockito.anyString())) + .thenReturn(expiredSasToken); + + // Inject the mock provider into the AbfsClient + injectMockSASTokenProvider(testFs, mockSasProvider); + + // Try a file operation and expect failure due to expired SAS token + intercept(AccessDeniedException.class, () -> {testFs.getFileStatus(testPath);}); + } + + // Helper method to inject a mock SASTokenProvider into the AbfsClient + private void injectMockSASTokenProvider(AzureBlobFileSystem fs, SASTokenProvider provider) throws Exception { + Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField("abfsStore"); + abfsStoreField.setAccessible(true); + AzureBlobFileSystemStore store = (AzureBlobFileSystemStore) abfsStoreField.get(fs); + + Field abfsClientField = AzureBlobFileSystemStore.class.getDeclaredField("client"); + abfsClientField.setAccessible(true); + AbfsClient client = (AbfsClient) abfsClientField.get(store); + + Field sasProviderField = AbfsClient.class.getDeclaredField("sasTokenProvider"); + sasProviderField.setAccessible(true); + sasProviderField.set(client, provider); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserboundSASWithOAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserboundSASWithOAuth.java deleted file mode 100644 index 4449d6faa5e67..0000000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserboundSASWithOAuth.java +++ /dev/null @@ -1,189 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.AccessDeniedException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Hashtable; -import java.util.List; -import java.util.UUID; - -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mockito; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileAlreadyExistsException; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; -import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; -import org.apache.hadoop.fs.azurebfs.extensions.MockUserBoundSASTokenProvider; -import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; -import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; -import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.AuthType; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.permission.AclEntry; -import org.apache.hadoop.fs.permission.AclEntryScope; -import org.apache.hadoop.fs.permission.AclStatus; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.security.AccessControlException; - -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; -import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH; -import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_FILE_ALREADY_EXISTS; -import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; -import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; -import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; -import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; -import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; -import static org.apache.hadoop.fs.permission.AclEntryType.GROUP; -import static org.apache.hadoop.fs.permission.AclEntryType.USER; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.assertj.core.api.Assumptions.assumeThat; - -import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; -import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; - -/** - * Test Perform Authorization Check operation for UserboundSASWithOAuth auth type - */ -public class ITestAzureBlobFileSystemUserboundSASWithOAuth extends AbstractAbfsIntegrationTest { - private static final String TEST_GROUP = UUID.randomUUID().toString(); - - private static final Logger LOG = - LoggerFactory.getLogger(ITestAzureBlobFileSystemUserboundSASWithOAuth.class); - - private boolean isHNSEnabled; - - public ITestAzureBlobFileSystemUserboundSASWithOAuth() throws Exception { - // These tests rely on specific settings in azure-auth-keys.xml: - String sasProvider = getRawConfiguration().get( - FS_AZURE_SAS_TOKEN_PROVIDER_TYPE); - assumeThat( - MockUserBoundSASTokenProvider.class.getCanonicalName()).isEqualTo( - sasProvider); - assumeThat(getRawConfiguration().get( - TestConfigurationKeys.FS_AZURE_TEST_APP_ID)).isNotNull(); - assumeThat(getRawConfiguration().get( - TestConfigurationKeys.FS_AZURE_TEST_APP_SECRET)).isNotNull(); - assumeThat(getRawConfiguration().get( - TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID)).isNotNull(); - assumeThat(getRawConfiguration().get( - TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID)).isNotNull(); - //todo: check if this would still be relevant for user bound SAS testing -// The test uses shared key to create a random filesystem and then creates another -// instance of this filesystem using SAS+OAuth authorization. - //assumeThat(this.getAuthType()).isEqualTo(AuthType.SharedKey); - } - - @BeforeEach - @Override - public void setup() throws Exception { - isHNSEnabled = this.getConfiguration().getBoolean( - TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); - if (!isHNSEnabled) { - assumeBlobServiceType(); - } - createFilesystemForSASTests(); - super.setup(); - } - - @Test - // FileSystemProperties are not supported by delegation SAS (hence user-bound SAS too) and should throw exception - public void testSetFileSystemProperties() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Hashtable - properties = new Hashtable<>(); - properties.put("FileSystemProperties", "true"); - TracingContext tracingContext = getTestTracingContext(fs, true); - assertThrows(IOException.class, () -> fs.getAbfsStore() - .setFilesystemProperties(properties, tracingContext)); - assertThrows(IOException.class, - () -> fs.getAbfsStore().getFilesystemProperties(tracingContext)); - } - - - @Test - public void testSignatureMaskOnExceptionMessage() throws Exception { - intercept(IOException.class, "sig=XXXX", - () -> getFileSystem().getAbfsClient() - .renamePath("testABC/test.xt", "testABC/abc.txt", - null, getTestTracingContext(getFileSystem(), false), - null, false)); - } - - - @Test - public void testSASQuesMarkPrefix() throws Exception { - AbfsConfiguration testConfig = this.getConfiguration(); - // the SAS Token Provider is changed - testConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, "org.apache.hadoop.fs.azurebfs.extensions.MockWithPrefixSASTokenProvider"); - - AzureBlobFileSystem testFs = (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()); - Path testFile = new Path("/testSASPrefixQuesMark"); - - // the creation of this filesystem should work correctly even when a SAS Token is generated with a ? prefix - testFs.create(testFile).close(); - } - - @Test - // Verify OAuth token provider and user-bound SAS provider are both configured and usable - //CURRENTLY ONLY WORKING WITH THE REMOVED (BUT OPTIONAL) UDK PARAM - public void testOAuthTokenProviderAndSASTokenFlow() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - - // Verify AbfsConfiguration has an OAuth token provider configured - AbfsConfiguration config = fs.getAbfsStore().getAbfsConfiguration(); - config.set("fs.azure.account.auth.type", "UserboundSASWithOAuth"); - - AccessTokenProvider tokenProvider = config.getTokenProvider(); - assertNotNull(tokenProvider, "AccessTokenProvider must be configured for UserboundSASWithOAuth"); - - // Acquire an OAuth token and assert it is non-empty - AzureADToken token = tokenProvider.getToken(); - assertNotNull(token, "OAuth token must not be null"); - assertNotNull(token.getAccessToken(), "OAuth access token must not be null"); - assertFalse(token.getAccessToken().isEmpty(), "OAuth access token must not be empty"); - - // Verify SASTokenProvider for user-bound SAS is present and usable - SASTokenProvider sasProvider = config.getSASTokenProviderForUserBoundSAS(); - assertNotNull(sasProvider, "SASTokenProvider for user-bound SAS must be configured"); - assertTrue(sasProvider instanceof MockUserBoundSASTokenProvider, - "Expected MockUserBoundSASTokenProvider to be used for tests"); - - // Request a SAS token and assert we get a non-empty result - String sasToken = sasProvider.getSASToken(getAccountName(), getFileSystemName(), "/", SASTokenProvider.GET_PROPERTIES_OPERATION); - assertNotNull(sasToken, "SAS token must not be null"); - assertFalse(sasToken.isEmpty(), "SAS token must not be empty"); - } -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/IntegrationTests.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/IntegrationTests.java deleted file mode 100644 index eaecceb4170fa..0000000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/IntegrationTests.java +++ /dev/null @@ -1,378 +0,0 @@ -package org.apache.hadoop.fs.azurebfs; - -import java.lang.reflect.Field; -import java.nio.charset.StandardCharsets; -import java.nio.file.AccessDeniedException; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.format.DateTimeFormatter; -import java.util.UUID; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mockito; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; -import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider; -import org.apache.hadoop.fs.azurebfs.extensions.MockUserBoundSASTokenProvider; -import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; -import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; -import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AuthType; - -import static java.time.temporal.ChronoUnit.DAYS; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_END_USER_OBJECT_ID; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.assertj.core.api.Assumptions.assumeThat; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertTrue; - -public class IntegrationTests extends AbstractAbfsIntegrationTest { - - private static Path testPath = new Path("/test.txt"); - private static final String TEST_OBJECT_ID = "123456789"; - - protected IntegrationTests() throws Exception { - //todo: add later -// String sasProvider = getRawConfiguration().get(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE); -// assumeThat(MockUserBoundSASTokenProvider.class.getCanonicalName()).isEqualTo(sasProvider); - - assumeThat(this.getAuthType()).isEqualTo(AuthType.SharedKey); - } - - @BeforeEach - @Override - public void setup() throws Exception { - Boolean isHNSEnabled = this.getConfiguration().getBoolean( - TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); - if (!isHNSEnabled) { - assumeBlobServiceType(); - } - - //todo: thisi is changed - createFilesystemForUserBoundSASTests(); - super.setup(); - } - // TEST FOR SAS- HOW DOES CREATE CONTAINER PASS!!! - - - // Common helper to inject a mock token provider into AbfsClient - private void injectMockTokenProvider(AzureBlobFileSystem fs, AccessTokenProvider mockProvider) throws Exception { - Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField("abfsStore"); - abfsStoreField.setAccessible(true); - AzureBlobFileSystemStore store = (AzureBlobFileSystemStore) abfsStoreField.get(fs); - - Field abfsClientField = AzureBlobFileSystemStore.class.getDeclaredField("client"); - abfsClientField.setAccessible(true); - AbfsClient client = (AbfsClient) abfsClientField.get(store); - - Field tokenProviderField = AbfsClient.class.getDeclaredField("tokenProvider"); - tokenProviderField.setAccessible(true); - tokenProviderField.set(client, mockProvider); - } - - private void addOAuthConfigs(AzureBlobFileSystem fs, Configuration testConfig, String accountName) { - AbfsConfiguration abfsConfig = fs.getAbfsStore().getAbfsConfiguration(); - - testConfig.set(FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID + "." +accountName, - abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); - testConfig.set(FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID, - abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); - - testConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + "." + accountName, - abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID)); - testConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID, - abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID)); - - testConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET+ "." + accountName, - abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET)); - testConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET, - abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET)); - - // Set a different SDUOID - testConfig.set(FS_AZURE_END_USER_OBJECT_ID, abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); -// testConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, -// "org.apache.hadoop.fs.azurebfs.extensions.MockUserBoundSASTokenProvider"); - } - - - @Test - public void testShouldFailWhenSduoidMismatchesServicePrincipalId() - //WIHOUT ABSTRACT INTEGRATION CLASS - throws Exception { - // Arrange - //final AzureBlobFileSystem fs = this.getFileSystem(); - AzureBlobFileSystem fs = getFileSystem(); - //AzureBlobFileSystem fs = createFs(); - AzureBlobFileSystem testFs = new AzureBlobFileSystem(); - - String accountName = getAccountName(); - -// Configuration testConfig = getConfiguration() - Configuration testConfig = new Configuration(getRawConfiguration()); - addOAuthConfigs(fs, testConfig, accountName); - - testConfig.set(FS_AZURE_END_USER_OBJECT_ID, TEST_OBJECT_ID); - testFs.initialize(fs.getUri(), testConfig); - intercept(AccessDeniedException.class, - ()-> { - testFs.create(testPath); - }); - } - - @Test - public void testReadAndWrite() throws Exception { - AzureBlobFileSystem fs = getFileSystem(); - AzureBlobFileSystem testFs = new AzureBlobFileSystem(); - - String accountName = getAccountName(); - -// Configuration testConfig = getConfiguration() - Configuration testConfig = new Configuration(getRawConfiguration()); - // addOAuthConfigs(fs, testConfig, accountName); - testFs.initialize(fs.getUri(), testConfig); - - Path reqPath = new Path(UUID.randomUUID().toString()); - - final String msg1 = "purple"; - final String msg2 = "yellow"; - int expectedFileLength = msg1.length() * 2; - - byte[] readBuffer = new byte[1024]; - - // create file with content "purplepurple" - try (FSDataOutputStream stream = testFs.create(reqPath)) { - stream.writeBytes(msg1); - stream.hflush(); - stream.writeBytes(msg1); - } - - // open file and verify content is "purplepurple" - try (FSDataInputStream stream = testFs.open(reqPath)) { - int bytesRead = stream.read(readBuffer, 0, readBuffer.length); - assertEquals(expectedFileLength, bytesRead); - String fileContent = new String(readBuffer, 0, bytesRead, StandardCharsets.UTF_8); - assertEquals(msg1 + msg1, fileContent); - } - - // overwrite file with content "yellowyellow" - try (FSDataOutputStream stream = testFs.create(reqPath)) { - stream.writeBytes(msg2); - stream.hflush(); - stream.writeBytes(msg2); - } - - // open file and verify content is "yellowyellow" - try (FSDataInputStream stream = testFs.open(reqPath)) { - int bytesRead = stream.read(readBuffer, 0, readBuffer.length); - assertEquals(expectedFileLength, bytesRead); - String fileContent = new String(readBuffer, 0, bytesRead, StandardCharsets.UTF_8); - assertEquals(msg2 + msg2, fileContent); - } - - // append to file so final content is "yellowyellowpurplepurple" - try (FSDataOutputStream stream = testFs.append(reqPath)) { - stream.writeBytes(msg1); - stream.hflush(); - stream.writeBytes(msg1); - } - - // open file and verify content is "yellowyellowpurplepurple" - try (FSDataInputStream stream = testFs.open(reqPath)) { - int bytesRead = stream.read(readBuffer, 0, readBuffer.length); - assertEquals(2 * expectedFileLength, bytesRead); - String fileContent = new String(readBuffer, 0, bytesRead, StandardCharsets.UTF_8); - assertEquals(msg2 + msg2 + msg1 + msg1, fileContent); - } - } - - - @Test - // Verify OAuth token provider and user-bound SAS provider are both configured and usable - public void testOAuthTokenProviderAndSASTokenFlow() throws Exception { - AzureBlobFileSystem fs = getFileSystem(); - AzureBlobFileSystem testFs = new AzureBlobFileSystem(); - - String accountName = getAccountName(); - Configuration testConfig = new Configuration(getRawConfiguration()); - addOAuthConfigs(fs, testConfig, accountName); - testFs.initialize(fs.getUri(), testConfig); - - // Verify AbfsConfiguration has an OAuth token provider configured - AbfsConfiguration abfsConfiguration = testFs.getAbfsStore().getAbfsConfiguration(); - - AccessTokenProvider tokenProvider = abfsConfiguration.getTokenProvider(); - assertNotNull(tokenProvider, "AccessTokenProvider must be configured for UserboundSASWithOAuth"); - - // Acquire an OAuth token and assert it is non-empty - AzureADToken token = tokenProvider.getToken(); - assertNotNull(token, "OAuth token must not be null"); - assertNotNull(token.getAccessToken(), "OAuth access token must not be null"); - assertFalse(token.getAccessToken().isEmpty(), "OAuth access token must not be empty"); - - // Verify SASTokenProvider for user-bound SAS is present and usable - SASTokenProvider sasProvider = abfsConfiguration.getSASTokenProviderForUserBoundSAS(); - assertNotNull(sasProvider, "SASTokenProvider for user-bound SAS must be configured"); - assertTrue(sasProvider instanceof MockUserBoundSASTokenProvider, - "Expected MockUserBoundSASTokenProvider to be used for tests"); - - // Request a SAS token and assert we get a non-empty result - String sasToken = sasProvider.getSASToken("abfsdrivercanaryhns.dfs.core.windows.net", "userbound", "/", SASTokenProvider.GET_PROPERTIES_OPERATION); - assertNotNull(sasToken, "SAS token must not be null"); - assertFalse(sasToken.isEmpty(), "SAS token must not be empty"); - } - - @Test - public void testOpenFile() throws Exception { - AzureBlobFileSystem fs = getFileSystem(getRawConfiguration()); //dont change - AzureBlobFileSystem testFs = new AzureBlobFileSystem(); - - AbfsConfiguration abfsConfig = fs.getAbfsStore().getAbfsConfiguration(); - - String accountName = getAccountName(); - - Configuration testConfig = new Configuration(getRawConfiguration()); //dont change - // Configuration testConfig = fs.getAbfsStore().getAbfsConfiguration().getRawConfiguration(); - - addOAuthConfigs(fs, testConfig, accountName); - testFs.initialize(fs.getUri(), testConfig); - -// System.out.print(testFs.getAbfsStore().getAbfsConfiguration()); -// System.out.print(abfsConfig); - - testFs.create(testPath).close(); - testFs.open(testPath); - testFs.getFileStatus(testPath); - } - - - @Test - public void testReadWriteFailsWithInvalidOAuthToken() throws Exception { - AzureBlobFileSystem fs = getFileSystem(); - AzureBlobFileSystem testFs = new AzureBlobFileSystem(); - - String accountName = getAccountName(); - - Configuration testConfig = new Configuration(getRawConfiguration()); - addOAuthConfigs(fs, testConfig, accountName); - - testFs.initialize(fs.getUri(), testConfig); - - // Create mock token provider with invalid token - AccessTokenProvider mockProvider = Mockito.mock(AccessTokenProvider.class); - AzureADToken mockToken = Mockito.mock(AzureADToken.class); - Mockito.when(mockToken.getAccessToken()).thenReturn("1234=abcd"); // Invalid token - Mockito.when(mockProvider.getToken()).thenReturn(mockToken); - - // Inject mock provider into AbfsClient - injectMockTokenProvider(testFs, mockProvider); - - intercept(AccessDeniedException.class, () -> {testFs.create(testPath); }); - } - - @Test - public void testReadWriteFailsWithInvalidSASToken() throws Exception { - AzureBlobFileSystem fs = getFileSystem(); - AzureBlobFileSystem testFs = new AzureBlobFileSystem(); - - String accountName = getAccountName(); - - Configuration testConfig = new Configuration(getRawConfiguration()); - addOAuthConfigs(fs, testConfig, accountName); - testConfig.unset("fs.azure.sas.token.provider.type"); - testConfig.set("fs.azure.sas.token.provider.type", "org.apache.hadoop.fs.azurebfs.extensions.MockInvalidSASTokenProvider"); - - testFs.initialize(fs.getUri(), testConfig); - - intercept(AccessDeniedException.class, () -> {testFs.create(testPath); }); - } - - @Test - public void testOperationWithValidAndExpiredSASToken() throws Exception { - // Set up the Configuration and FileSystem - AzureBlobFileSystem fs = getFileSystem(); - AzureBlobFileSystem testFs = new AzureBlobFileSystem(); - - Configuration testConfig = new Configuration(getRawConfiguration()); - addOAuthConfigs(fs, testConfig, getAccountName()); - - testFs.initialize(fs.getUri(), testConfig); - - // Get a real SAS token from the configured provider - AbfsConfiguration abfsConfig = testFs.getAbfsStore().getAbfsConfiguration(); - SASTokenProvider realSasProvider = abfsConfig.getSASTokenProviderForUserBoundSAS(); - assertNotNull(realSasProvider, "SASTokenProvider for user-bound SAS must be configured"); - String validSasToken = realSasProvider.getSASToken( - getAccountName(), - testFs.toString(), - String.valueOf(testPath), - SASTokenProvider.GET_PROPERTIES_OPERATION); - assertNotNull(validSasToken, "SAS token must not be null"); - assertFalse(validSasToken.isEmpty(), "SAS token must not be empty"); - - // 1. Operation should work with valid SAS token - // (No exception expected) - org.apache.hadoop.fs.Path path = testPath; - testFs.create(path); // Should succeed - - // 2. Now, modify the ske/se fields to be expired and inject a mock provider - String expiredDate = OffsetDateTime.now(ZoneOffset.UTC) - .minusDays(1) - .format(DateTimeFormatter.ISO_DATE_TIME); - String expiredSasToken = java.util.Arrays.stream(validSasToken.split("&")) - .map(kv -> { - String[] pair = kv.split("=", 2); - if (pair[0].equals("ske") || pair[0].equals("se")) { - return pair[0] + "=" + expiredDate; - } else { - return kv; - } - }) - .collect(java.util.stream.Collectors.joining("&")); - - // Create a mock SASTokenProvider that returns the expired SAS token - SASTokenProvider mockSasProvider = org.mockito.Mockito.mock( - SASTokenProvider.class); - org.mockito.Mockito.when( - mockSasProvider.getSASToken(org.mockito.Mockito.anyString(), - org.mockito.Mockito.anyString(), org.mockito.Mockito.anyString(), - org.mockito.Mockito.anyString())) - .thenReturn(expiredSasToken); - - // Inject the mock provider into the AbfsClient - injectMockSASTokenProvider(testFs, mockSasProvider); - - // Try a file operation and expect failure due to expired SAS token - intercept(AccessDeniedException.class, () -> {testFs.getFileStatus(path);}); - } - - // Helper to inject a mock SASTokenProvider into the AbfsClient - private void injectMockSASTokenProvider(AzureBlobFileSystem fs, SASTokenProvider provider) throws Exception { - Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField("abfsStore"); - abfsStoreField.setAccessible(true); - AzureBlobFileSystemStore store = (AzureBlobFileSystemStore) abfsStoreField.get(fs); - - Field abfsClientField = AzureBlobFileSystemStore.class.getDeclaredField("client"); - abfsClientField.setAccessible(true); - AbfsClient client = (AbfsClient) abfsClientField.get(store); - - // Use AbfsClient.class to get the field, not client.getClass() - java.lang.reflect.Field sasProviderField = AbfsClient.class.getDeclaredField("sasTokenProvider"); - sasProviderField.setAccessible(true); - sasProviderField.set(client, provider); - } -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java index 429422fc30549..4a60bfad29151 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java @@ -518,7 +518,6 @@ public void setAuthConfig(AbfsConfiguration abfsConf, String providerClassKey = ""; String providerClassValue = ""; - //todo: add user-bound auth type handling tests switch (authType) { case OAuth: providerClassKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME @@ -549,12 +548,6 @@ public void setAuthConfig(AbfsConfiguration abfsConf, providerClassValue = TEST_SAS_PROVIDER_CLASS_CONFIG_1; break; - case UserboundSASWithOAuth: -// providerClassKey = FS_AZURE_SAS_TOKEN_PROVIDER_TYPE -// + (isAccountSetting ? accountNameSuffix : ""); -// providerClassValue = TEST_SAS_PROVIDER_CLASS_CONFIG_1; - break; - default: // set nothing } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java index e60086f2298a7..d82a4d2879b93 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java @@ -55,8 +55,8 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID = "fs.azure.test.app.service.principal.object.id"; - public static final String FS_AZURE_END_USER_TENANT_ID = "fs.azure.test.end.user.tenant.id"; - public static final String FS_AZURE_END_USER_OBJECT_ID = "fs.azure.test.end.user.object.id"; + public static final String FS_AZURE_TEST_END_USER_TENANT_ID = "fs.azure.test.end.user.tenant.id"; + public static final String FS_AZURE_TEST_END_USER_OBJECT_ID = "fs.azure.test.end.user.object.id"; public static final String FS_AZURE_TEST_APP_ID = "fs.azure.test.app.id"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockInvalidSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockInvalidSASTokenProvider.java new file mode 100644 index 0000000000000..a3d7a8c64e738 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockInvalidSASTokenProvider.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.extensions; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; + +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.MOCK_SASTOKENPROVIDER_FAIL_INIT; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN; + +/** + * A mock SAS token provider to test error conditions. + */ +public class MockInvalidSASTokenProvider implements SASTokenProvider { + String invalidSASToken = "testInvalidSASToken"; + + @Override + public void initialize(Configuration configuration, String accountName) { + //do nothing + } + + /** + * Returns null SAS token query or Empty if returnEmptySASToken is set. + * @param accountName + * @param fileSystem the name of the fileSystem. + * @param path the file or directory path. + * @param operation the operation to be performed on the path. + * @return + */ + @Override + public String getSASToken(String accountName, String fileSystem, String path, + String operation) { + return invalidSASToken; + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java index ffb087c63800a..cba107b6f7b48 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java @@ -40,21 +40,40 @@ import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; import org.apache.hadoop.security.AccessControlException; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_X_WWW_FORM_URLENCODED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_POST; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CONNECTION_TIMEOUT; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_READ_TIMEOUT; /** - * A mock user-bound SAS token provider implementation. + * A mock user-bound SAS token provider implementation for testing purposes. + * Provides functionality to generate user delegation SAS tokens for Azure Blob Storage. */ - public class MockUserBoundSASTokenProvider implements SASTokenProvider { - private DelegationSASGenerator_Version_July5 generator; + // Constants for URLs and endpoints + private static final String AZURE_BLOB_ENDPOINT_TEMPLATE = "https://%s.blob.core.windows.net/"; + private static final String AZURE_LOGIN_ENDPOINT_TEMPLATE = "https://login.microsoftonline.com/%s/oauth2/v2.0/token"; + private static final String USER_DELEGATION_QUERY_PARAMS = "?restype=service&comp=userdelegationkey"; + + + // HTTP related constants + private static final String UTF_8 = StandardCharsets.UTF_8.toString(); + private static final int RESPONSE_BUFFER_SIZE = 4 * 1024; public static final String TEST_OWNER = "325f1619-4205-432f-9fce-3fd594325ce5"; public static final String CORRELATION_ID = "66ff4ffc-ff17-417e-a2a9-45db8c5b0b5c"; public static final String NO_AGENT_PATH = "NoAgentPath"; + private DelegationSASGenerator_Version_July5 generator; + + /** + * Initializes the SAS token provider with configuration settings. + * + * @param configuration Configuration containing Azure storage settings + * @param accountName The name of the storage account to initialize for + * @throws IOException if there is an error during initialization + */ @Override public void initialize(Configuration configuration, String accountName) throws IOException { String appID = configuration.get(TestConfigurationKeys.FS_AZURE_TEST_APP_ID); @@ -63,46 +82,68 @@ public void initialize(Configuration configuration, String accountName) throws I String skoid = configuration.get(TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID); String skt = SASGenerator.ISO_8601_FORMATTER.format(Instant.now().minus(SASGenerator.FIVE_MINUTES)); String ske = SASGenerator.ISO_8601_FORMATTER.format(Instant.now().plus(SASGenerator.ONE_DAY)); - String skv = SASGenerator.AuthenticationVersion.July5.toString(); + String skv = SASGenerator.AuthenticationVersion.Jul5.toString(); - String skdutid = configuration.get(TestConfigurationKeys.FS_AZURE_END_USER_TENANT_ID); - String sduoid = configuration.get(TestConfigurationKeys.FS_AZURE_END_USER_OBJECT_ID); + String skdutid = configuration.get(TestConfigurationKeys.FS_AZURE_TEST_END_USER_TENANT_ID); + String sduoid = configuration.get(TestConfigurationKeys.FS_AZURE_TEST_END_USER_OBJECT_ID); byte[] key = getUserDelegationKey(accountName, appID, appSecret, sktid, skt, ske, skv, skdutid); generator = new DelegationSASGenerator_Version_July5(key, skoid, sktid, skt, ske, skv, skdutid, sduoid); } - // Invokes the AAD v2.0 authentication endpoint with a client credentials grant to get an - // access token. See https://docs.microsoft.com/en-us/azure/active-directory/develop/v2-oauth2-client-creds-grant-flow. + /** + * Gets the authorization header for Azure AD authentication. + * Invokes the AAD v2.0 authentication endpoint with a client credentials + * grant to get an access token. + * See https://docs.microsoft.com/en-us/azure/active-directory/develop/v2-oauth2-client-creds-grant-flow. + * + * @param accountName The storage account name + * @param appID The Azure AD application ID + * @param appSecret The Azure AD application secret + * @param sktid The service principal tenant ID + * @return The authorization header string with bearer token + * @throws IOException if there is an error getting the authorization token + */ private String getAuthorizationHeader(String accountName, String appID, String appSecret, String sktid) throws IOException { - String authEndPoint = String.format("https://login.microsoftonline.com/%s/oauth2/v2.0/token", sktid); + String authEndPoint = String.format(AZURE_LOGIN_ENDPOINT_TEMPLATE, sktid); ClientCredsTokenProvider provider = new ClientCredsTokenProvider(authEndPoint, appID, appSecret); return "Bearer " + provider.getToken().getAccessToken(); } + /** + * Retrieves a user delegation key from Azure Storage. + * + * @param accountName The storage account name + * @param appID The Azure AD application ID + * @param appSecret The Azure AD application secret + * @param sktid The service principal tenant ID + * @param skt The start time for the delegation key + * @param ske The expiry time for the delegation key + * @param skv The API version for the request + * @param skdutid The delegated user tenant ID + * @return The user delegation key as a byte array + * @throws IOException if there is an error retrieving the delegation key + */ private byte[] getUserDelegationKey(String accountName, String appID, String appSecret, String sktid, String skt, String ske, String skv, String skdutid) throws IOException { - String method = "POST"; String account = accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)); - - final StringBuilder sb = new StringBuilder(128); - sb.append("https://"); - sb.append(account); - sb.append(".blob.core.windows.net/?restype=service&comp=userdelegationkey"); + String baseUrl = String.format(AZURE_BLOB_ENDPOINT_TEMPLATE, account); + String urlString = baseUrl + USER_DELEGATION_QUERY_PARAMS; URL url; try { - url = new URL(sb.toString()); + url = new URL(urlString); } catch (MalformedURLException ex) { - throw new InvalidUriException(sb.toString()); + throw new InvalidUriException(urlString); } - List requestHeaders = new ArrayList(); + List requestHeaders = new ArrayList<>(); requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_VERSION, skv)); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, "application/x-www-form-urlencoded")); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.AUTHORIZATION, getAuthorizationHeader(account, appID, appSecret, sktid))); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, APPLICATION_X_WWW_FORM_URLENCODED)); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.AUTHORIZATION, + getAuthorizationHeader(account, appID, appSecret, sktid))); final StringBuilder requestBody = new StringBuilder(512); requestBody.append(""); @@ -113,16 +154,17 @@ private byte[] getUserDelegationKey(String accountName, String appID, String app requestBody.append(skdutid); requestBody.append(""); - AbfsJdkHttpOperation op = new AbfsJdkHttpOperation(url, method, requestHeaders, - Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT), null); + AbfsJdkHttpOperation op = new AbfsJdkHttpOperation(url, HTTP_METHOD_POST, requestHeaders, + Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), + Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT), null); - byte[] requestBuffer = requestBody.toString().getBytes(StandardCharsets.UTF_8.toString()); + byte[] requestBuffer = requestBody.toString().getBytes(UTF_8); op.sendPayload(requestBuffer, 0, requestBuffer.length); - byte[] responseBuffer = new byte[4 * 1024]; - op.processResponse(responseBuffer, 0, responseBuffer.length); //GETTING NULL HERE [400- wrong XML] + byte[] responseBuffer = new byte[RESPONSE_BUFFER_SIZE]; + op.processResponse(responseBuffer, 0, responseBuffer.length); - String responseBody = new String(responseBuffer, 0, (int) op.getBytesReceived(), StandardCharsets.UTF_8); + String responseBody = new String(responseBuffer, 0, (int) op.getBytesReceived(), UTF_8); int beginIndex = responseBody.indexOf("") + "".length(); int endIndex = responseBody.indexOf(""); String value = responseBody.substring(beginIndex, endIndex); @@ -130,10 +172,8 @@ private byte[] getUserDelegationKey(String accountName, String appID, String app } /** - * Invokes the authorizer to obtain a SAS token. + * {@inheritDoc} * - * @param accountName the name of the storage account. - * @param fileSystem the name of the fileSystem. * @param path the file or directory path. * @param operation the operation to be performed on the path. * @return a SAS token to perform the request operation. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index 8505f5f3266f9..1f635723dd268 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -32,6 +32,8 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedClass; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mockito; @@ -49,7 +51,9 @@ import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException; import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -80,6 +84,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT; import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_END_USER_OBJECT_ID; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -765,6 +770,76 @@ public void testExpectHundredContinue() throws Exception { .isFalse(); } + /** + * Parameterized test to verify the correct setup of authentication providers + * for each supported AuthType in the Azure Blob FileSystem configuration. + * For each AuthType, this test checks that the expected provider(s) are present + * and that unsupported providers throw the correct exceptions. + * + * OAuth: Token provider must be present, SAS provider must throw exception. + * SharedKey: Token provider must throw exception, SAS provider must throw exception. + * SAS: SAS provider must be present, token provider must throw exception. + * UserboundSASWithOAuth: Both AccessTokenProvider and SASTokenProvider must be present. + * Custom: Test is skipped. + * + * @param authType the authentication type to test + * @throws Exception if any error occurs during test execution + */ + @ParameterizedTest + @EnumSource(AuthType.class) + public void testAuthTypeProviderSetup(AuthType authType) throws Exception { + this.getConfiguration().set("fs.azure.account.auth.type", authType.name()); + if (authType.name().equals("Custom")) { + return; + } + + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance( + getRawConfiguration()); + + AbfsConfiguration abfsConfig = fs.getAbfsStore().getAbfsConfiguration(); + + switch (authType) { + case OAuth: + assertNotNull(abfsConfig.getTokenProvider(), + "OAuth should have token provider"); + assertThrows(AzureBlobFileSystemException.class, + () -> abfsConfig.getSASTokenProvider(), + "SharedKey should not have SAS provider"); + break; + + case SharedKey: + assertThrows(TokenAccessProviderException.class, + () -> abfsConfig.getTokenProvider(), + "SharedKey should not have token provider"); + assertThrows(AzureBlobFileSystemException.class, + () -> abfsConfig.getSASTokenProvider(), + "SharedKey should not have SAS provider"); + break; + + case SAS: + assertThrows(TokenAccessProviderException.class, + () -> abfsConfig.getTokenProvider(), + "SharedKey should not have token provider"); + assertNotNull(abfsConfig.getSASTokenProvider(), + "SAS should have SAS provider"); + break; + + case UserboundSASWithOAuth: + Object[] providers = abfsConfig.getUserBoundSASBothTokenProviders(); + assertNotNull(providers, "Providers array must not be null"); + assertTrue(providers[0] instanceof AccessTokenProvider, + "First should be AccessTokenProvider"); + assertTrue(providers[1] instanceof SASTokenProvider, + "Second should be SASTokenProvider"); + break; + + default: + fail("Unexpected AuthType: " + authType); + } + + fs.close(); + } + @Test public void testIsNonEmptyDirectory() throws IOException { testIsNonEmptyDirectoryInternal(EMPTY_STRING, true, EMPTY_STRING, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java index 469781c0a4a87..c6b6ac3480a96 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java @@ -41,6 +41,18 @@ public class DelegationSASGenerator_Version_July5 extends SASGenerator { private final String skdutid; private final String sduoid; + /** + * Initializes all the fields required for generating delegation SAS. + * + * @param userDelegationKey the user delegation key as a byte array + * @param skoid Azure AD delegator app's object ID + * @param sktid Azure AD delegator app's tenant ID + * @param skt The start time for the delegation key + * @param ske The expiry time for the delegation key + * @param skv the API version + * @param skdutid Azure AD delegated app's tenant ID + * @param sduoid Azure AD delegated app's user object ID + */ public DelegationSASGenerator_Version_July5(byte[] userDelegationKey, String skoid, String sktid, String skt, String ske, String skv, String skdutid, String sduoid) { super(userDelegationKey); this.skoid = skoid; @@ -52,6 +64,19 @@ public DelegationSASGenerator_Version_July5(byte[] userDelegationKey, String sko this.sduoid = sduoid; } + /** + * Generates a delegation SAS token for the specified resource and operation. + * + * @param accountName The storage account name + * @param containerName The container name + * @param path The path to the resource + * @param operation The operation to authorize + * @param saoid The Azure AD object ID of the application + * @param suoid The Azure AD object ID of the user + * @param scid The correlation ID + * @return The generated SAS token as a query string + * @throws IllegalArgumentException if the operation is not recognized + */ public String getDelegationSAS(String accountName, String containerName, String path, String operation, String saoid, String suoid, String scid) { @@ -60,7 +85,7 @@ public String getDelegationSAS(String accountName, String containerName, String //Ref: https://learn.microsoft.com/en-us/rest/api/storageservices/create-user-delegation-sas // SAS version (sv) used here is 2025-07-05 - final String sv = AuthenticationVersion.July5.toString(); + final String sv = AuthenticationVersion.Jul5.toString(); final String st = ISO_8601_FORMATTER.format(Instant.now().minus(FIVE_MINUTES)); final String se = ISO_8601_FORMATTER.format(Instant.now().plus(ONE_DAY)); @@ -121,7 +146,6 @@ public String getDelegationSAS(String accountName, String containerName, String String signature = computeSignatureForSAS(sp, st, se, sv, sr, accountName, containerName, path, saoid, suoid, scid); - // String signature = "testttsstst"; AbfsUriQueryBuilder qb = new AbfsUriQueryBuilder(); qb.addQuery("skoid", skoid); @@ -132,11 +156,11 @@ public String getDelegationSAS(String accountName, String containerName, String qb.addQuery("skv", skv); //skdutid and sduoid are required for user bound SAS only - if(!Objects.equals(skdutid, EMPTY_STRING)){ - qb.addQuery("skdutid", skdutid); + if (!Objects.equals(skdutid, EMPTY_STRING)) { + qb.addQuery("skdutid", skdutid); } - if(!Objects.equals(sduoid, EMPTY_STRING)){ - qb.addQuery("sduoid", sduoid); + if (!Objects.equals(sduoid, EMPTY_STRING)) { + qb.addQuery("sduoid", sduoid); } if (saoid != null) { @@ -160,6 +184,22 @@ public String getDelegationSAS(String accountName, String containerName, String return qb.toString().substring(1); } + /** + * Computes the signature for the SAS token based on the provided parameters. + * + * @param sp Signed permissions + * @param st Signed start time + * @param se Signed expiry time + * @param sv Signed version + * @param sr Signed resource + * @param accountName The storage account name + * @param containerName The container name + * @param path The path to the resource + * @param saoid The Azure AD object ID of the application + * @param suoid The Azure AD object ID of the user + * @param scid The correlation ID + * @return The computed HMAC256 signature + */ private String computeSignatureForSAS(String sp, String st, String se, String sv, String sr, String accountName, String containerName, String path, String saoid, String suoid, String scid) { @@ -234,7 +274,6 @@ private String computeSignatureForSAS(String sp, String st, String se, String sv String stringToSign = sb.toString(); LOG.debug("Delegation SAS stringToSign: " + stringToSign.replace("\n", ".")); - System.out.println("Delegation SAS stringToSign: " + stringToSign.replace("\n", ".")); return computeHmac256(stringToSign); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java index c51001c9f1da4..e3b6127e7cc03 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java @@ -42,7 +42,7 @@ public enum AuthenticationVersion { Nov18("2018-11-09"), Dec19("2019-12-12"), Feb20("2020-02-10"), - July5("2025-07-05"); + Jul5("2025-07-05"); private final String ver; From 91afee55ebd5d1b6654b15f0189be2ab17b3078a Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Sun, 9 Nov 2025 23:32:37 -0800 Subject: [PATCH 8/9] merge trunk --- .../hadoop-azure/src/site/markdown/index.md | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/index.md b/hadoop-tools/hadoop-azure/src/site/markdown/index.md index 91141e26e98f8..abf5943c8146f 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/index.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/index.md @@ -371,7 +371,21 @@ To retrieve using shell script, specify the path to the script for the config `fs.azure.shellkeyprovider.script`. ShellDecryptionKeyProvider class use the script specified to retrieve the key. -### OAuth 2.0 Client Credentials +### OAuth 2.0 Authentication +The below are the main options of identity configurations for OAuth settings. +All of these would have OAuth set as the auth type + +```xml + + fs.azure.account.auth.type + OAuth + + Use OAuth authentication + + +``` + +#### Client Credentials OAuth 2.0 credentials of (client id, client secret, endpoint) are provided in the configuration/JCEKS file. @@ -417,7 +431,7 @@ the key names are slightly different here. ``` -### OAuth 2.0: Username and Password +#### Username and Password An OAuth 2.0 endpoint, username and password are provided in the configuration/JCEKS file. @@ -459,7 +473,7 @@ An OAuth 2.0 endpoint, username and password are provided in the configuration/J ``` -### OAuth 2.0: Refresh Token +#### Refresh Token With an existing Oauth 2.0 token, make a request to the Active Directory endpoint `https://login.microsoftonline.com/Common/oauth2/token` for this token to be refreshed. @@ -502,7 +516,7 @@ With an existing Oauth 2.0 token, make a request to the Active Directory endpoin ``` -### Azure Managed Identity +#### Azure Managed Identity [Azure Managed Identities](https://docs.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/overview), formerly "Managed Service Identities". @@ -550,7 +564,7 @@ The Azure Portal/CLI is used to create the service identity. ``` -### Azure Workload Identity +#### Azure Workload Identity [Azure Workload Identities](https://docs.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/overview), formerly "Azure AD pod identity". @@ -795,6 +809,7 @@ requests. User can specify them as fixed SAS Token to be used across all the req - Custom token provider - Managed Identity - Workload Identity + Refer to respective OAuth 2.0 sections above to correctly chose the OAuth provider type From 09139dc1eabc3fcc5c8bb552cc29e3760ca6b7d2 Mon Sep 17 00:00:00 2001 From: Manika Joshi Date: Mon, 10 Nov 2025 01:55:09 -0800 Subject: [PATCH 9/9] imports removed --- .../ITestAzureBlobFileSystemUserBoundSAS.java | 43 +++++++++++++------ .../MockDelegationSASTokenProvider.java | 6 +-- .../MockInvalidSASTokenProvider.java | 5 --- .../MockUserBoundSASTokenProvider.java | 6 +-- .../fs/azurebfs/services/ITestAbfsClient.java | 1 - ...> DelegationSASGeneratorVersionJuly5.java} | 11 ++++- 6 files changed, 44 insertions(+), 28 deletions(-) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/{DelegationSASGenerator_Version_July5.java => DelegationSASGeneratorVersionJuly5.java} (97%) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserBoundSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserBoundSAS.java index 6a8b2783971e6..83e3ee8321715 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserBoundSAS.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemUserBoundSAS.java @@ -28,7 +28,6 @@ import java.util.List; import java.util.stream.Collectors; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -57,7 +56,9 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CLIENT_SERVICE_PRINCIPAL_OBJECT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_END_USER_OBJECT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_END_USER_TENANT_ID; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.assertj.core.api.Assumptions.assumeThat; @@ -65,7 +66,8 @@ * Integration tests for AzureBlobFileSystem using User-Bound SAS and OAuth. * Covers scenarios for token provider configuration, SAS token validity, and basic file operations. */ -public class ITestAzureBlobFileSystemUserBoundSAS extends AbstractAbfsIntegrationTest { +public class ITestAzureBlobFileSystemUserBoundSAS + extends AbstractAbfsIntegrationTest { private static Path testPath = new Path("/test.txt"); @@ -115,6 +117,8 @@ public void setup() throws Exception { abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET)); abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET, abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET)); + abfsConfig.set(FS_AZURE_TEST_END_USER_TENANT_ID, + abfsConfig.get(FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID)); abfsConfig.set(FS_AZURE_TEST_END_USER_OBJECT_ID, abfsConfig.get(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID)); abfsConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, @@ -154,7 +158,8 @@ private void injectMockTokenProvider(AzureBlobFileSystem fs, */ private AzureBlobFileSystem createTestFileSystem() throws RuntimeException { try { - return (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()); + return (AzureBlobFileSystem) FileSystem.newInstance( + getRawConfiguration()); } catch (IOException e) { throw new RuntimeException(e); } @@ -167,7 +172,8 @@ private AzureBlobFileSystem createTestFileSystem() throws RuntimeException { @Test public void testShouldFailWhenSduoidMismatchesServicePrincipalId() throws Exception { - this.getConfiguration().set(FS_AZURE_TEST_END_USER_OBJECT_ID, TEST_OBJECT_ID); + this.getConfiguration() + .set(FS_AZURE_TEST_END_USER_OBJECT_ID, TEST_OBJECT_ID); AzureBlobFileSystem testFs = createTestFileSystem(); intercept(AccessDeniedException.class, () -> { @@ -201,7 +207,8 @@ public void testOAuthTokenProviderAndSASTokenFlow() throws Exception { // Verify AbfsConfiguration has an SASTokenProvider configured SASTokenProvider sasProvider - = abfsConfiguration.getUserBoundSASTokenProvider(AuthType.UserboundSASWithOAuth); + = abfsConfiguration.getUserBoundSASTokenProvider( + AuthType.UserboundSASWithOAuth); assertNotNull(sasProvider, "SASTokenProvider for user-bound SAS must be configured"); assertInstanceOf(MockUserBoundSASTokenProvider.class, sasProvider, @@ -216,7 +223,7 @@ public void testOAuthTokenProviderAndSASTokenFlow() throws Exception { } /* - * Tests listing and deleting files under an implicit directory + * Tests listing and deleting files under an implicit directory */ @Test public void testOperationsForImplicitPaths() throws Exception { @@ -245,7 +252,8 @@ public void testOperationsForImplicitPaths() throws Exception { listOp.getResult().getListResultSchema().paths(); assertNotNull(listedEntries, "List result should not be null"); - assertEquals(2, listedEntries.size(), "Expected exactly two files under implicit directory"); + assertEquals(2, listedEntries.size(), + "Expected exactly two files under implicit directory"); client.deletePath( implicitDir.toString(), @@ -346,7 +354,8 @@ public void testOperationWithValidAndExpiredSASToken() throws Exception { // Get a real SAS token from the configured provider AbfsConfiguration abfsConfig = testFs.getAbfsStore().getAbfsConfiguration(); SASTokenProvider realSasProvider - = abfsConfig.getUserBoundSASTokenProvider(AuthType.UserboundSASWithOAuth); + = abfsConfig.getUserBoundSASTokenProvider( + AuthType.UserboundSASWithOAuth); assertNotNull(realSasProvider, "SASTokenProvider for user-bound SAS must be configured"); String validSasToken = realSasProvider.getSASToken( @@ -388,20 +397,26 @@ public void testOperationWithValidAndExpiredSASToken() throws Exception { injectMockSASTokenProvider(testFs, mockSasProvider); // Try a file operation and expect failure due to expired SAS token - intercept(AccessDeniedException.class, () -> {testFs.getFileStatus(testPath);}); + intercept(AccessDeniedException.class, + () -> {testFs.getFileStatus(testPath);}); } // Helper method to inject a mock SASTokenProvider into the AbfsClient - private void injectMockSASTokenProvider(AzureBlobFileSystem fs, SASTokenProvider provider) throws Exception { - Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField("abfsStore"); + private void injectMockSASTokenProvider(AzureBlobFileSystem fs, + SASTokenProvider provider) throws Exception { + Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField( + "abfsStore"); abfsStoreField.setAccessible(true); - AzureBlobFileSystemStore store = (AzureBlobFileSystemStore) abfsStoreField.get(fs); + AzureBlobFileSystemStore store + = (AzureBlobFileSystemStore) abfsStoreField.get(fs); - Field abfsClientField = AzureBlobFileSystemStore.class.getDeclaredField("client"); + Field abfsClientField = AzureBlobFileSystemStore.class.getDeclaredField( + "client"); abfsClientField.setAccessible(true); AbfsClient client = (AbfsClient) abfsClientField.get(store); - Field sasProviderField = AbfsClient.class.getDeclaredField("sasTokenProvider"); + Field sasProviderField = AbfsClient.class.getDeclaredField( + "sasTokenProvider"); sasProviderField.setAccessible(true); sasProviderField.set(client, provider); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java index 5032bdb7c01b3..d221f3ea481a3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java @@ -36,7 +36,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; import org.apache.hadoop.fs.azurebfs.services.AbfsJdkHttpOperation; import org.apache.hadoop.fs.azurebfs.utils.Base64; -import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator_Version_July5; +import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGeneratorVersionJuly5; import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; import org.apache.hadoop.security.AccessControlException; @@ -49,7 +49,7 @@ */ public class MockDelegationSASTokenProvider implements SASTokenProvider { - private DelegationSASGenerator_Version_July5 generator; + private DelegationSASGeneratorVersionJuly5 generator; public static final String TEST_OWNER = "325f1619-4205-432f-9fce-3fd594325ce5"; public static final String CORRELATION_ID = "66ff4ffc-ff17-417e-a2a9-45db8c5b0b5c"; @@ -66,7 +66,7 @@ public void initialize(Configuration configuration, String accountName) throws I String skv = SASGenerator.AuthenticationVersion.Dec19.toString(); byte[] key = getUserDelegationKey(accountName, appID, appSecret, sktid, skt, ske, skv); - generator = new DelegationSASGenerator_Version_July5(key, skoid, sktid, skt, ske, skv, EMPTY_STRING, EMPTY_STRING); + generator = new DelegationSASGeneratorVersionJuly5(key, skoid, sktid, skt, ske, skv, EMPTY_STRING, EMPTY_STRING); } // Invokes the AAD v2.0 authentication endpoint with a client credentials grant to get an diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockInvalidSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockInvalidSASTokenProvider.java index a3d7a8c64e738..0d39054dab24f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockInvalidSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockInvalidSASTokenProvider.java @@ -18,13 +18,8 @@ package org.apache.hadoop.fs.azurebfs.extensions; -import java.io.IOException; - import org.apache.hadoop.conf.Configuration; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.MOCK_SASTOKENPROVIDER_FAIL_INIT; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN; - /** * A mock SAS token provider to test error conditions. */ diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java index cba107b6f7b48..e4ede0345e887 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java @@ -36,7 +36,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; import org.apache.hadoop.fs.azurebfs.services.AbfsJdkHttpOperation; import org.apache.hadoop.fs.azurebfs.utils.Base64; -import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator_Version_July5; +import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGeneratorVersionJuly5; import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; import org.apache.hadoop.security.AccessControlException; @@ -65,7 +65,7 @@ public class MockUserBoundSASTokenProvider implements SASTokenProvider { public static final String CORRELATION_ID = "66ff4ffc-ff17-417e-a2a9-45db8c5b0b5c"; public static final String NO_AGENT_PATH = "NoAgentPath"; - private DelegationSASGenerator_Version_July5 generator; + private DelegationSASGeneratorVersionJuly5 generator; /** * Initializes the SAS token provider with configuration settings. @@ -89,7 +89,7 @@ public void initialize(Configuration configuration, String accountName) throws I byte[] key = getUserDelegationKey(accountName, appID, appSecret, sktid, skt, ske, skv, skdutid); - generator = new DelegationSASGenerator_Version_July5(key, skoid, sktid, skt, ske, skv, skdutid, sduoid); + generator = new DelegationSASGeneratorVersionJuly5(key, skoid, sktid, skt, ske, skv, skdutid, sduoid); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index 1f635723dd268..6b3222817e415 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -84,7 +84,6 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT; import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_END_USER_OBJECT_ID; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGeneratorVersionJuly5.java similarity index 97% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGeneratorVersionJuly5.java index c6b6ac3480a96..8694a8d4f53aa 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGeneratorVersionJuly5.java @@ -31,7 +31,7 @@ /** * Test Delegation SAS generator. */ -public class DelegationSASGenerator_Version_July5 extends SASGenerator { +public class DelegationSASGeneratorVersionJuly5 extends SASGenerator { private final String skoid; private final String sktid; private final String skt; @@ -53,7 +53,14 @@ public class DelegationSASGenerator_Version_July5 extends SASGenerator { * @param skdutid Azure AD delegated app's tenant ID * @param sduoid Azure AD delegated app's user object ID */ - public DelegationSASGenerator_Version_July5(byte[] userDelegationKey, String skoid, String sktid, String skt, String ske, String skv, String skdutid, String sduoid) { + public DelegationSASGeneratorVersionJuly5(byte[] userDelegationKey, + String skoid, + String sktid, + String skt, + String ske, + String skv, + String skdutid, + String sduoid) { super(userDelegationKey); this.skoid = skoid; this.sktid = sktid;