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 1 commit
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 @@ -942,24 +943,18 @@ public AccessTokenProvider getTokenProvider() throws TokenAccessProviderExceptio
}

/**
* The user can choose between a configured fixed sas token, and a user
* implementation of the SASTokenProvider interface. Preference will be given
* to SASTokenProvider class provided as the value of "fs.azure.sas.token.provider.type".
* If above config is not set, it is expected that user wants to use a
* fixed SAS Token provided as value of "fs.azure.sas.fixed.token".
* <ol>
* <li>If both the configs are not provided,
* initialization fails and {@link TokenAccessProviderException} is thrown.</li>
* <li>If both are present, SASTokenProvider class will be used to generate SAS Token.</li>
* <li>If only fixed SAS Token is configured, this will return null
* and Fixed SAS token will be used to sign requests.</li>
* </ol>
* Avoid using a 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.
* @return sasTokenProvider object.
* 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 {
Expand All @@ -970,33 +965,33 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio
}

try {
Class<? extends SASTokenProvider> sasTokenProviderImplementation =
Class<? extends SASTokenProvider> customSasTokenProviderImplementation =
getTokenProviderClass(authType, FS_AZURE_SAS_TOKEN_PROVIDER_TYPE,
null,
SASTokenProvider.class);
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
sasTokenProviderImplementation != null || configuredFixedToken != null,
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 SASTokenProvider Implementation if configured.
if (sasTokenProviderImplementation != null) {
LOG.trace("Using SASTokenProvider class because it is given precedence when it is set.");
// 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
sasTokenProviderImplementation, rawConfig);
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", sasTokenProviderImplementation);
"Failed to initialize %s", customSasTokenProviderImplementation);

LOG.trace("Initializing {}", sasTokenProviderImplementation.getName());
LOG.trace("Initializing {}", customSasTokenProviderImplementation.getName());
sasTokenProvider.initialize(rawConfig, accountName);
LOG.trace("{} init complete", sasTokenProviderImplementation.getName());
LOG.trace("{} init complete", customSasTokenProviderImplementation.getName());
return sasTokenProvider;
} else {
// Configured Fixed SAS Token will be used to sign the requests.
return null;
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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
@InterfaceStability.Unstable
public interface SASTokenProvider {

// Blob Level operations
String CHECK_ACCESS_OPERATION = "check-access";
String CREATE_DIRECTORY_OPERATION = "create-directory";
String CREATE_FILE_OPERATION = "create-file";
Expand All @@ -51,13 +50,6 @@ public interface SASTokenProvider {
String SET_PROPERTIES_OPERATION = "set-properties";
String WRITE_OPERATION = "write";


// Filesystem Level Operations
Copy link
Contributor

Choose a reason for hiding this comment

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

Is the purpose for removing these related to SAS issue for container API's ?

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, that's right.
Theya re not there anyway on trunk. They were added in this PR only and removed later.

String CREATE_FILESYSTEM_OPERATION = "create-filesystem";
String GET_FILESYSTEM_PROPERTIES_OPERATIONS = "get-filesystem-properties";
String SET_FILESYSTEM_PROPERTIES_OPERATIONS = "set-filesystem-properties";
String DELETE_FILESYSTEM_OPERATION = "delete-filesystem";

/**
* Initialize authorizer for Azure Blob File System.
* @param configuration Configuration object
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -311,8 +311,6 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext)
final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);

appendSASTokenToQuery(ROOT_PATH, SASTokenProvider.CREATE_FILESYSTEM_OPERATION, abfsUriQueryBuilder);

final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = getAbfsRestOperation(
AbfsRestOperationType.CreateFileSystem,
Expand All @@ -335,8 +333,6 @@ public AbfsRestOperation setFilesystemProperties(final String properties,
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);

appendSASTokenToQuery(ROOT_PATH, SASTokenProvider.SET_FILESYSTEM_PROPERTIES_OPERATIONS, abfsUriQueryBuilder);

final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = getAbfsRestOperation(
AbfsRestOperationType.SetFileSystemProperties,
Expand Down Expand Up @@ -377,8 +373,6 @@ public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext)
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);

appendSASTokenToQuery(ROOT_PATH, SASTokenProvider.GET_FILESYSTEM_PROPERTIES_OPERATIONS, abfsUriQueryBuilder);

final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = getAbfsRestOperation(
AbfsRestOperationType.GetFileSystemProperties,
Expand All @@ -395,8 +389,6 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);

appendSASTokenToQuery(ROOT_PATH, SASTokenProvider.DELETE_FILESYSTEM_OPERATION, abfsUriQueryBuilder);

final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = getAbfsRestOperation(
AbfsRestOperationType.DeleteFileSystem,
Expand Down Expand Up @@ -1296,22 +1288,6 @@ public static String getDirectoryQueryParameter(final String path) {
return directory;
}

/**
* Chooses between the SAS token provided by SASTokeProvider class and the configured fixed SAS token.
* Preference given to SASTokenProvider implementation to generate the SAS.
* If SASTokenProvider is null, returns the fixed SAS Token configured.
* @param operation
* @param path
* @return sasToken
* @throws IOException
*/
private String chooseSASToken(String operation, String path) throws IOException {
if (sasTokenProvider == null) {
return abfsConfiguration.get(ConfigurationKeys.FS_AZURE_SAS_FIXED_TOKEN);
}
return sasTokenProvider.getSASToken(this.accountName, this.filesystem, path, operation);
}

