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 80f803d80dab0..fd06df595a2ed 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 @@ -914,31 +914,50 @@ public AccessTokenProvider getTokenProvider() throws TokenAccessProviderExceptio } } + /** + * The following method chooses between a configured fixed sas token, and a user implementation of the SASTokenProvider interface, + * depending on which one is available. In case a user SASTokenProvider implementation is not present, and a fixed token is configured, + * it simply returns null, to set the sasTokenProvider object for current configuration instance to null. + * The fixed token is read and used later. This is done to: + * 1. check for cases where both are not set, while initializing AbfsConfiguration, + * to not proceed further than thi stage itself when none of the options are available. + * 2. avoid using similar tokenProvider implementation to just read the configured fixed token, + * as this could create confusion. The configuration is introduced + * primarily to avoid using any tokenProvider class/interface. 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 + * @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)); + throw new SASTokenProviderException(String.format("Invalid auth type: %s is being used, expecting SAS", authType)); } try { - String configKey = FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; - Class 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 sasTokenProviderImplementation = + getTokenProviderClass(authType, FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, null, + SASTokenProvider.class); + String configuredFixedToken = this.rawConfig.get(FS_AZURE_SAS_FIXED_TOKEN, null); + + Preconditions.checkArgument(!(sasTokenProviderImplementation == null && configuredFixedToken == null), + String.format("The value for both \"%s\" and \"%s\" cannot be invalid.", FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, FS_AZURE_SAS_FIXED_TOKEN)); + + if (sasTokenProviderImplementation != null) { + LOG.trace("Using SASTokenProvider class because it is given precedence when it is set"); + SASTokenProvider sasTokenProvider = ReflectionUtils.newInstance(sasTokenProviderImplementation, rawConfig); + Preconditions.checkArgument(sasTokenProvider != null, String.format("Failed to initialize %s", sasTokenProviderImplementation)); + + LOG.trace("Initializing {}", sasTokenProviderImplementation.getName()); + sasTokenProvider.initialize(rawConfig, accountName); + LOG.trace("{} init complete", sasTokenProviderImplementation.getName()); + return sasTokenProvider; + } + else { + return null; + } } catch (Exception e) { throw new TokenAccessProviderException("Unable to load SAS token provider class: " + e, e); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 5534b5fb44a51..e7a5ddc09f36e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -202,8 +202,7 @@ public void initialize(URI uri, Configuration configuration) this.setWorkingDirectory(this.getHomeDirectory()); if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener); if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), tracingContext) == null) { try { this.createFileSystem(tracingContext); 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 e5e7056126564..e6d55dacec454 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 @@ -1608,7 +1608,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"); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index a59f76b6d0fe0..9486471ffa0ec 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -240,6 +240,9 @@ public static String accountProperty(String property, String 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 fixed SAS token **/ + public static final String FS_AZURE_SAS_FIXED_TOKEN = "fs.azure.sas.fixed.token"; + /** Key for SAS token provider **/ public static final String FS_AZURE_SAS_TOKEN_PROVIDER_TYPE = "fs.azure.sas.token.provider.type"; 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 25562660ae231..e452cebbb6767 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 @@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -257,6 +258,8 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext) throws final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + // appending SAS Token to query + appendSASTokenToQuery(ROOT_PATH, "", abfsUriQueryBuilder); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( @@ -281,6 +284,8 @@ public AbfsRestOperation setFilesystemProperties(final String properties, Tracin final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + // appending SAS token to query + appendSASTokenToQuery(ROOT_PATH, "", abfsUriQueryBuilder); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( @@ -323,6 +328,8 @@ public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + // appending SAS token to query + appendSASTokenToQuery(ROOT_PATH, "", abfsUriQueryBuilder); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( @@ -340,6 +347,8 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + // appending SAS token to query + appendSASTokenToQuery(ROOT_PATH, "", abfsUriQueryBuilder); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( @@ -1096,6 +1105,14 @@ public static String getDirectoryQueryParameter(final String path) { return directory; } + private String chooseSASToken(String operation, String path) throws IOException { + // chooses the SAS token provider class if it is configured, otherwise reads the configured fixed token + 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 @@ -1127,8 +1144,7 @@ private String appendSASTokenToQuery(String path, try { LOG.trace("Fetch SAS token for {} on {}", operation, path); if (cachedSasToken == null) { - sasToken = sasTokenProvider.getSASToken(this.accountName, - this.filesystem, path, operation); + sasToken = chooseSASToken(operation, path); if ((sasToken == null) || sasToken.isEmpty()) { throw new UnsupportedOperationException("SASToken received is empty or null"); } diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 31498df17904a..d0b7f83767e96 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -315,6 +315,7 @@ driven by them. 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. +2. By directly configuring a fixed Shared Access Signature (SAS) token in the account configuration settings files. What can be changed is what secrets/credentials are used to authenticate the caller. @@ -625,6 +626,24 @@ 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. + +### 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 + + fs.azure.account.auth.type + SAS + + + fs.azure.sas.fixed.token + {SAS Token generated or obtained directly from public interfaces} + Fixed SAS Token directly configured + +``` +*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. + ## Technical notes ### Proxy setup diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java new file mode 100644 index 0000000000000..2a74e6f86bd5b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java @@ -0,0 +1,145 @@ +/** + * 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 org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException; +import org.apache.hadoop.fs.azurebfs.services.AuthType; +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 org.junit.Assume; +import org.junit.Test; + +import java.io.IOException; + +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; + +public class ITestAzureBlobFileSystemChooseSAS extends AbstractAbfsIntegrationTest{ + + private String accountSAS; + + public ITestAzureBlobFileSystemChooseSAS() throws Exception { + // The test uses shared key to create a random filesystem and then creates another + // instance of this filesystem using SAS authorization. + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + } + + private void generateAccountSAS() throws AzureBlobFileSystemException { + final String accountKey = getConfiguration().getStorageAccountKey(); + AccountSASGenerator configAccountSASGenerator = new AccountSASGenerator(Base64.decode(accountKey)); + accountSAS = configAccountSASGenerator.getAccountSAS(getAccountName()); + } + + @Override + public void setup() throws Exception { + createFilesystemForSASTests(); + super.setup(); + // obtaining an account SAS token from in-built generator to set as configuration for testing filesystem level operations + generateAccountSAS(); + } + + /** + * Tests the scenario where both the token provider class and a fixed token are configured: + * whether the correct choice is made (precedence given to token provider class), and the chosen SAS Token works as expected + * @throws Exception + */ + @Test + public void testBothProviderFixedTokenConfigured() throws Exception { + AbfsConfiguration testAbfsConfig = getConfiguration(); + + // configuring a SASTokenProvider class: this provides a user delegation SAS + // user delegation SAS Provider is set + // This easily distinguishes between results of filesystem level and blob level operations to ensure correct SAS is chosen, + // when both a provider class and fixed token is configured. + testAbfsConfig.set(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, "org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider"); + + // configuring the fixed SAS token + testAbfsConfig.set(FS_AZURE_SAS_FIXED_TOKEN, accountSAS); + + // creating a new fs instance with the updated configs + AzureBlobFileSystem newTestFs = (AzureBlobFileSystem) FileSystem.newInstance(testAbfsConfig.getRawConfiguration()); + + // testing a file system level operation + TracingContext tracingContext = getTestTracingContext(newTestFs, true); + // expected to fail in the ideal case, as delegation SAS will be chosen, provider class is given preference when both are configured + // this expectation is because filesystem level operations are beyond the scope of Delegation SAS Token + intercept(SASTokenProviderException.class, + () -> { + newTestFs.getAbfsStore().getFilesystemProperties(tracingContext); + }); + + // testing blob level operation to ensure delegation SAS token is otherwise valid and above operation fails only because it is fs level + Path testPath = new Path("/testCorrectSASToken"); + newTestFs.create(testPath).close(); + } + + /** + * Tests the scenario where only the fixed token is configured, and no token provider class is set: + * whether fixed token is read correctly from configs, and whether the chosen SAS Token works as expected + * @throws IOException + */ + @Test + public void testOnlyFixedTokenConfigured() throws IOException { + AbfsConfiguration testAbfsConfig = getConfiguration(); + + // clearing any previously configured SAS Token Provider class + testAbfsConfig.unset(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE); + + // setting an account SAS token in the fixed token field + testAbfsConfig.set(FS_AZURE_SAS_FIXED_TOKEN, accountSAS); + + // creating a new FS with updated configs + AzureBlobFileSystem newTestFs = (AzureBlobFileSystem) FileSystem.newInstance(testAbfsConfig.getRawConfiguration()); + + // attempting an operation using the selected SAS Token + // as an account SAS is configured, both filesystem level operations (on root) and blob level operations should succeed + try { + newTestFs.getFileStatus(new Path("/")); + Path testPath = new Path("/testCorrectSASToken"); + newTestFs.create(testPath).close(); + newTestFs.delete(new Path("/"), true); + } catch (Exception e) { + fail("Exception has been thrown: "+e.getMessage()); + } + + } + + /** + * Tests the scenario where both the token provider class and the fixed token are not configured: + * whether the code errors out at the initialization stage itself + * @throws IOException + */ + @Test + public void testBothProviderFixedTokenUnset() throws Exception { + AbfsConfiguration testAbfsConfig = getConfiguration(); + + testAbfsConfig.unset(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE); + testAbfsConfig.unset(FS_AZURE_SAS_FIXED_TOKEN); + + intercept(TokenAccessProviderException.class, + () -> { + AzureBlobFileSystem newTestFs = (AzureBlobFileSystem) FileSystem.newInstance(testAbfsConfig.getRawConfiguration()); + }); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockSASTokenProvider.java index 50ac20970f45f..8965fd75f8ed8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockSASTokenProvider.java @@ -21,11 +21,14 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.utils.Base64; import org.apache.hadoop.fs.azurebfs.utils.ServiceSASGenerator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A mock SAS token provider implementation @@ -35,10 +38,19 @@ public class MockSASTokenProvider implements SASTokenProvider { private byte[] accountKey; private ServiceSASGenerator generator; private boolean skipAuthorizationForTestSetup = false; + protected static final Logger LOG = + LoggerFactory.getLogger(MockSASTokenProvider.class); // For testing we use a container SAS for all operations. private String generateSAS(byte[] accountKey, String accountName, String fileSystemName) { - return generator.getContainerSASWithFullControl(accountName, fileSystemName); + String containerSAS = ""; + try { + containerSAS = generator.getContainerSASWithFullControl(accountName, fileSystemName); + } catch (InvalidConfigurationValueException e) { + LOG.debug(e.getMessage()); + containerSAS = ""; + } + return containerSAS; } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AccountSASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AccountSASGenerator.java new file mode 100644 index 0000000000000..5e7902adf118a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AccountSASGenerator.java @@ -0,0 +1,91 @@ +/** + * 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.utils; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.AbfsUriQueryBuilder; + +import java.time.Instant; + +/** + * Account SAS Generator to be used by tests + */ + +public class AccountSASGenerator extends SASGenerator { + /** + * Creates Account SAS + * https://learn.microsoft.com/en-us/rest/api/storageservices/create-account-sas + * @param accountKey: the storage account key + */ + public AccountSASGenerator(byte[] accountKey) { + super(accountKey); + } + + public String getAccountSAS(String accountName) throws AzureBlobFileSystemException { + // retaining only the account name + accountName = getCanonicalAccountName(accountName); + String sp = "racwdl"; + String sv = "2021-06-08"; + String srt = "sco"; + + String st = ISO_8601_FORMATTER.format(Instant.now().minus(FIVE_MINUTES)); + String se = ISO_8601_FORMATTER.format(Instant.now().plus(ONE_DAY)); + + String ss = "bf"; + String spr = "https"; + String signature = computeSignatureForSAS(sp, ss, srt, st, se, sv, accountName); + + AbfsUriQueryBuilder qb = new AbfsUriQueryBuilder(); + qb.addQuery("sp", sp); + qb.addQuery("ss", ss); + qb.addQuery("srt", srt); + qb.addQuery("st", st); + qb.addQuery("se", se); + qb.addQuery("sv", sv); + qb.addQuery("sig", signature); + return qb.toString().substring(1); + } + + private String computeSignatureForSAS(String signedPerm, String signedService, String signedResType, + String signedStart, String signedExp, String signedVersion, String accountName) { + + StringBuilder sb = new StringBuilder(); + sb.append(accountName); + sb.append("\n"); + sb.append(signedPerm); + sb.append("\n"); + sb.append(signedService); + sb.append("\n"); + sb.append(signedResType); + sb.append("\n"); + sb.append(signedStart); + sb.append("\n"); + sb.append(signedExp); + sb.append("\n"); + sb.append("\n"); // signedIP + sb.append("\n"); // signedProtocol + sb.append(signedVersion); + sb.append("\n"); + sb.append("\n"); //signed encryption scope + + String stringToSign = sb.toString(); + LOG.debug("Account SAS stringToSign: " + stringToSign.replace("\n", ".")); + return computeHmac256(stringToSign); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java index 2e9289d8d44c7..5c9057ff00b2f 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 @@ -27,6 +27,9 @@ import javax.crypto.Mac; import javax.crypto.spec.SecretKeySpec; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.utils.Base64; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** @@ -70,7 +73,7 @@ private SASGenerator() { * Called by subclasses to initialize the cryptographic SHA-256 HMAC provider. * @param key - a 256-bit secret key */ - protected SASGenerator(byte[] key) { + protected SASGenerator(byte[] key) { this.key = key; initializeMac(); } @@ -85,6 +88,25 @@ private void initializeMac() { } } + protected String getCanonicalAccountName(String accountName) throws InvalidConfigurationValueException { + // returns the account name without the endpoint + // given accountnames with endpoint have the format accountname.endpoint + // For example, input of xyz.dfs.core.windows.net should return "xyz" only + int dotIndex = accountName.indexOf(AbfsHttpConstants.DOT); + if (dotIndex == 0) { + // case when accountname starts with a ".": endpoint is present, accountName is null + // for example .dfs.azure.com, which is invalid + throw new InvalidConfigurationValueException("Account Name is not fully qualified"); + } + if (dotIndex > 0) { + // case when endpoint is present with accountName + return accountName.substring(0, dotIndex); + } else { + // case when accountName is already canonicalized + return accountName; + } + } + protected String computeHmac256(final String stringToSign) { byte[] utf8Bytes; try { @@ -98,4 +120,4 @@ protected String computeHmac256(final String stringToSign) { } return Base64.encode(hmac); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ServiceSASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ServiceSASGenerator.java index 24a1cea255b4a..2cbed62646e7f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ServiceSASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ServiceSASGenerator.java @@ -20,6 +20,7 @@ import java.time.Instant; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.services.AbfsUriQueryBuilder; /** @@ -36,7 +37,8 @@ public ServiceSASGenerator(byte[] accountKey) { super(accountKey); } - public String getContainerSASWithFullControl(String accountName, String containerName) { + public String getContainerSASWithFullControl(String accountName, String containerName) throws InvalidConfigurationValueException { + accountName = getCanonicalAccountName(accountName); String sp = "rcwdl"; String sv = AuthenticationVersion.Feb20.toString(); String sr = "c";