HADOOP-18980. S3A credential provider remapping: make extensible (#6406)

Contributed by Viraj Jasani
This commit is contained in:
Viraj Jasani 2024-02-02 08:02:48 -09:00 committed by GitHub
parent be13e94843
commit 7504b8505f
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 376 additions and 3 deletions

View File

@ -25,6 +25,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
@ -480,6 +481,27 @@ public static Collection<String> getTrimmedStringCollection(String str){
return set;
}
/**
* Splits an "=" separated value <code>String</code>, trimming leading and
* trailing whitespace on each value after splitting by comma and new line separator.
*
* @param str a comma separated <code>String</code> with values, may be null
* @return a <code>Map</code> of <code>String</code> keys and values, empty
* Collection if null String input.
*/
public static Map<String, String> getTrimmedStringCollectionSplitByEquals(
String str) {
String[] trimmedList = getTrimmedStrings(str);
Map<String, String> pairs = new HashMap<>();
for (String s : trimmedList) {
String[] splitByKeyVal = getTrimmedStringsSplitByEquals(s);
if (splitByKeyVal.length == 2) {
pairs.put(splitByKeyVal[0], splitByKeyVal[1]);
}
}
return pairs;
}
/**
* Splits a comma or newline separated value <code>String</code>, trimming
* leading and trailing whitespace on each value.
@ -497,6 +519,22 @@ public static String[] getTrimmedStrings(String str){
return str.trim().split("\\s*[,\n]\\s*");
}
/**
* Splits "=" separated value <code>String</code>, trimming
* leading and trailing whitespace on each value.
*
* @param str an "=" separated <code>String</code> with values,
* may be null
* @return an array of <code>String</code> values, empty array if null String
* input
*/
public static String[] getTrimmedStringsSplitByEquals(String str){
if (null == str || str.trim().isEmpty()) {
return emptyStringArray;
}
return str.trim().split("\\s*=\\s*");
}
final public static String[] emptyStringArray = {};
final public static char COMMA = ',';
final public static String COMMA_STR = ",";

View File

@ -44,6 +44,8 @@
import org.apache.commons.lang3.time.FastDateFormat;
import org.apache.hadoop.test.UnitTestcaseTimeLimit;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.assertj.core.api.Assertions;
import org.junit.Test;
public class TestStringUtils extends UnitTestcaseTimeLimit {
@ -512,6 +514,60 @@ public void testCreateStartupShutdownMessage() {
assertTrue(msg.startsWith("STARTUP_MSG:"));
}
@Test
public void testStringCollectionSplitByEquals() {
Map<String, String> splitMap =
StringUtils.getTrimmedStringCollectionSplitByEquals("");
Assertions
.assertThat(splitMap)
.describedAs("Map of key value pairs split by equals(=) and comma(,)")
.hasSize(0);
splitMap = StringUtils.getTrimmedStringCollectionSplitByEquals(null);
Assertions
.assertThat(splitMap)
.describedAs("Map of key value pairs split by equals(=) and comma(,)")
.hasSize(0);
splitMap = StringUtils.getTrimmedStringCollectionSplitByEquals(
"element.first.key1 = element.first.val1");
Assertions
.assertThat(splitMap)
.describedAs("Map of key value pairs split by equals(=) and comma(,)")
.hasSize(1)
.containsEntry("element.first.key1", "element.first.val1");
splitMap = StringUtils.getTrimmedStringCollectionSplitByEquals(
"element.xyz.key1 =element.abc.val1 , element.xyz.key2= element.abc.val2");
Assertions
.assertThat(splitMap)
.describedAs("Map of key value pairs split by equals(=) and comma(,)")
.hasSize(2)
.containsEntry("element.xyz.key1", "element.abc.val1")
.containsEntry("element.xyz.key2", "element.abc.val2");
splitMap = StringUtils.getTrimmedStringCollectionSplitByEquals(
"\nelement.xyz.key1 =element.abc.val1 \n"
+ ", element.xyz.key2=element.abc.val2,element.xyz.key3=element.abc.val3"
+ " , element.xyz.key4 =element.abc.val4,element.xyz.key5= "
+ "element.abc.val5 ,\n \n \n "
+ " element.xyz.key6 = element.abc.val6 \n , \n"
+ "element.xyz.key7=element.abc.val7,\n");
Assertions
.assertThat(splitMap)
.describedAs("Map of key value pairs split by equals(=) and comma(,)")
.hasSize(7)
.containsEntry("element.xyz.key1", "element.abc.val1")
.containsEntry("element.xyz.key2", "element.abc.val2")
.containsEntry("element.xyz.key3", "element.abc.val3")
.containsEntry("element.xyz.key4", "element.abc.val4")
.containsEntry("element.xyz.key5", "element.abc.val5")
.containsEntry("element.xyz.key6", "element.abc.val6")
.containsEntry("element.xyz.key7", "element.abc.val7");
}
// Benchmark for StringUtils split
public static void main(String []args) {
final String TO_SPLIT = "foo,bar,baz,blah,blah";

View File

@ -68,6 +68,13 @@ private Constants() {
public static final String AWS_CREDENTIALS_PROVIDER =
"fs.s3a.aws.credentials.provider";
/**
* AWS credentials providers mapping with key/value pairs.
* Value = {@value}
*/
public static final String AWS_CREDENTIALS_PROVIDER_MAPPING =
"fs.s3a.aws.credentials.provider.mapping";
/**
* Extra set of security credentials which will be prepended to that
* set in {@code "hadoop.security.credential.provider.path"}.

View File

@ -66,6 +66,7 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@ -1670,4 +1671,26 @@ public static String formatRange(long rangeStart, long rangeEnd) {
return String.format("bytes=%d-%d", rangeStart, rangeEnd);
}
/**
* Get the equal op (=) delimited key-value pairs of the <code>name</code> property as
* a collection of pair of <code>String</code>s, trimmed of the leading and trailing whitespace
* after delimiting the <code>name</code> by comma and new line separator.
* If no such property is specified then empty <code>Map</code> is returned.
*
* @param configuration the configuration object.
* @param name property name.
* @return property value as a <code>Map</code> of <code>String</code>s, or empty
* <code>Map</code>.
*/
public static Map<String, String> getTrimmedStringCollectionSplitByEquals(
final Configuration configuration,
final String name) {
String valueString = configuration.get(name);
if (null == valueString) {
return new HashMap<>();
}
return org.apache.hadoop.util.StringUtils
.getTrimmedStringCollectionSplitByEquals(valueString);
}
}

View File

@ -51,6 +51,7 @@
import org.apache.hadoop.fs.store.LogExactlyOnce;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER_MAPPING;
import static org.apache.hadoop.fs.s3a.adapter.AwsV1BindingSupport.isAwsV1SdkAvailable;
/**
@ -216,6 +217,9 @@ public static AWSCredentialProviderList buildAWSProviderList(
key,
defaultValues.toArray(new Class[defaultValues.size()]));
Map<String, String> awsCredsMappedClasses =
S3AUtils.getTrimmedStringCollectionSplitByEquals(conf,
AWS_CREDENTIALS_PROVIDER_MAPPING);
Map<String, String> v1v2CredentialProviderMap = V1_V2_CREDENTIAL_PROVIDER_MAP;
final Set<String> forbiddenClassnames =
forbidden.stream().map(c -> c.getName()).collect(Collectors.toSet());
@ -232,6 +236,10 @@ public static AWSCredentialProviderList buildAWSProviderList(
LOG_REMAPPED_ENTRY.warn("Credentials option {} contains AWS v1 SDK entry {}; mapping to {}",
key, className, mapped);
className = mapped;
} else if (awsCredsMappedClasses != null && awsCredsMappedClasses.containsKey(className)) {
final String mapped = awsCredsMappedClasses.get(className);
LOG_REMAPPED_ENTRY.debug("Credential entry {} is mapped to {}", className, mapped);
className = mapped;
}
// now scan the forbidden list. doing this after any mappings ensures the v1 names
// are also blocked

View File

@ -66,6 +66,55 @@ The change in interface will mean that custom credential providers will need to
implement `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider` instead of
`com.amazonaws.auth.AWSCredentialsProvider`.
[HADOOP-18980](https://issues.apache.org/jira/browse/HADOOP-18980) introduces extended version of
the credential provider remapping. `fs.s3a.aws.credentials.provider.mapping` can be used to
list comma-separated key-value pairs of mapped credential providers that are separated by
equal operator (=).
The key can be used by `fs.s3a.aws.credentials.provider` or
`fs.s3a.assumed.role.credentials.provider` configs, and the key will be translated into
the specified value of credential provider class based on the key-value pair
provided by the config `fs.s3a.aws.credentials.provider.mapping`.
For example, if `fs.s3a.aws.credentials.provider.mapping` is set with value:
```xml
<property>
<name>fs.s3a.aws.credentials.provider.mapping</name>
<vale>
com.amazonaws.auth.AnonymousAWSCredentials=org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider,
com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper=org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider,
com.amazonaws.auth.InstanceProfileCredentialsProvider=org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
</vale>
</property>
```
and if `fs.s3a.aws.credentials.provider` is set with:
```xml
<property>
<name>fs.s3a.aws.credentials.provider</name>
<vale>com.amazonaws.auth.AnonymousAWSCredentials</vale>
</property>
```
`com.amazonaws.auth.AnonymousAWSCredentials` will be internally remapped to
`org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` by S3A while preparing
the AWS credential provider list.
Similarly, if `fs.s3a.assumed.role.credentials.provider` is set with:
```xml
<property>
<name>fs.s3a.assumed.role.credentials.provider</name>
<vale>com.amazonaws.auth.InstanceProfileCredentialsProvider</vale>
</property>
```
`com.amazonaws.auth.InstanceProfileCredentialsProvider` will be internally
remapped to `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` by
S3A while preparing the assumed role AWS credential provider list.
### Original V1 `AWSCredentialsProvider` interface
Note how the interface begins with the capitalized "AWS" acronym.

View File

@ -280,6 +280,28 @@ For more information see [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade
credentials.
</description>
</property>
<property>
<name>fs.s3a.aws.credentials.provider.mapping</name>
<description>
Comma-separated key-value pairs of mapped credential providers that are
separated by equal operator (=). The key can be used by
fs.s3a.aws.credentials.provider config, and it will be translated into
the specified value of credential provider class based on the key-value
pair provided by this config.
Example:
com.amazonaws.auth.AnonymousAWSCredentials=org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider,
com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper=org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider,
com.amazonaws.auth.InstanceProfileCredentialsProvider=org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
With the above key-value pairs, if fs.s3a.aws.credentials.provider specifies
com.amazonaws.auth.AnonymousAWSCredentials, it will be remapped to
org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider by S3A while
preparing AWS credential provider list for any S3 access.
We can use the same credentials provider list for both v1 and v2 SDK clients.
</description>
</property>
```
### <a name="auth_env_vars"></a> Authenticating via the AWS Environment Variables

View File

@ -108,6 +108,23 @@ public void testBadCredentialsConstructor() throws Exception {
}
}
/**
* Test aws credentials provider remapping with key that maps to
* BadCredentialsProviderConstructor.
*/
@Test
public void testBadCredentialsConstructorWithRemap() throws Exception {
Configuration conf = createConf("aws.test.map1");
conf.set(AWS_CREDENTIALS_PROVIDER_MAPPING,
"aws.test.map1=" + BadCredentialsProviderConstructor.class.getName());
final InstantiationIOException ex =
intercept(InstantiationIOException.class, CONSTRUCTOR_EXCEPTION, () ->
createFailingFS(conf));
if (InstantiationIOException.Kind.UnsupportedConstructor != ex.getKind()) {
throw ex;
}
}
/**
* Create a configuration bonded to the given provider classname.
* @param provider provider to bond to
@ -169,6 +186,20 @@ public void testBadCredentials() throws Exception {
createFailingFS(conf));
}
/**
* Test aws credentials provider remapping with key that maps to
* BadCredentialsProvider.
*/
@Test
public void testBadCredentialsWithRemap() throws Exception {
Configuration conf = createConf("aws.test.map.key");
conf.set(AWS_CREDENTIALS_PROVIDER_MAPPING,
"aws.test.map.key=" + BadCredentialsProvider.class.getName());
intercept(AccessDeniedException.class,
"",
() -> createFailingFS(conf));
}
/**
* Test using the anonymous credential provider with the public csv
* test file; if the test file path is unset then it will be skipped.

View File

@ -25,7 +25,9 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
@ -47,14 +49,17 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider;
import org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider;
import org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory;
import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
import org.apache.hadoop.fs.s3a.auth.delegation.CountInvocationsProvider;
import org.apache.hadoop.fs.s3a.impl.InstantiationIOException;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.util.Sets;
import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER_MAPPING;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.DEFAULT_CSVTEST_FILE;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.authenticationContains;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.buildClassListString;
@ -206,6 +211,66 @@ public void testFallbackToDefaults() throws Throwable {
assertTrue("empty credentials", credentials.size() > 0);
}
/**
* Test S3A credentials provider remapping with assumed role
* credentials provider.
*/
@Test
public void testAssumedRoleWithRemap() throws Throwable {
Configuration conf = new Configuration(false);
conf.set(ASSUMED_ROLE_CREDENTIALS_PROVIDER,
"custom.assume.role.key1,custom.assume.role.key2,custom.assume.role.key3");
conf.set(AWS_CREDENTIALS_PROVIDER_MAPPING,
"custom.assume.role.key1="
+ CredentialProviderListFactory.ENVIRONMENT_CREDENTIALS_V2
+ " ,custom.assume.role.key2 ="
+ CountInvocationsProvider.NAME
+ ", custom.assume.role.key3= "
+ CredentialProviderListFactory.PROFILE_CREDENTIALS_V1);
final AWSCredentialProviderList credentials =
buildAWSProviderList(
new URI("s3a://bucket1"),
conf,
ASSUMED_ROLE_CREDENTIALS_PROVIDER,
new ArrayList<>(),
new HashSet<>());
Assertions
.assertThat(credentials.size())
.describedAs("List of Credentials providers")
.isEqualTo(3);
}
/**
* Test S3A credentials provider remapping with aws
* credentials provider.
*/
@Test
public void testAwsCredentialProvidersWithRemap() throws Throwable {
Configuration conf = new Configuration(false);
conf.set(AWS_CREDENTIALS_PROVIDER,
"custom.aws.creds.key1,custom.aws.creds.key2,custom.aws.creds.key3,custom.aws.creds.key4");
conf.set(AWS_CREDENTIALS_PROVIDER_MAPPING,
"custom.aws.creds.key1="
+ CredentialProviderListFactory.ENVIRONMENT_CREDENTIALS_V2
+ " ,\ncustom.aws.creds.key2="
+ CountInvocationsProvider.NAME
+ "\n, custom.aws.creds.key3="
+ CredentialProviderListFactory.PROFILE_CREDENTIALS_V1
+ ",custom.aws.creds.key4 = "
+ CredentialProviderListFactory.PROFILE_CREDENTIALS_V2);
final AWSCredentialProviderList credentials =
buildAWSProviderList(
new URI("s3a://bucket1"),
conf,
AWS_CREDENTIALS_PROVIDER,
new ArrayList<>(),
new HashSet<>());
Assertions
.assertThat(credentials.size())
.describedAs("List of Credentials providers")
.isEqualTo(4);
}
@Test
public void testProviderConstructor() throws Throwable {
final AWSCredentialProviderList list = new AWSCredentialProviderList("name",
@ -656,6 +721,80 @@ public void testV2ClassNotFound() throws Throwable {
LOG.info("{}", expected.toString());
}
/**
* Tests for the string utility that will be used by S3A credentials provider.
*/
@Test
public void testStringCollectionSplitByEquals() {
final Configuration configuration = new Configuration();
configuration.set("custom_key", "");
Map<String, String> splitMap =
S3AUtils.getTrimmedStringCollectionSplitByEquals(
configuration, "custom_key");
Assertions
.assertThat(splitMap)
.describedAs(
"Map of key value pairs derived from config, split by equals(=) and comma(,)")
.hasSize(0);
splitMap =
S3AUtils.getTrimmedStringCollectionSplitByEquals(
configuration, "not_present");
Assertions
.assertThat(splitMap)
.describedAs(
"Map of key value pairs derived from config, split by equals(=) and comma(,)")
.hasSize(0);
configuration.set("custom_key", "element.first.key1 = element.first.val1");
splitMap = S3AUtils.getTrimmedStringCollectionSplitByEquals(
configuration, "custom_key");
Assertions
.assertThat(splitMap)
.describedAs(
"Map of key value pairs derived from config, split by equals(=) and comma(,)")
.hasSize(1)
.containsEntry("element.first.key1", "element.first.val1");
configuration.set("custom_key",
"element.xyz.key1 =element.abc.val1 , element.xyz.key2= element.abc.val2");
splitMap =
S3AUtils.getTrimmedStringCollectionSplitByEquals(
configuration, "custom_key");
Assertions
.assertThat(splitMap)
.describedAs(
"Map of key value pairs derived from config, split by equals(=) and comma(,)")
.hasSize(2)
.containsEntry("element.xyz.key1", "element.abc.val1")
.containsEntry("element.xyz.key2", "element.abc.val2");
configuration.set("custom_key",
"\nelement.xyz.key1 =element.abc.val1 \n"
+ ", element.xyz.key2=element.abc.val2,element.xyz.key3=element.abc.val3"
+ " , element.xyz.key4 =element.abc.val4,element.xyz.key5= "
+ "element.abc.val5 ,\n \n \n "
+ " element.xyz.key6 = element.abc.val6 \n , \n"
+ "element.xyz.key7=element.abc.val7,\n");
splitMap = S3AUtils.getTrimmedStringCollectionSplitByEquals(
configuration, "custom_key");
Assertions
.assertThat(splitMap)
.describedAs(
"Map of key value pairs derived from config, split by equals(=) and comma(,)")
.hasSize(7)
.containsEntry("element.xyz.key1", "element.abc.val1")
.containsEntry("element.xyz.key2", "element.abc.val2")
.containsEntry("element.xyz.key3", "element.abc.val3")
.containsEntry("element.xyz.key4", "element.abc.val4")
.containsEntry("element.xyz.key5", "element.abc.val5")
.containsEntry("element.xyz.key6", "element.abc.val6")
.containsEntry("element.xyz.key7", "element.abc.val7");
}
/**
* V2 credentials which raises an instantiation exception in
* the factory method.