Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HADOOP-18516: [ABFS][Authentication] Support Fixed SAS Token for ABFS Authentication #6552

Merged
merged 22 commits into from
May 30, 2024
Merged
Show file tree
Hide file tree
Changes from 7 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.lang.reflect.Field;

import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.azurebfs.services.FixedSASTokenProvider;
import org.apache.hadoop.util.Preconditions;

import org.apache.commons.lang3.StringUtils;
Expand Down Expand Up @@ -976,33 +977,60 @@ public AccessTokenProvider getTokenProvider() throws TokenAccessProviderExceptio
}
}

/**
* Returns the SASTokenProvider implementation to be used to generate SAS token.<br>
* Users can choose between a custom implementation of {@link SASTokenProvider}
* or an in house implementation {@link FixedSASTokenProvider}.<br>
* For Custom implementation "fs.azure.sas.token.provider.type" needs to be provided.<br>
* For Fixed SAS Token use "fs.azure.sas.fixed.token" needs to be provided.<br>
* In case both are provided, Preference will be given to Custom implementation.<br>
* Avoid using a custom tokenProvider implementation just to read the configured
* fixed token, as this could create confusion. Also,implementing the SASTokenProvider
* requires relying on the raw configurations. It is more stable to depend on
* the AbfsConfiguration with which a filesystem is initialized, and eliminate
* chances of dynamic modifications and spurious situations.<br>
* @return sasTokenProvider object based on configurations provided
* @throws AzureBlobFileSystemException
*/
public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemException {
AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey);
if (authType != AuthType.SAS) {
throw new SASTokenProviderException(String.format(
"Invalid auth type: %s is being used, expecting SAS", authType));
"Invalid auth type: %s is being used, expecting SAS.", authType));
}

try {
String configKey = FS_AZURE_SAS_TOKEN_PROVIDER_TYPE;
Class<? extends SASTokenProvider> sasTokenProviderClass =
getTokenProviderClass(authType, configKey, null,
SASTokenProvider.class);

Preconditions.checkArgument(sasTokenProviderClass != null,
String.format("The configuration value for \"%s\" is invalid.", configKey));

SASTokenProvider sasTokenProvider = ReflectionUtils
.newInstance(sasTokenProviderClass, rawConfig);
Preconditions.checkArgument(sasTokenProvider != null,
String.format("Failed to initialize %s", sasTokenProviderClass));

LOG.trace("Initializing {}", sasTokenProviderClass.getName());
sasTokenProvider.initialize(rawConfig, accountName);
LOG.trace("{} init complete", sasTokenProviderClass.getName());
return sasTokenProvider;
Class<? extends SASTokenProvider> customSasTokenProviderImplementation =
getTokenProviderClass(authType, FS_AZURE_SAS_TOKEN_PROVIDER_TYPE,
null, SASTokenProvider.class);
String configuredFixedToken = this.rawConfig.get(FS_AZURE_SAS_FIXED_TOKEN,
null);

Preconditions.checkArgument(
anujmodi2021 marked this conversation as resolved.
Show resolved Hide resolved
customSasTokenProviderImplementation != null || configuredFixedToken != null,
"At least one of the \"%s\" and \"%s\" must be set.",
FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, FS_AZURE_SAS_FIXED_TOKEN);

// Prefer Custom SASTokenProvider Implementation if configured.
if (customSasTokenProviderImplementation != null) {
LOG.trace("Using Custom SASTokenProvider implementation because it is given precedence when it is set.");
SASTokenProvider sasTokenProvider = ReflectionUtils.newInstance(
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
customSasTokenProviderImplementation, rawConfig);
Preconditions.checkArgument(sasTokenProvider != null,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

better to directly raise a new TokenAccessProviderException() here so that there's no double wrapping of stack traces.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes.
I realized we do have a dedicated exception type for SAS Token related issues.
Using SASTokenProviderException instead.

"Failed to initialize %s", customSasTokenProviderImplementation);

LOG.trace("Initializing {}", customSasTokenProviderImplementation.getName());
sasTokenProvider.initialize(rawConfig, accountName);
LOG.trace("{} init complete", customSasTokenProviderImplementation.getName());
return sasTokenProvider;
} else {
LOG.trace("Using FixedSASTokenProvider implementation");
FixedSASTokenProvider fixedSASTokenProvider = new FixedSASTokenProvider(configuredFixedToken);
return fixedSASTokenProvider;
}
} catch (Exception e) {
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
throw new TokenAccessProviderException("Unable to load SAS token provider class: " + e, e);
throw new TokenAccessProviderException(
"Unable to load SAS token provider class: " + e, e);
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
}
}

