Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HADOOP-19384. S3A: Add support for ProfileCredentialsProvider #7284

Open
wants to merge 10 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1442,6 +1442,7 @@
token binding it may be used
to communicate wih the STS endpoint to request session/role
credentials.
org.apache.hadoop.fs.s3a.auth.ProfileAWSCredentialsProvider is also supported, but is not enabled by default.
</description>
</property>

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/**
* 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.s3a.auth;

import java.net.URI;
import java.nio.file.FileSystems;
import java.nio.file.Path;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import software.amazon.awssdk.auth.credentials.AwsCredentials;
import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider;
import software.amazon.awssdk.profiles.ProfileFile;

import org.apache.commons.lang3.SystemUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;

@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ProfileAWSCredentialsProvider extends AbstractAWSCredentialProvider {
Copy link
Contributor

Choose a reason for hiding this comment

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

I wonder if we should be logging at debug when the env vars are being used? Nothing secret will be logged and it could be useful.

private static final Logger LOG = LoggerFactory.getLogger(ProfileAWSCredentialsProvider.class);

public static final String NAME
= "org.apache.hadoop.fs.s3a.auth.ProfileAWSCredentialsProvider";

/** Conf setting for credentials file path.*/
public static final String PROFILE_FILE = "fs.s3a.auth.profile.file";

/** Conf setting for profile name.*/
public static final String PROFILE_NAME = "fs.s3a.auth.profile.name";

/** Environment variable for credentials file path.*/
public static final String CREDENTIALS_FILE_ENV = "AWS_SHARED_CREDENTIALS_FILE";
/** Environment variable for profile name.*/
public static final String PROFILE_ENV = "AWS_PROFILE";

private final ProfileCredentialsProvider pcp;

private static Path getCredentialsPath(Configuration conf) {
String credentialsFile = conf.get(PROFILE_FILE, null);
if (credentialsFile == null) {
credentialsFile = SystemUtils.getEnvironmentVariable(CREDENTIALS_FILE_ENV, null);
if (credentialsFile != null) {
LOG.debug("Fetched credentials file path from environment variable");
}
} else {
LOG.debug("Fetched credentials file path from conf");
}
if (credentialsFile == null) {
LOG.debug("Using default credentials file path");
return FileSystems.getDefault().getPath(SystemUtils.getUserHome().getPath(),
".aws", "credentials");
} else {
return FileSystems.getDefault().getPath(credentialsFile);
}
}

private static String getCredentialsName(Configuration conf) {
String profileName = conf.get(PROFILE_NAME, null);
if (profileName == null) {
profileName = SystemUtils.getEnvironmentVariable(PROFILE_ENV, null);
if (profileName == null) {
profileName = "default";
LOG.debug("Using default profile name");
} else {
LOG.debug("Fetched profile name from environment variable");
}
} else {
LOG.debug("Fetched profile name from conf");
}
return profileName;
}

public ProfileAWSCredentialsProvider(URI uri, Configuration conf) {
super(uri, conf);
ProfileCredentialsProvider.Builder builder = ProfileCredentialsProvider.builder();
builder.profileName(getCredentialsName(conf))
.profileFile(ProfileFile.builder()
.content(getCredentialsPath(conf))
.type(ProfileFile.Type.CREDENTIALS)
.build());
pcp = builder.build();
}