/**
* If configured for SAS AuthType, appends SAS token to queryBuilder.
* @param path
Expand Down Expand Up @@ -1343,7 +1319,8 @@ private String appendSASTokenToQuery(String path,
try {
LOG.trace("Fetch SAS token for {} on {}", operation, path);
if (cachedSasToken == null) {
sasToken = chooseSASToken(operation, path);
sasToken = sasTokenProvider.getSASToken(this.accountName,
this.filesystem, path, operation);
if ((sasToken == null) || sasToken.isEmpty()) {
throw new UnsupportedOperationException("SASToken received is empty or null");
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
package org.apache.hadoop.fs.azurebfs.services;

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;
}
}
5 changes: 4 additions & 1 deletion hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,8 @@ driven by them.
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.

The authentication mechanism is set in `fs.azure.account.auth.type` (or the
Expand Down Expand Up @@ -624,7 +626,8 @@ 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.
*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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@
package org.apache.hadoop.fs.azurebfs;

import java.io.IOException;
import java.nio.file.AccessDeniedException;

import org.assertj.core.api.Assertions;
import org.junit.Assume;
import org.junit.Test;

Expand All @@ -29,29 +31,29 @@
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException;
import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider;
import org.apache.hadoop.fs.azurebfs.services.AuthType;
import org.apache.hadoop.fs.azurebfs.services.FixedSASTokenProvider;
import org.apache.hadoop.fs.azurebfs.utils.AccountSASGenerator;
import org.apache.hadoop.fs.azurebfs.utils.Base64;
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;

import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_FIXED_TOKEN;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;

/**
* Tests to validate the choice between using a SASTokenProvider to generate
* a SAS or using a Fixed SAS Token configured by user.
* Tests to validate the choice between using a custom SASTokenProvider
* implementation and FixedSASTokenProvider.
*/
public class ITestAzureBlobFileSystemChooseSAS extends AbstractAbfsIntegrationTest{
steveloughran marked this conversation as resolved.
Show resolved Hide resolved

private String accountSAS = null;


/**
* To differentiate which config was used we will use different type of SAS Tokens.
* For Fixed SAS Token we will use an Account SAS with permissions to do File system level operations.
* For SASTokenProvider we will use a User Delegation SAS Token Provider
* such that File System level operations are not permitted.
*/
* To differentiate which SASTokenProvider was used we will use different type of SAS Tokens.
* FixedSASTokenProvider will return an Account SAS with only read permissions.
* SASTokenProvider will return a User Delegation SAS Token with both read and write permissions.
= */
public ITestAzureBlobFileSystemChooseSAS() throws Exception {
// SAS Token configured might not have permissions for creating file system.
// Shared Key must be configured to create one. Once created, a new instance
Expand All @@ -66,7 +68,7 @@ public void setup() throws Exception {
generateAccountSAS(); }

/**
* Generates a Account SAS Token using the Account Shared Key to be used as a fixed SAS Token.
* Generates an Account SAS Token using the Account Shared Key to be used as a fixed SAS Token.
* This will be used by individual tests to set in the configurations.
* @throws AzureBlobFileSystemException
*/
Expand All @@ -77,8 +79,8 @@ private void generateAccountSAS() throws AzureBlobFileSystemException {
}

/**
* Tests the scenario where both the SASTokenProvider and a fixed SAS token are configured.
* SASTokenProvider class should be chosen and User Delegation SAS should be used.
* Tests the scenario where both the custom SASTokenProvider and a fixed SAS token are configured.
* Custom implementation of SASTokenProvider class should be chosen and User Delegation SAS should be used.
* @throws Exception
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
*/
@Test
Expand All @@ -97,16 +99,16 @@ public void testBothProviderFixedTokenConfigured() throws Exception {
// Creating a new file system with updated configs.
try (AzureBlobFileSystem newTestFs = (AzureBlobFileSystem)
FileSystem.newInstance(testAbfsConfig.getRawConfiguration())) {
TracingContext tracingContext = getTestTracingContext(newTestFs, true);

// Asserting that filesystem level operations fails with User Delegation SAS.
intercept(SASTokenProviderException.class, () -> {
newTestFs.getAbfsStore().getFilesystemProperties(tracingContext);
});
// Asserting that MockDelegationSASTokenProvider is used.
Assertions.assertThat(testAbfsConfig.getSASTokenProvider())
.describedAs("Custom SASTokenProvider Class must be used")
.isInstanceOf(MockDelegationSASTokenProvider.class);

// Asserting that User delegation SAS token is otherwise valid and blob level operations succeed.
Path testPath = new Path("/testCorrectSASToken");
// Assert that User Delegation SAS is used and both read and write operations are permitted.
Path testPath = path(getMethodName());
newTestFs.create(testPath).close();
newTestFs.open(testPath).close();
}
}

Expand All @@ -128,11 +130,18 @@ public void testOnlyFixedTokenConfigured() throws Exception {
try (AzureBlobFileSystem newTestFs = (AzureBlobFileSystem)
FileSystem.newInstance(testAbfsConfig.getRawConfiguration())) {

// Asserting that account SAS is used as both filesystem and blob level operations succeed.
newTestFs.getFileStatus(new Path("/"));
Path testPath = new Path("/testCorrectSASToken");
newTestFs.create(testPath).close();
newTestFs.delete(new Path("/"), true);
// Asserting that FixedSASTokenProvider is used.
Assertions.assertThat(testAbfsConfig.getSASTokenProvider())
.describedAs("Custom SASTokenProvider Class must be used")
.isInstanceOf(FixedSASTokenProvider.class);

// Assert that Account SAS is used and only read operations are permitted.
Path testPath = path(getMethodName());
intercept(AccessDeniedException.class, () -> {
newTestFs.create(testPath);
});
// Read Operation is permitted
newTestFs.getFileStatus(new Path(ROOT_PATH));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,7 @@
import org.apache.hadoop.fs.azurebfs.utils.ServiceSASGenerator;

/**
* A mock SAS token provider implementation for testing purpose.
* Account SAS with full permission is created using storage account key.
* A mock SAS token provider implementation
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 of the line

*/
public class MockSASTokenProvider implements SASTokenProvider {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@

/**
* Test Account SAS Generator.
* SAS generated by this will have full access to storage account blob and file services.
* SAS generated by this will have only read access to storage account blob and file services.
*/
public class AccountSASGenerator extends SASGenerator {
/**
Expand All @@ -41,7 +41,7 @@ public String getAccountSAS(String accountName) throws
AzureBlobFileSystemException {
// retaining only the account name
accountName = getCanonicalAccountName(accountName);
String sp = "racwdl";
String sp = "radyl";
String sv = "2021-06-08";
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
String srt = "sco";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,11 +93,6 @@ public String getDelegationSAS(String accountName, String containerName, String
case SASTokenProvider.SET_OWNER_OPERATION:
sp = "o";
break;
// File System Level Operations are not permitted by this user delegation SAS
case SASTokenProvider.CREATE_FILESYSTEM_OPERATION:
case SASTokenProvider.GET_FILESYSTEM_PROPERTIES_OPERATIONS:
case SASTokenProvider.SET_FILESYSTEM_PROPERTIES_OPERATIONS:
case SASTokenProvider.DELETE_FILESYSTEM_OPERATION:
default:
throw new IllegalArgumentException(operation);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@

/**
* Test Service SAS Generator.
* SAS Generated by this will have full access to specified container(filesystem).
*/
public class ServiceSASGenerator extends SASGenerator {

Expand Down
Loading