Expand All @@ -1015,14 +1043,14 @@ public EncryptionContextProvider createEncryptionContextProvider() {
Class<? extends EncryptionContextProvider> encryptionContextClass =
getAccountSpecificClass(configKey, null,
EncryptionContextProvider.class);
Preconditions.checkArgument(encryptionContextClass != null, String.format(
Preconditions.checkArgument(encryptionContextClass != null,
"The configuration value for %s is invalid, or config key is not account-specific",
configKey));
configKey);

EncryptionContextProvider encryptionContextProvider =
ReflectionUtils.newInstance(encryptionContextClass, rawConfig);
Preconditions.checkArgument(encryptionContextProvider != null,
String.format("Failed to initialize %s", encryptionContextClass));
"Failed to initialize %s", encryptionContextClass);

LOG.trace("{} init complete", encryptionContextClass.getName());
return encryptionContextProvider;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1308,10 +1308,10 @@

/**
* Incrementing exists() calls from superclass for statistic collection.
*
* @param f source path.
* @return true if the path exists.
* @throws IOException
* @return false if path does not exists

Check failure on line 1313 in hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

View check run for this annotation

ASF Cloudbees Jenkins ci-hadoop / Apache Yetus

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java#L1313

javadoc: warning: @return has already been specified
* @throws IOException if some issue in checking path
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

. at the end

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

*/
@Override
public boolean exists(Path f) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1729,7 +1729,7 @@ private void initializeClient(URI uri, String fileSystemName,
creds = new SharedKeyCredentials(accountName.substring(0, dotIndex),
abfsConfiguration.getStorageAccountKey());
} else if (authType == AuthType.SAS) {
LOG.trace("Fetching SAS token provider");
LOG.trace("Fetching SAS Token Provider");
sasTokenProvider = abfsConfiguration.getSASTokenProvider();
} else {
LOG.trace("Fetching token provider");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -278,14 +278,13 @@ public final class ConfigurationKeys {
/** Add extra layer of verification of the integrity of the request content during transport: {@value}. */
public static final String FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION = "fs.azure.enable.checksum.validation";

public static String accountProperty(String property, String account) {
return property + "." + account;
}

public static final String FS_AZURE_ENABLE_DELEGATION_TOKEN = "fs.azure.enable.delegation.token";
public static final String FS_AZURE_DELEGATION_TOKEN_PROVIDER_TYPE = "fs.azure.delegation.token.provider.type";

/** Key for SAS token provider **/
/** Key for fixed SAS token: {@value}. **/
public static final String FS_AZURE_SAS_FIXED_TOKEN = "fs.azure.sas.fixed.token";

/** Key for SAS token provider: {@value}. **/
public static final String FS_AZURE_SAS_TOKEN_PROVIDER_TYPE = "fs.azure.sas.token.provider.type";

/** For performance, AbfsInputStream/AbfsOutputStream re-use SAS tokens until the expiry is within this number of seconds. **/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -998,6 +998,7 @@ public AbfsRestOperation flush(final String path, final long position,
abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData));
abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose));

// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
abfsUriQueryBuilder, cachedSasToken);
Expand Down Expand Up @@ -1094,6 +1095,7 @@ public AbfsRestOperation read(final String path,
}

