Custom AWSCredentialsProviderChain for the S3 storage module: supports existing druid config, file, environment variable, system property, profile and (most importantly) instance profile based credential options

This commit is contained in:
Glenn Nethercutt 2014-10-03 23:09:38 -04:00
parent 858aff396c
commit 39a7af28d6
1 changed files with 66 additions and 13 deletions

View File

@ -19,7 +19,13 @@
package io.druid.storage.s3;
import com.amazonaws.AmazonClientException;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSCredentialsProviderChain;
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
import com.amazonaws.auth.SystemPropertiesCredentialsProvider;
import com.amazonaws.auth.profile.ProfileCredentialsProvider;
import com.fasterxml.jackson.databind.Module;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
@ -62,14 +68,18 @@ public class S3StorageDruidModule implements DruidModule
binder.bind(S3TaskLogs.class).in(LazySingleton.class);
}
@Provides
@LazySingleton
public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config)
private class ConfigDrivenAwsCredentialsConfigProvider implements AWSCredentialsProvider
{
private AWSCredentialsConfig config;
public ConfigDrivenAwsCredentialsConfigProvider(AWSCredentialsConfig config) {
this.config = config;
}
@Override
public com.amazonaws.auth.AWSCredentials getCredentials()
{
if (!Strings.isNullOrEmpty(config.getAccessKey()) && !Strings.isNullOrEmpty(config.getSecretKey())) {
return new AWSCredentialsProvider() {
@Override
public com.amazonaws.auth.AWSCredentials getCredentials() {
return new com.amazonaws.auth.AWSCredentials() {
@Override
public String getAWSAccessKeyId() {
@ -82,13 +92,56 @@ public class S3StorageDruidModule implements DruidModule
}
};
}
throw new AmazonClientException("Unable to load AWS credentials from druid AWSCredentialsConfig");
}
@Override
public void refresh() {}
};
} else {
return new FileSessionCredentialsProvider(config.getFileSessionCredentials());
}
private class LazyFileSessionCredentialsProvider implements AWSCredentialsProvider
{
private AWSCredentialsConfig config;
private FileSessionCredentialsProvider provider;
public LazyFileSessionCredentialsProvider(AWSCredentialsConfig config) {
this.config = config;
}
private FileSessionCredentialsProvider getUnderlyingProvider() {
if (provider == null) {
synchronized (config) {
if (provider == null) {
provider = new FileSessionCredentialsProvider(config.getFileSessionCredentials());
}
}
}
return provider;
}
@Override
public com.amazonaws.auth.AWSCredentials getCredentials()
{
return getUnderlyingProvider().getCredentials();
}
@Override
public void refresh() {
getUnderlyingProvider().refresh();
}
}
@Provides
@LazySingleton
public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config)
{
return new AWSCredentialsProviderChain(
new ConfigDrivenAwsCredentialsConfigProvider(config),
new LazyFileSessionCredentialsProvider(config),
new EnvironmentVariableCredentialsProvider(),
new SystemPropertiesCredentialsProvider(),
new ProfileCredentialsProvider(),
new InstanceProfileCredentialsProvider());
}
@Provides