27
27
import java .util .concurrent .atomic .AtomicInteger ;
28
28
import java .util .stream .Collectors ;
29
29
30
- import com .amazonaws .AmazonClientException ;
31
30
import com .amazonaws .auth .AWSCredentials ;
32
31
import com .amazonaws .auth .AWSCredentialsProvider ;
33
- import com .amazonaws .auth .AnonymousAWSCredentials ;
32
+ import com .amazonaws .auth .BasicAWSCredentials ;
33
+ import com .amazonaws .auth .BasicSessionCredentials ;
34
34
import org .apache .hadoop .classification .VisibleForTesting ;
35
+ import org .apache .hadoop .fs .s3a .adapter .V1V2AwsCredentialProviderAdapter ;
35
36
import org .apache .hadoop .util .Preconditions ;
37
+
36
38
import org .slf4j .Logger ;
37
39
import org .slf4j .LoggerFactory ;
38
40
43
45
import org .apache .hadoop .fs .s3a .auth .NoAwsCredentialsException ;
44
46
import org .apache .hadoop .io .IOUtils ;
45
47
48
+ import software .amazon .awssdk .auth .credentials .AnonymousCredentialsProvider ;
49
+ import software .amazon .awssdk .auth .credentials .AwsCredentials ;
50
+ import software .amazon .awssdk .auth .credentials .AwsCredentialsProvider ;
51
+ import software .amazon .awssdk .auth .credentials .AwsSessionCredentials ;
52
+ import software .amazon .awssdk .core .exception .SdkException ;
53
+
46
54
/**
47
55
* A list of providers.
48
56
*
51
59
* <ol>
52
60
* <li>Allows extra providers to be added dynamically.</li>
53
61
* <li>If any provider in the chain throws an exception other than
54
- * an {@link AmazonClientException }, that is rethrown, rather than
62
+ * an {@link SdkException }, that is rethrown, rather than
55
63
* swallowed.</li>
56
64
* <li>Has some more diagnostics.</li>
57
- * <li>On failure, the last "relevant" AmazonClientException raised is
65
+ * <li>On failure, the last "relevant" {@link SdkException} raised is
58
66
* rethrown; exceptions other than 'no credentials' have priority.</li>
59
- * <li>Special handling of {@link AnonymousAWSCredentials }.</li>
67
+ * <li>Special handling of {@link AnonymousCredentialsProvider }.</li>
60
68
* </ol>
61
69
*/
62
70
@ InterfaceAudience .Private
63
71
@ InterfaceStability .Evolving
64
- public final class AWSCredentialProviderList implements AWSCredentialsProvider ,
72
+ public final class AWSCredentialProviderList implements AwsCredentialsProvider ,
65
73
AutoCloseable {
66
74
67
75
private static final Logger LOG = LoggerFactory .getLogger (
@@ -73,9 +81,9 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
73
81
CREDENTIALS_REQUESTED_WHEN_CLOSED
74
82
= "Credentials requested after provider list was closed" ;
75
83
76
- private final List <AWSCredentialsProvider > providers = new ArrayList <>(1 );
84
+ private final List <AwsCredentialsProvider > providers = new ArrayList <>(1 );
77
85
private boolean reuseLastProvider = true ;
78
- private AWSCredentialsProvider lastProvider ;
86
+ private AwsCredentialsProvider lastProvider ;
79
87
80
88
private final AtomicInteger refCount = new AtomicInteger (1 );
81
89
@@ -99,7 +107,9 @@ public AWSCredentialProviderList() {
99
107
*/
100
108
public AWSCredentialProviderList (
101
109
Collection <AWSCredentialsProvider > providers ) {
102
- this .providers .addAll (providers );
110
+ for (AWSCredentialsProvider provider : providers ) {
111
+ this .providers .add (V1V2AwsCredentialProviderAdapter .adapt (provider ));
112
+ }
103
113
}
104
114
105
115
/**
@@ -110,6 +120,19 @@ public AWSCredentialProviderList(
110
120
public AWSCredentialProviderList (final String name ,
111
121
final AWSCredentialsProvider ... providerArgs ) {
112
122
setName (name );
123
+ for (AWSCredentialsProvider provider : providerArgs ) {
124
+ this .providers .add (V1V2AwsCredentialProviderAdapter .adapt (provider ));
125
+ }
126
+ }
127
+
128
+ /**
129
+ * Create with an initial list of SDK V2 credential providers.
130
+ * @param name name for error messages, may be ""
131
+ * @param providerArgs provider list.
132
+ */
133
+ public AWSCredentialProviderList (final String name ,
134
+ final AwsCredentialsProvider ... providerArgs ) {
135
+ setName (name );
113
136
Collections .addAll (providers , providerArgs );
114
137
}
115
138
@@ -127,12 +150,21 @@ public void setName(final String name) {
127
150
128
151
/**
129
152
* Add a new provider.
130
- * @param p provider
153
+ * @param provider provider
131
154
*/
132
- public void add (AWSCredentialsProvider p ) {
133
- providers .add (p );
155
+ public void add (AWSCredentialsProvider provider ) {
156
+ providers .add (V1V2AwsCredentialProviderAdapter . adapt ( provider ) );
134
157
}
135
158
159
+ /**
160
+ * Add a new SDK V2 provider.
161
+ * @param provider provider
162
+ */
163
+ public void add (AwsCredentialsProvider provider ) {
164
+ providers .add (provider );
165
+ }
166
+
167
+
136
168
/**
137
169
* Add all providers from another list to this one.
138
170
* @param other the other list.
@@ -142,15 +174,18 @@ public void addAll(AWSCredentialProviderList other) {
142
174
}
143
175
144
176
/**
145
- * Refresh all child entries.
177
+ * This method will get credentials using SDK V2's resolveCredentials and then convert it into
178
+ * V1 credentials. This required by delegation token binding classes.
179
+ * @return SDK V1 credentials
146
180
*/
147
- @ Override
148
- public void refresh () {
149
- if (isClosed ()) {
150
- return ;
151
- }
152
- for (AWSCredentialsProvider provider : providers ) {
153
- provider .refresh ();
181
+ public AWSCredentials getCredentials () {
182
+ AwsCredentials credentials = resolveCredentials ();
183
+ if (credentials instanceof AwsSessionCredentials ) {
184
+ return new BasicSessionCredentials (credentials .accessKeyId (),
185
+ credentials .secretAccessKey (),
186
+ ((AwsSessionCredentials ) credentials ).sessionToken ());
187
+ } else {
188
+ return new BasicAWSCredentials (credentials .accessKeyId (), credentials .secretAccessKey ());
154
189
}
155
190
}
156
191
@@ -160,26 +195,26 @@ public void refresh() {
160
195
* @return a set of credentials (possibly anonymous), for authenticating.
161
196
*/
162
197
@ Override
163
- public AWSCredentials getCredentials () {
198
+ public AwsCredentials resolveCredentials () {
164
199
if (isClosed ()) {
165
200
LOG .warn (CREDENTIALS_REQUESTED_WHEN_CLOSED );
166
201
throw new NoAuthWithAWSException (name +
167
202
CREDENTIALS_REQUESTED_WHEN_CLOSED );
168
203
}
169
204
checkNotEmpty ();
170
205
if (reuseLastProvider && lastProvider != null ) {
171
- return lastProvider .getCredentials ();
206
+ return lastProvider .resolveCredentials ();
172
207
}
173
208
174
- AmazonClientException lastException = null ;
175
- for (AWSCredentialsProvider provider : providers ) {
209
+ SdkException lastException = null ;
210
+ for (AwsCredentialsProvider provider : providers ) {
176
211
try {
177
- AWSCredentials credentials = provider .getCredentials ();
212
+ AwsCredentials credentials = provider .resolveCredentials ();
178
213
Preconditions .checkNotNull (credentials ,
179
214
"Null credentials returned by %s" , provider );
180
- if ((credentials .getAWSAccessKeyId () != null &&
181
- credentials . getAWSSecretKey () != null )
182
- || ( credentials instanceof AnonymousAWSCredentials )) {
215
+ if ((credentials .accessKeyId () != null && credentials . secretAccessKey () != null ) || (
216
+ provider instanceof AnonymousCredentialsProvider
217
+ || provider instanceof AnonymousAWSCredentialsProvider )) {
183
218
lastProvider = provider ;
184
219
LOG .debug ("Using credentials from {}" , provider );
185
220
return credentials ;
@@ -196,7 +231,7 @@ public AWSCredentials getCredentials() {
196
231
}
197
232
LOG .debug ("No credentials from {}: {}" ,
198
233
provider , e .toString ());
199
- } catch (AmazonClientException e ) {
234
+ } catch (SdkException e ) {
200
235
lastException = e ;
201
236
LOG .debug ("No credentials provided by {}: {}" ,
202
237
provider , e .toString (), e );
@@ -223,13 +258,13 @@ public AWSCredentials getCredentials() {
223
258
* @return providers
224
259
*/
225
260
@ VisibleForTesting
226
- List <AWSCredentialsProvider > getProviders () {
261
+ List <AwsCredentialsProvider > getProviders () {
227
262
return providers ;
228
263
}
229
264
230
265
/**
231
266
* Verify that the provider list is not empty.
232
- * @throws AmazonClientException if there are no providers.
267
+ * @throws SdkException if there are no providers.
233
268
*/
234
269
public void checkNotEmpty () {
235
270
if (providers .isEmpty ()) {
@@ -317,7 +352,7 @@ public void close() {
317
352
}
318
353
319
354
// do this outside the synchronized block.
320
- for (AWSCredentialsProvider p : providers ) {
355
+ for (AwsCredentialsProvider p : providers ) {
321
356
if (p instanceof Closeable ) {
322
357
IOUtils .closeStream ((Closeable ) p );
323
358
} else if (p instanceof AutoCloseable ) {
0 commit comments