steveloughran commented on code in PR #6552:
URL: https://github.com/apache/hadoop/pull/6552#discussion_r1489922525
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java:
##########
@@ -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,
+ * 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
+ && configuredFixedToken == null),
+ String.format(
+ "The value for both \"%s\" and \"%s\" cannot be invalid.",
Review Comment:
prefer a message "at least one of %s and %s must be set"
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java:
##########
@@ -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,
Review Comment:
nit: use html ol and li elements so it renders properly (in IDEs as well as
javadocs}
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java:
##########
@@ -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,
+ * 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
+ && 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",
Review Comment:
you don't need to use String.format; checkArgument takes a format string and
list of arguments to generate it on demand
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java:
##########
@@ -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 **/
Review Comment:
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
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java:
##########
@@ -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,
+ * 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
+ && configuredFixedToken == null),
+ String.format(
Review Comment:
you don't need to use String.format; checkArgument takes a format string and
list of arguments to generate it on demand
##########
hadoop-tools/hadoop-azure/src/site/markdown/abfs.md:
##########
@@ -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.
Review Comment:
numbering is out of order. This is why using 1. everywhere is easier: less
maintenance. markdown renders will generate the correct numbering for you
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java:
##########
@@ -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,
+ * 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
Review Comment:
what if both of them are set?
##########
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
Review Comment:
you can cut this from test javadocs
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -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
Review Comment:
make the javadoc of the new method
##########
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;
Review Comment:
import ordering doesn't match style rules for new code.
##########
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) {
Review Comment:
don't catch, you've just lost *the entire stack trace*. let the test handler
catch and report the problem. always
##########
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
Review Comment:
nit: lines seem too long; if over 100 chars split to multiple lines
##########
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{
Review Comment:
add a javadoc to say what the test does
##########
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
Review Comment:
add .
##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockSASTokenProvider.java:
##########
@@ -35,10 +39,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());
Review Comment:
this likely to happen? it
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -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
Review Comment:
nit: use "append" over "appending". or, given it is in the method name, cut
entirely here and below
##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockSASTokenProvider.java:
##########
@@ -35,10 +39,19 @@ public class MockSASTokenProvider implements
SASTokenProvider {
private byte[] accountKey;
private ServiceSASGenerator generator;
private boolean skipAuthorizationForTestSetup = false;
+ protected static final Logger LOG =
Review Comment:
why protected?
##########
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;
Review Comment:
import ordering
##########
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
Review Comment:
.
##########
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");
Review Comment:
prefer MockDelegationSASTokenProvider.class.getName()
##########
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());
Review Comment:
this needs to be closed() after use, so use in a try-with-resources clause
##########
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";
Review Comment:
this specific version isn't in the linked doc or the service version page
off it. does that matter?
##########
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);
Review Comment:
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).
##########
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());
+ });
+ }
+}
Review Comment:
nit, add a trailing line
##########
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);
+ }
+}
Review Comment:
nit, add a newline
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]