Skip to content
Merged
Show file tree
Hide file tree
Changes from 5 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,56 @@
package org.apache.hadoop.fs.s3a.auth;

Check failure on line 1 in hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/ProfileAWSCredentialsProvider.java

View check run for this annotation

ASF Cloudbees Jenkins ci-hadoop / Apache Yetus

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/ProfileAWSCredentialsProvider.java#L1

asflicense: Missing Apache License

import software.amazon.awssdk.auth.credentials.AwsCredentials;
import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider;
import software.amazon.awssdk.profiles.ProfileFile;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

import org.apache.commons.lang3.SystemUtils;
import org.apache.hadoop.conf.Configuration;

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

@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.

public static final String NAME
= "org.apache.hadoop.fs.s3a.auth.ProfileAWSCredentialsProvider";
public static final String PROFILE_FILE = "fs.s3a.auth.profile.file";
Copy link
Contributor

Choose a reason for hiding this comment

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

add javadocs for these and the field at L25

public static final String PROFILE_NAME = "fs.s3a.auth.profile.name";

private final ProfileCredentialsProvider pcp;

private static Path getCredentialsPath(Configuration conf) {
String credentialsFile = conf.get(PROFILE_FILE, null);
if (credentialsFile == null) {
credentialsFile = SystemUtils.getEnvironmentVariable("AWS_SHARED_CREDENTIALS_FILE", null);
Copy link
Contributor

Choose a reason for hiding this comment

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

move all env vars to string constants; add javadocs for each of these

}
Path path = (credentialsFile == null) ?
FileSystems.getDefault().getPath(SystemUtils.getUserHome().getPath(),".aws","credentials")
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: can you add a space after the commas in the arguments

: FileSystems.getDefault().getPath(credentialsFile);
return path;
}

private static String getCredentialsName(Configuration conf) {
String profileName = conf.get(PROFILE_NAME, null);
if (profileName == null) {
profileName = SystemUtils.getEnvironmentVariable("AWS_PROFILE", "default");
}
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());
Copy link
Contributor

Choose a reason for hiding this comment

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

should be split across lines, one per build attribute

pcp = builder.build();
}

public AwsCredentials resolveCredentials() {
return pcp.resolveCredentials();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,7 @@

package org.apache.hadoop.fs.s3a;

import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.*;
import java.net.URI;
import java.nio.file.AccessDeniedException;
import java.util.ArrayList;
Expand All @@ -35,6 +34,7 @@
import java.util.stream.Collectors;
import javax.annotation.Nullable;

import org.apache.hadoop.fs.s3a.auth.*;
Copy link
Contributor

Choose a reason for hiding this comment

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

revert

import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.slf4j.Logger;
Expand All @@ -47,11 +47,6 @@

import org.apache.hadoop.conf.Configuration;
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.fs.s3a.test.PublicDatasetTestUtils;
Expand Down Expand Up @@ -139,6 +134,32 @@ 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);
try (FileWriter fileWriter = new FileWriter("testcred"); 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, "testcred");
URI testUri = new URI("s3a://bucket1");
AWSCredentialProviderList list = createAWSCredentialProviderList(testUri, conf);
assertCredentialProviders(Collections.singletonList(ProfileAWSCredentialsProvider.class), list);
AwsCredentials credentials = list.resolveCredentials();
assertEquals("defaultaccesskeyid", credentials.accessKeyId());
assertEquals("defaultsecretkeyid", credentials.secretAccessKey());
conf.set(ProfileAWSCredentialsProvider.PROFILE_NAME, "nondefault");
list = createAWSCredentialProviderList(testUri, conf);
credentials = list.resolveCredentials();
assertEquals("nondefaultaccesskeyid", credentials.accessKeyId());
assertEquals("nondefaultsecretkeyid", credentials.secretAccessKey());
}

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