final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();

// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION,
abfsUriQueryBuilder, cachedSasToken);
Expand Down Expand Up @@ -1389,16 +1391,17 @@ private String appendSASTokenToQuery(String path,
sasToken = cachedSasToken;
LOG.trace("Using cached SAS token.");
}

// if SAS Token contains a prefix of ?, it should be removed
if (sasToken.charAt(0) == '?') {
sasToken = sasToken.substring(1);
}

queryBuilder.setSASToken(sasToken);
LOG.trace("SAS token fetch complete for {} on {}", operation, path);
} catch (Exception ex) {
throw new SASTokenProviderException(String.format("Failed to acquire a SAS token for %s on %s due to %s",
operation,
path,
throw new SASTokenProviderException(String.format(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: changes in this class can be avoided as it adds to the diff and we are adding only new lines (optional)

"Failed to acquire a SAS token for %s on %s due to %s", operation, path,
ex.toString()));
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
package org.apache.hadoop.fs.azurebfs.services;

Check failure on line 1 in hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/FixedSASTokenProvider.java

View check run for this annotation

ASF Cloudbees Jenkins ci-hadoop / Apache Yetus

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/FixedSASTokenProvider.java#L1

asflicense: Missing Apache License

import java.io.IOException;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;

public class FixedSASTokenProvider implements SASTokenProvider {
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
private String fixedSASToken;

public FixedSASTokenProvider(final String fixedSASToken) {
this.fixedSASToken = fixedSASToken;
}

@Override
public void initialize(final Configuration configuration,
final String accountName)
throws IOException {
}

@Override
public String getSASToken(final String account,
final String fileSystem,
final String path,
final String operation) throws IOException {
return fixedSASToken;
}
}
32 changes: 26 additions & 6 deletions hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
Original file line number Diff line number Diff line change
Expand Up @@ -309,12 +309,13 @@ in different deployment situations.
The ABFS client can be deployed in different ways, with its authentication needs
driven by them.

1. With the storage account's authentication secret in the configuration:
"Shared Key".
1. Using OAuth 2.0 tokens of one form or another.
1. Deployed in-Azure with the Azure VMs providing OAuth 2.0 tokens to the application,
"Managed Instance".
1. Using Shared Access Signature (SAS) tokens provided by a custom implementation of the SASTokenProvider interface.
1. With the storage account's authentication secret in the configuration: "Shared Key".
2. Using OAuth 2.0 tokens of one form or another.
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
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.

SAS Based Authentication should be used only with HNS Enabled accounts. Non-HNS Accounts, Does not work with any of the SAS Based Authentications.

What can be changed is what secrets/credentials are used to authenticate the caller.

Expand Down Expand Up @@ -625,6 +626,25 @@ tokens by implementing the SASTokenProvider interface.

The declared class must implement `org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider`.

*Note:* When using a token provider implementation that provides a User Delegation SAS Token or Service SAS Token, some operations may be out of scope and may fail.<br>
Also, SAS Token Based Authentication is recommended to be used only for HNS-Enabled Accounts. Some operations might fail for Non-HNS Accounts.

### Fixed Shared Access Signature (SAS) Token

A Shared Access Signature Token can be directly configured in the account settings file. This should ideally be used for an Account SAS Token, that can be fixed as a constant for an account.
```xml
<property>
<name>fs.azure.account.auth.type</name>
<value>SAS</value>
</property>
<property>
<name>fs.azure.sas.fixed.token</name>
<value>{SAS Token generated or obtained directly from public interfaces}</value>
<description>Fixed SAS Token directly configured</description>
</property>
```
*Note:* When `fs.azure.sas.token.provider.type` and `fs.azure.fixed.sas.token` are both configured, precedence will be given to the custom token provider implementation.

## <a name="technical"></a> Technical notes

### <a name="proxy"></a> Proxy setup
Expand Down
Loading
Loading