pranavsaxena-microsoft commented on code in PR #5148: URL: https://github.com/apache/hadoop/pull/5148#discussion_r1056036544
########## hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java: ########## @@ -914,31 +914,50 @@ public AccessTokenProvider getTokenProvider() throws TokenAccessProviderExceptio } } + /** + * @return sasTokenProvider object + * @throws AzureBlobFileSystemException + * 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, Review Comment: Can we rewrite the condition as: !(sasTokenProviderImplementation == null && sasTokenProviderImplementation == null). Reason being, from reading perspective its not natural to understand. Knowledge what checkArgument does is not clear without reading the impl: ``` public static void checkArgument(boolean expression, Object errorMessage) { if (!expression) { throw new IllegalArgumentException(String.valueOf(errorMessage)); } } ``` So `!(sasTokenProviderImplementation == null && sasTokenProviderImplementation == null)` will tell readers that both should not be null. ########## hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java: ########## @@ -914,31 +914,50 @@ public AccessTokenProvider getTokenProvider() throws TokenAccessProviderExceptio } } + /** + * @return sasTokenProvider object + * @throws AzureBlobFileSystemException Review Comment: return and throws should come after the description. ########## hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java: ########## @@ -1166,7 +1183,7 @@ protected URL createRequestUrl(final String path, final String query) } catch (AzureBlobFileSystemException ex) { LOG.debug("Unexpected error.", ex); throw new InvalidUriException(path); - } + } Review Comment: please do check the indentation. ########## hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/SASGenerator.java: ########## @@ -85,6 +88,22 @@ private void initializeMac() { } } + protected String getCanonicalAccountName(String accountName) throws InvalidConfigurationValueException { + // returns the account name without the endpoint + int dotIndex = accountName.indexOf(AbfsHttpConstants.DOT); Review Comment: please add in comments, what is the semantic of string it is going to operate on. ########## hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java: ########## @@ -1096,6 +1105,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 { Review Comment: nit: we can remove else. Since in if-block, the method will terminate. Outside if-block it would be else part only. ########## hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java: ########## @@ -0,0 +1,144 @@ +/** + * 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 + 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); Review Comment: 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 ########## hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChooseSAS.java: ########## @@ -0,0 +1,144 @@ +/** + * 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 Review Comment: why is it expected to fail. Agree DelegationSAS will be taken, but what is the point which would raise exception. ########## hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java: ########## @@ -914,31 +914,50 @@ public AccessTokenProvider getTokenProvider() throws TokenAccessProviderExceptio } } + /** + * @return sasTokenProvider object + * @throws AzureBlobFileSystemException + * 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, Review Comment: Please do checkstyle of the code. I was reading on IDE, many lines are outside 80 mark. Reformatting shortkey: ctrl+alt+L -- 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: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org