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: Support fixed SAS token config in addition to SASTokenProvider class #5148

Open
wants to merge 36 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 26 commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
07504ce
Added new configs for fixed SAS Token
sreeb-msft Nov 14, 2022
289b993
Modified config keys for fixed sas token
sreeb-msft Nov 15, 2022
087c6b7
Added new FixedTokenProvider class to read fixed SAS
sreeb-msft Nov 15, 2022
ac82730
Enabled passing tracingContext
sreeb-msft Nov 15, 2022
072f358
added functionality for choosing SAS provider modes
sreeb-msft Nov 15, 2022
306ce9b
Changed passing tracingContext
sreeb-msft Nov 17, 2022
5712f67
Using setter for sasTokenProvider
sreeb-msft Nov 17, 2022
c34f09b
Added setter for sasTokenProvider
sreeb-msft Nov 17, 2022
2e95f5d
SAS token chosen without namespace knowledge
sreeb-msft Nov 18, 2022
3a809b4
Added tests for choosing SAS Token
sreeb-msft Nov 18, 2022
4e1fe4b
Style changes
sreeb-msft Nov 18, 2022
d7179de
simplifying SAS Token choice
sreeb-msft Nov 28, 2022
95e0363
Deleted pid file
sreeb-msft Nov 28, 2022
c915274
Delete FixedSASTokenProvider
sreeb-msft Dec 7, 2022
8288cdc
Delete MockActualSASTokenProvider
sreeb-msft Dec 7, 2022
5174b3c
HADOOP-18516. Appending sas token to fs level operations
sreeb-msft Dec 7, 2022
40d334f
HADOOP-18516. Tests for correct SAS token choice
sreeb-msft Dec 7, 2022
0c20560
HADOOP-18516. Canonicalizing account name for service SAS generator
sreeb-msft Dec 7, 2022
d026c62
Merge branch 'HADOOP-18516' of https://github.com/sreeb-msft/hadoop i…
sreeb-msft Dec 7, 2022
ea99761
HADOOP-18516. Removed unnecessary import
sreeb-msft Dec 7, 2022
d523e50
HADOOP-18516. Ensuring account name is canonicalized
sreeb-msft Dec 8, 2022
6fff60f
HADOOP-18516. Style changes
sreeb-msft Dec 8, 2022
7612036
HADOOP-18516. Style changes
sreeb-msft Dec 8, 2022
573d4c5
HADOOP-18516. Updated docs
sreeb-msft Dec 8, 2022
cc045ac
HADOOP-18516. Added EOF newline
sreeb-msft Dec 8, 2022
49c2f70
HADOOP-18516. Whitespace fix in doc
sreeb-msft Dec 9, 2022
29a13e0
HADOOP-18516. Revert xms version update
sreeb-msft Dec 14, 2022
7a3ea12
Removed full import
sreeb-msft Dec 14, 2022
b603468
HADOOP-18516. Updated log comment
sreeb-msft Dec 14, 2022
1656836
Merge branch 'trunk' into HADOOP-18516
sreeb-msft Dec 19, 2022
2a896e9
HADOOP-18516. Minor changes
sreeb-msft Dec 22, 2022
0f6e222
HADOOP-18516. Style changes
sreeb-msft Dec 28, 2022
7536b9e
Revert "HADOOP-18516. Style changes"
sreeb-msft Dec 28, 2022
3109c68
HADOOP-18516. Style changes
sreeb-msft Dec 28, 2022
6f75596
HADOOP-18516. Readability changes
sreeb-msft Dec 28, 2022
07c1ca6
HADOOP-18516. Javadoc + if condition changed in AbfsConfiguration
sreeb-msft Dec 28, 2022
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 @@ -885,31 +885,50 @@ public AccessTokenProvider getTokenProvider() throws TokenAccessProviderExceptio
}
}

/**
* @return sasTokenProvider object
* @throws AzureBlobFileSystemException
sreeb-msft marked this conversation as resolved.
Show resolved Hide resolved
* 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.
*/

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,
sreeb-msft marked this conversation as resolved.
Show resolved Hide resolved
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 instead of config although both are available for use");
sreeb-msft marked this conversation as resolved.
Show resolved Hide resolved
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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ public void initialize(URI uri, Configuration configuration)

if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) {
TracingContext tracingContext = new TracingContext(clientCorrelationId,
fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener);
fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener);
sreeb-msft marked this conversation as resolved.
Show resolved Hide resolved
if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), tracingContext) == null) {
try {
this.createFileSystem(tracingContext);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1607,7 +1607,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 @@ -230,6 +230,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";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -87,7 +88,7 @@ public class AbfsClient implements Closeable {

private final URL baseUrl;
private final SharedKeyCredentials sharedKeyCredentials;
private final String xMsVersion = "2019-12-12";
private final String xMsVersion = "2021-10-04";
sreeb-msft marked this conversation as resolved.
Show resolved Hide resolved
private final ExponentialRetryPolicy retryPolicy;
private final String filesystem;
private final AbfsConfiguration abfsConfiguration;
Expand Down Expand Up @@ -251,6 +252,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(
Expand All @@ -275,6 +278,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(
Expand Down Expand Up @@ -317,6 +322,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(
Expand All @@ -334,6 +341,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(
Expand Down Expand Up @@ -1090,6 +1099,15 @@ 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);
} else {
sreeb-msft marked this conversation as resolved.
Show resolved Hide resolved
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 @@ -1121,8 +1139,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 Expand Up @@ -1156,7 +1173,7 @@ protected URL createRequestUrl(final String path, final String query)
} catch (AzureBlobFileSystemException ex) {
LOG.debug("Unexpected error.", ex);
throw new InvalidUriException(path);
}
}
sreeb-msft marked this conversation as resolved.
Show resolved Hide resolved

final StringBuilder sb = new StringBuilder();
sb.append(base);
Expand Down
19 changes: 19 additions & 0 deletions hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
Original file line number Diff line number Diff line change
Expand Up @@ -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.

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,136 @@
/**
* 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
sreeb-msft marked this conversation as resolved.
Show resolved Hide resolved
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);
Comment on lines +106 to +110
Copy link
Contributor

Choose a reason for hiding this comment

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

As per https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-azure/pom.xml#L566-L613, this test would be run in parallel in same JVM process of other test, unset and set config can have affect on other tests. Kindly check if synchronization can be done. If not, we can add this class in https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-azure/pom.xml#L643-L652


// 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
newTestFs.getFileStatus(new Path("/"));
Path testPath = new Path("/testCorrectSASToken");
newTestFs.create(testPath).close();
newTestFs.delete(new Path("/"), true);
}

/**
* 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(expected = TokenAccessProviderException.class)
public void testBothProviderFixedTokenUnset() throws IOException {
AbfsConfiguration testAbfsConfig = getConfiguration();

testAbfsConfig.unset(FS_AZURE_SAS_TOKEN_PROVIDER_TYPE);
testAbfsConfig.unset(FS_AZURE_SAS_FIXED_TOKEN);

AzureBlobFileSystem newTestFs = (AzureBlobFileSystem) FileSystem.newInstance(testAbfsConfig.getRawConfiguration());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down
Loading