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 2 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 @@ -941,31 +941,57 @@ 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,
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
* 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,
Class<? extends SASTokenProvider> sasTokenProviderImplementation =
getTokenProviderClass(authType, FS_AZURE_SAS_TOKEN_PROVIDER_TYPE,
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;
String configuredFixedToken = this.rawConfig.get(FS_AZURE_SAS_FIXED_TOKEN,
null);

Preconditions.checkArgument(!(sasTokenProviderImplementation == null
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
&& configuredFixedToken == null),
String.format(
anujmodi2021 marked this conversation as resolved.
Show resolved Hide resolved
"The value for both \"%s\" and \"%s\" cannot be invalid.",
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
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(
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
sasTokenProviderImplementation, 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.

String.format("Failed to initialize %s",
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
sasTokenProviderImplementation));

LOG.trace("Initializing {}", sasTokenProviderImplementation.getName());
sasTokenProvider.initialize(rawConfig, accountName);
LOG.trace("{} init complete", sasTokenProviderImplementation.getName());
return sasTokenProvider;
} else {
return null;
Copy link
Contributor

Choose a reason for hiding this comment

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

Instead of returning null, can we create an implementation of SasTokenProvider interface, which implements getSasToken() method and return the value in the configuration FS_AZURE_SAS_FIXED_TOKEN. This would simplify code logic, remove null checks, and reduce git diffs.

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

}
} 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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1728,7 +1728,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 @@ -269,6 +269,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 **/
Copy link
Contributor

Choose a reason for hiding this comment

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

add a . at the end of this and the first sentence of every other javadoc; some java versions require it. Ideally a {@value} element too, so the IDEs show what the value is

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.
Added everywhere.

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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,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.azurebfs.utils.NamespaceUtil;
import org.apache.hadoop.fs.store.LogExactlyOnce;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions;
Expand Down Expand Up @@ -309,6 +310,8 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext)

final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
// appending SAS Token to query
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
appendSASTokenToQuery(ROOT_PATH, "", abfsUriQueryBuilder);

final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = getAbfsRestOperation(
Expand All @@ -331,6 +334,8 @@ public AbfsRestOperation setFilesystemProperties(final String properties,

final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
// appending SAS Token to query
appendSASTokenToQuery(ROOT_PATH, "", abfsUriQueryBuilder);
Copy link
Contributor

Choose a reason for hiding this comment

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

why operationName is not provided here. If the thing is that for container APIs, existing SAS mechanism can not work, we should still prevent them. We might have to add intelligence which SAS implementations are allowed for container APIs.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Nice Catch. Operations name added for file system level operations as well.

Just FYI
The purpose of having operation name here is to determine what all permissions are needed to be set in SAS Token generated. It is up to the custom SAS Token Provider Implementation to use this information and set appropriate permissions. For Example, MockDelegationSASTokenProvider does not allow file system level operations hence it will error out for these operations with SASTokenProviderException.

Where as AccountSASGenerator will have fixed permissions irrespective of operation type.

Operation Name are added here so that if a user wants to define their own implementation, they can choose to consume this information as per their needs and logic. They can also choose to simply ignore them.

Copy link
Contributor

Choose a reason for hiding this comment

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

appendSASTokenToQuery was not there before this patch. Is this added for fixed sas token only. If yes, then we would have to add if-condition to keep only fixedSasToken (if in config) applying to the container APIs, and in non-fixedSasToken case, getSASToken should not be called. Reason being, developers would already have their implementations and those might not be adaptable to the container APIs.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

As discussed, we don't want to support any container API on SAS Token Authentication. Removed these changes


final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = getAbfsRestOperation(
Expand Down Expand Up @@ -371,6 +376,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 = getAbfsRestOperation(
Expand All @@ -387,6 +394,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 = getAbfsRestOperation(
Expand Down Expand Up @@ -1285,6 +1294,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
Copy link
Contributor

Choose a reason for hiding this comment

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

make the javadoc of the new method

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

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 @@ -1316,8 +1333,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");
}
Expand Down
25 changes: 22 additions & 3 deletions hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
Original file line number Diff line number Diff line change
Expand Up @@ -311,10 +311,11 @@ 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,
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".
1. Using Shared Access Signature (SAS) tokens provided by a custom implementation of the SASTokenProvider interface.
4. 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.

Expand Down Expand Up @@ -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
<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
Original file line number Diff line number Diff line change
@@ -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;
Copy link
Contributor

Choose a reason for hiding this comment

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

import ordering doesn't match style rules for new code.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Modified as per the rules

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{
steveloughran marked this conversation as resolved.
Show resolved Hide resolved

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
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: lines seem too long; if over 100 chars split to multiple lines

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.
Checked Everywhere

* @throws Exception
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
*/
@Test
public void testBothProviderFixedTokenConfigured() throws Exception {
AbfsConfiguration testAbfsConfig = getConfiguration();
Copy link
Contributor

Choose a reason for hiding this comment

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

Since, these tests may run in parallel with test of other classes. Lets make clone of the configuration object and use it in the test.

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


// 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");
Copy link
Contributor

Choose a reason for hiding this comment

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

prefer MockDelegationSASTokenProvider.class.getName()

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


// 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());
steveloughran marked this conversation as resolved.
Show resolved Hide resolved

// 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);
Copy link
Contributor

Choose a reason for hiding this comment

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

recursive root delete is a funny one. what does abfs do here? does it delete everything? I'm curious now. (s3a fs returns false before even trying to talk to the store).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In case of delete on a root path, ABFS list down all the children of root and delete them individually. In case a child is itself a directory, it will be deleted recursively.

Copy link
Contributor

Choose a reason for hiding this comment

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

ok. I think for s3a we decided against that on the basis that it was pretty dangerous to do accidentally. nobody ever does "rm -rf /" after all. At least not more than once...

} catch (Exception e) {
Copy link
Contributor

Choose a reason for hiding this comment

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

don't catch, you've just lost the entire stack trace. let the test handler catch and report the problem. always

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Makes sense.
Taken

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());
});
}
}
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,11 @@

import java.io.IOException;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

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;
Expand All @@ -35,10 +39,19 @@ public class MockSASTokenProvider implements SASTokenProvider {
private byte[] accountKey;
private ServiceSASGenerator generator;
private boolean skipAuthorizationForTestSetup = false;
protected static final Logger LOG =
Copy link
Contributor

Choose a reason for hiding this comment

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

why protected?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Made private

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());
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
containerSAS = "";
}
return containerSAS;
}

@Override
Expand Down
Loading
Loading