public AwsCredentials resolveCredentials() {
return pcp.resolveCredentials();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,9 @@ For more information see [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade
Comma-separated class names of credential provider classes which implement
software.amazon.awssdk.auth.credentials.AwsCredentialsProvider.

org.apache.hadoop.fs.s3a.auth.ProfileAWSCredentialsProvider is not included in
the chain by default.

When S3A delegation tokens are not enabled, this list will be used
to directly authenticate with S3 and other AWS services.
When S3A Delegation tokens are enabled, depending upon the delegation
Expand Down Expand Up @@ -171,6 +174,7 @@ There are a number of AWS Credential Providers inside the `hadoop-aws` JAR:
| `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` | Anonymous Login |
| `org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider` | [Assumed Role credentials](./assumed_roles.html) |
| `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | EC2/k8s instance credentials |
| `org.apache.hadoop.fs.s3a.auth.ProfileAWSCredentialsProvider` | Session Credentials in profile file |


There are also many in the Amazon SDKs, with the common ones being as follows
Expand Down Expand Up @@ -222,6 +226,25 @@ Note:
configuration files MUST be in the `~/.aws/` directory on the local filesystem in
all hosts in the cluster.

### <a name="auth_simple"></a> Credentials from profile with `ProfileAWSCredentialsProvider`*

This is a non-default provider that fetches credentials from a profile file,
acting as a Hadoop wrapper around [ProfileCredentialsProvider](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/auth/credentials/ProfileCredentialsProvider.html). The profile file and
profile name are both resolved as follows.

1. If the configuration setting is specified, that takes priority ( `fs.s3a.auth.profile.file`
for profile file and `fs.s3a.auth.profile.name` for profile name).
2. If a configuration setting is absent, but the environment variable for
the setting( `AWS_SHARED_CREDENTIALS_FILE` for profile file and `AWS_PROFILE` for
profile name) is defined, then the variable is used.
3. If neither configuration setting nor environment variable is present, then
the values default to `~/.aws/credentials` for the profile file, and `default`
for the profile name.

Copy link
Contributor

Choose a reason for hiding this comment

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

Add the paragraph


*Important*: This profile file must be on every node in the _cluster_.
If this is not the case, delegation tokens can be used to collect the current credentials and propagate them


*Important*: This profile file must be on every node in the _cluster_.
If this is not the case, delegation tokens can be used to collect the current credentials and propagate them.

### <a name="auth_session"></a> Using Session Credentials with `TemporaryAWSCredentialsProvider`

[Temporary Security Credentials](http://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_temp.html)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,11 @@

package org.apache.hadoop.fs.s3a;

import java.io.IOException;
import java.io.BufferedWriter;
import java.io.File;
import java.io.FileWriter;
import java.io.InterruptedIOException;
import java.io.IOException;
import java.net.URI;
import java.nio.file.AccessDeniedException;
import java.util.ArrayList;
Expand Down Expand Up @@ -52,6 +55,7 @@
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.ProfileAWSCredentialsProvider;
import org.apache.hadoop.fs.s3a.auth.delegation.CountInvocationsProvider;
import org.apache.hadoop.fs.s3a.impl.InstantiationIOException;
import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
Expand Down Expand Up @@ -139,6 +143,35 @@ public void testInstantiationChain() throws Throwable {
assertCredentialProviders(expectedClasses, list);
}

@Test
public void testProfileAWSCredentialsProvider() throws Throwable {
Configuration conf = new Configuration(false);
conf.set(AWS_CREDENTIALS_PROVIDER, ProfileAWSCredentialsProvider.NAME);
File tempFile = File.createTempFile("testcred", ".conf", new File("target"));
tempFile.deleteOnExit();
try (FileWriter fileWriter = new FileWriter(tempFile);
BufferedWriter bufferedWriter = new BufferedWriter(fileWriter)) {
bufferedWriter.write("[default]\n"
+ "aws_access_key_id = defaultaccesskeyid\n"
+ "aws_secret_access_key = defaultsecretkeyid\n");
bufferedWriter.write("[nondefault]\n"
+ "aws_access_key_id = nondefaultaccesskeyid\n"
+ "aws_secret_access_key = nondefaultsecretkeyid\n");
}
conf.set(ProfileAWSCredentialsProvider.PROFILE_FILE, tempFile.getAbsolutePath());
URI testUri = new URI("s3a://bucket1");
AWSCredentialProviderList list = createAWSCredentialProviderList(testUri, conf);
assertCredentialProviders(Collections.singletonList(ProfileAWSCredentialsProvider.class), list);
AwsCredentials credentials = list.resolveCredentials();
Assertions.assertThat(credentials.accessKeyId()).isEqualTo("defaultaccesskeyid");
Assertions.assertThat(credentials.secretAccessKey()).isEqualTo("defaultsecretkeyid");
conf.set(ProfileAWSCredentialsProvider.PROFILE_NAME, "nondefault");
list = createAWSCredentialProviderList(testUri, conf);
credentials = list.resolveCredentials();
Assertions.assertThat(credentials.accessKeyId()).isEqualTo("nondefaultaccesskeyid");
Assertions.assertThat(credentials.secretAccessKey()).isEqualTo("nondefaultsecretkeyid");
}

@Test
public void testDefaultChain() throws Exception {
URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2");
Expand Down