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..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() {
@@ -1279,7 +1281,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 extends AccessTokenProvider> tokenProviderClass =
getTokenProviderClass(authType,
@@ -1474,6 +1476,68 @@ 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 getUserBoundSASTokenProvider(AuthType authType) throws AzureBlobFileSystemException {
+
+ try {
+ Class extends SASTokenProvider> 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);
+ }
+ }
+
+ /**
+ * 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 d51559de18ea0..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,11 +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) {
+ LOG.trace("Fetching SAS and OAuth Token Provider for user bound SAS");
+ AzureADAuthenticator.init(abfsConfiguration);
+ Object[] providers
+ = abfsConfiguration.getUserBoundSASBothTokenProviders();
+ tokenProvider = (AccessTokenProvider) providers[0];
+ sasTokenProvider = (SASTokenProvider) providers[1];
+ ExtensionHelper.bind(tokenProvider, uri,
+ abfsConfiguration.getRawConfiguration());
} else {
LOG.trace("Fetching token provider");
tokenProvider = abfsConfiguration.getTokenProvider();
ExtensionHelper.bind(tokenProvider, uri,
- abfsConfiguration.getRawConfiguration());
+ abfsConfiguration.getRawConfiguration());
}
// Encryption setup
@@ -1769,16 +1778,11 @@ private void initializeClient(URI uri, String fileSystemName,
}
}
- LOG.trace("Initializing AbfsClient for {}", baseUrl);
- 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 fe4991c9582d5..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";
@@ -187,7 +188,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"),
+ JUL_05_2025("2025-07-05");
private final String xMsApiVersion;
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 71da8f9bda96e..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,22 +337,24 @@ 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;
- }
+ /**
+ * 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,
+ final AccessTokenProvider tokenProvider,
final SASTokenProvider sasTokenProvider,
final EncryptionContextProvider encryptionContextProvider,
final AbfsClientContext abfsClientContext,
@@ -361,6 +363,7 @@ public AbfsClient(final URL baseUrl,
this(baseUrl, sharedKeyCredentials, abfsConfiguration,
encryptionContextProvider, abfsClientContext, abfsServiceType);
this.sasTokenProvider = sasTokenProvider;
+ this.tokenProvider = tokenProvider;
}
@Override
@@ -1157,7 +1160,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..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,24 +47,26 @@ 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);
- }
+ /**
+ * 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,
+ final AccessTokenProvider tokenProvider,
final SASTokenProvider sasTokenProvider,
final EncryptionContextProvider encryptionContextProvider,
final AbfsClientContext abfsClientContext) throws IOException {
@@ -73,10 +75,10 @@ public AbfsClientHandler(final URL baseUrl,
// only for default client.
initServiceType(abfsConfiguration);
this.dfsAbfsClient = createDfsClient(baseUrl, sharedKeyCredentials,
- abfsConfiguration, null, sasTokenProvider, encryptionContextProvider,
+ abfsConfiguration, tokenProvider, sasTokenProvider, encryptionContextProvider,
abfsClientContext);
this.blobAbfsClient = createBlobClient(baseUrl, sharedKeyCredentials,
- abfsConfiguration, null, sasTokenProvider, encryptionContextProvider,
+ abfsConfiguration, tokenProvider, sasTokenProvider, encryptionContextProvider,
abfsClientContext);
}
@@ -154,7 +156,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,
@@ -188,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 f574f4704ab5c..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,23 +154,68 @@
*/
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,
+ final AccessTokenProvider tokenProvider,
+ final SASTokenProvider sasTokenProvider,
+ final EncryptionContextProvider encryptionContextProvider,
+ final AbfsClientContext abfsClientContext) throws IOException {
+ super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, sasTokenProvider,
encryptionContextProvider, abfsClientContext, AbfsServiceType.DFS);
}
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..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
@@ -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/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 extends ListResultEntrySchema> 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/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
index ebccae55c0a93..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,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_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";
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..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,10 +36,11 @@
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;
+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;
@@ -48,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";
@@ -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_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/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
new file mode 100644
index 0000000000000..cba107b6f7b48
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockUserBoundSASTokenProvider.java
@@ -0,0 +1,200 @@
+/**
+ * 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_Version_July5;
+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 for testing purposes.
+ * Provides functionality to generate user delegation SAS tokens for Azure Blob Storage.
+ */
+public class MockUserBoundSASTokenProvider implements SASTokenProvider {
+
+ // 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);
+ 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.Jul5.toString();
+
+ 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);
+ }
+
+ /**
+ * 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(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 account = accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT));
+ String baseUrl = String.format(AZURE_BLOB_ENDPOINT_TEMPLATE, account);
+ String urlString = baseUrl + USER_DELEGATION_QUERY_PARAMS;
+
+ URL url;
+ try {
+ url = new URL(urlString);
+ } catch (MalformedURLException ex) {
+ throw new InvalidUriException(urlString);
+ }
+
+ 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("");
+
+ 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(UTF_8);
+ op.sendPayload(requestBuffer, 0, requestBuffer.length);
+
+ byte[] responseBuffer = new byte[RESPONSE_BUFFER_SIZE];
+ op.processResponse(responseBuffer, 0, responseBuffer.length);
+
+ 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);
+ return Base64.decode(value);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * @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/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.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator_Version_July5.java
similarity index 65%
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 eec0d86f0b6bb..c6b6ac3480a96 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
@@ -19,37 +19,74 @@
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;
/**
* 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;
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) {
+ /**
+ * 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;
this.sktid = sktid;
this.skt = skt;
this.ske = ske;
this.skv = skv;
+ this.skdutid = skdutid;
+ 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) {
- final String sv = AuthenticationVersion.Feb20.toString();
+ // 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.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));
String sr = "b";
@@ -117,6 +154,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);
}
@@ -138,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) {
@@ -183,17 +245,32 @@ 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
+ 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);
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", "."));
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..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
@@ -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"),
+ Jul5("2025-07-05");
private final String ver;