HADOOP-14802. Add support for using container saskeys for all accesses.

Contributed by Sivaguru Sankaridurg

(cherry picked from commit 021974f4cb)

Conflicts:
	hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
This commit is contained in:
Steve Loughran 2017-08-30 22:40:50 +01:00
parent 7dd5bb9492
commit 25e9c29608
No known key found for this signature in database
GPG Key ID: 950CC3E032B79CA2
5 changed files with 84 additions and 9 deletions

View File

@ -1385,6 +1385,16 @@
</description>
</property>
<property>
<name>fs.azure.saskey.usecontainersaskeyforallaccess</name>
<value>true</value>
<description>
Use container saskey for access to all blobs within the container.
Blob-specific saskeys are not used when this setting is enabled.
This setting provides better performance compared to blob-specific saskeys.
</description>
</property>
<!-- Azure Data Lake File System Configurations -->
<property>

View File

@ -125,6 +125,7 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
xmlPropsToSkipCompare.add("fs.azure.secure.mode");
xmlPropsToSkipCompare.add("fs.azure.authorization");
xmlPropsToSkipCompare.add("fs.azure.authorization.caching.enable");
xmlPropsToSkipCompare.add("fs.azure.saskey.usecontainersaskeyforallaccess");
xmlPropsToSkipCompare.add("fs.azure.user.agent.prefix");
// ADL properties are in a different subtree

View File

@ -71,6 +71,13 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
private String storageAccount;
private RetryPolicyFactory retryPolicy;
private int timeoutIntervalInMs;
private boolean useContainerSasKeyForAllAccess;
/**
* Configuration key to specify if containerSasKey should be used for all accesses
*/
public static final String KEY_USE_CONTAINER_SASKEY_FOR_ALL_ACCESS =
"fs.azure.saskey.usecontainersaskeyforallaccess";
public SecureStorageInterfaceImpl(boolean useLocalSASKeyMode,
Configuration conf) throws SecureModeException {
@ -88,6 +95,7 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
}
this.sasKeyGenerator = remoteSasKeyGenerator;
}
this.useContainerSasKeyForAllAccess = conf.getBoolean(KEY_USE_CONTAINER_SASKEY_FOR_ALL_ACCESS, true);
}
@Override
@ -145,7 +153,9 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
if (timeoutIntervalInMs > 0) {
container.getServiceClient().getDefaultRequestOptions().setTimeoutIntervalInMs(timeoutIntervalInMs);
}
return new SASCloudBlobContainerWrapperImpl(storageAccount, container, sasKeyGenerator);
return (useContainerSasKeyForAllAccess)
? new SASCloudBlobContainerWrapperImpl(storageAccount, container, null)
: new SASCloudBlobContainerWrapperImpl(storageAccount, container, sasKeyGenerator);
} catch (SASKeyGenerationException sasEx) {
String errorMsg = "Encountered SASKeyGeneration exception while "
+ "generating SAS Key for container : " + name
@ -226,12 +236,12 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
public CloudBlobWrapper getBlockBlobReference(String relativePath)
throws URISyntaxException, StorageException {
try {
CloudBlockBlob blob = new CloudBlockBlob(sasKeyGenerator.getRelativeBlobSASUri(
storageAccount, getName(), relativePath));
CloudBlockBlob blob = (sasKeyGenerator!=null)
? new CloudBlockBlob(sasKeyGenerator.getRelativeBlobSASUri(storageAccount, getName(), relativePath))
: container.getBlockBlobReference(relativePath);
blob.getServiceClient().setDefaultRequestOptions(
container.getServiceClient().getDefaultRequestOptions());
return new SASCloudBlockBlobWrapperImpl(
blob);
return new SASCloudBlockBlobWrapperImpl(blob);
} catch (SASKeyGenerationException sasEx) {
String errorMsg = "Encountered SASKeyGeneration exception while "
+ "generating SAS Key for relativePath : " + relativePath
@ -245,12 +255,13 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
public CloudBlobWrapper getPageBlobReference(String relativePath)
throws URISyntaxException, StorageException {
try {
CloudPageBlob blob = new CloudPageBlob(sasKeyGenerator.getRelativeBlobSASUri(
storageAccount, getName(), relativePath));
CloudPageBlob blob = (sasKeyGenerator!=null)
? new CloudPageBlob(sasKeyGenerator.getRelativeBlobSASUri(storageAccount, getName(), relativePath))
: container.getPageBlobReference(relativePath);
blob.getServiceClient().setDefaultRequestOptions(
container.getServiceClient().getDefaultRequestOptions());
return new SASCloudPageBlobWrapperImpl(
blob);
return new SASCloudPageBlobWrapperImpl(blob);
} catch (SASKeyGenerationException sasEx) {
String errorMsg = "Encountered SASKeyGeneration exception while "
+ "generating SAS Key for relativePath : " + relativePath

View File

@ -473,6 +473,15 @@ The maximum number of entries that that cache can hold can be customized using t
</property>
```
Use container saskey for access to all blobs within the container.
Blob-specific saskeys are not used when this setting is enabled.
This setting provides better performance compared to blob-specific saskeys.
```
<property>
<name>fs.azure.saskey.usecontainersaskeyforallaccess</name>
<value>true</value>
</property>
```
## Testing the hadoop-azure Module
The hadoop-azure module includes a full suite of unit tests. Most of the tests

View File

@ -0,0 +1,44 @@
/**
* 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.azure;
import org.apache.hadoop.conf.Configuration;
import static org.apache.hadoop.fs.azure.SecureStorageInterfaceImpl.KEY_USE_CONTAINER_SASKEY_FOR_ALL_ACCESS;
/**
* Test class to hold all WASB authorization tests that use blob-specific keys
* to access storage.
*/
public class TestNativeAzureFSAuthWithBlobSpecificKeys
extends TestNativeAzureFileSystemAuthorizationWithOwner {
@Override
public Configuration getConfiguration() {
Configuration conf = super.getConfiguration();
conf.set(KEY_USE_CONTAINER_SASKEY_FOR_ALL_ACCESS, "false");
return conf;
}
@Override
protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
Configuration conf = getConfiguration();
return AzureBlobStorageTestAccount.create(conf);
}
}