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 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> 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";