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..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 @@ -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, @@ -1770,7 +1778,12 @@ private void initializeClient(URI uri, String fileSystemName, } LOG.trace("Initializing AbfsClient for {}", baseUrl); - if (tokenProvider != null) { + 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()); 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/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/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 eec0d86f0b6bb..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 @@ -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,20 +38,26 @@ 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, 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"; @@ -117,6 +125,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); } @@ -183,6 +200,20 @@ 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 + if (!Objects.equals(skdutid, EMPTY_STRING)) { + sb.append(skdutid); + sb.append("\n"); + } + + if (!Objects.equals(sduoid, EMPTY_STRING)) { + sb.append(sduoid); + sb.append("\n"); + } + + sb.append("\n"); // sip sb.append("\n"); // spr sb.append(sv); @@ -197,6 +228,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;