-
Notifications
You must be signed in to change notification settings - Fork 2.9k
AWS: Refresh vended credentials #11389
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,6 +20,7 @@ | |
|
|
||
| import java.io.Serializable; | ||
| import java.util.Map; | ||
| import org.apache.iceberg.aws.s3.VendedCredentialsProvider; | ||
| import org.apache.iceberg.common.DynClasses; | ||
| import org.apache.iceberg.common.DynMethods; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
|
|
@@ -66,21 +67,37 @@ public class AwsClientProperties implements Serializable { | |
| */ | ||
| public static final String CLIENT_REGION = "client.region"; | ||
|
|
||
| /** | ||
| * When set, the {@link VendedCredentialsProvider} will be used to fetch and refresh vended | ||
| * credentials from this endpoint. | ||
| */ | ||
| public static final String REFRESH_CREDENTIALS_ENDPOINT = "client.refresh-credentials-endpoint"; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. shouldn't this be a s3 FileIO property?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This property needs to be set before |
||
|
|
||
| /** Controls whether vended credentials should be refreshed or not. Defaults to true. */ | ||
| public static final String REFRESH_CREDENTIALS_ENABLED = "client.refresh-credentials-enabled"; | ||
|
|
||
| private String clientRegion; | ||
| private final String clientCredentialsProvider; | ||
| private final Map<String, String> clientCredentialsProviderProperties; | ||
| private final String refreshCredentialsEndpoint; | ||
| private final boolean refreshCredentialsEnabled; | ||
|
|
||
| public AwsClientProperties() { | ||
| this.clientRegion = null; | ||
| this.clientCredentialsProvider = null; | ||
| this.clientCredentialsProviderProperties = null; | ||
| this.refreshCredentialsEndpoint = null; | ||
| this.refreshCredentialsEnabled = true; | ||
| } | ||
|
|
||
| public AwsClientProperties(Map<String, String> properties) { | ||
| this.clientRegion = properties.get(CLIENT_REGION); | ||
| this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER); | ||
| this.clientCredentialsProviderProperties = | ||
| PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX); | ||
| this.refreshCredentialsEndpoint = properties.get(REFRESH_CREDENTIALS_ENDPOINT); | ||
| this.refreshCredentialsEnabled = | ||
| PropertyUtil.propertyAsBoolean(properties, REFRESH_CREDENTIALS_ENABLED, true); | ||
| } | ||
|
|
||
| public String clientRegion() { | ||
|
|
@@ -122,11 +139,12 @@ public <T extends AwsClientBuilder> void applyClientCredentialConfigurations(T b | |
| } | ||
|
|
||
| /** | ||
| * Returns a credentials provider instance. If params were set, we return a new credentials | ||
| * instance. If none of the params are set, we try to dynamically load the provided credentials | ||
| * provider class. Upon loading the class, we try to invoke {@code create(Map<String, String>)} | ||
| * static method. If that fails, we fall back to {@code create()}. If credential provider class | ||
| * wasn't set, we fall back to default credentials provider. | ||
| * Returns a credentials provider instance. If {@link #refreshCredentialsEndpoint} is set, an | ||
| * instance of {@link VendedCredentialsProvider} is returned. If params were set, we return a new | ||
| * credentials instance. If none of the params are set, we try to dynamically load the provided | ||
| * credentials provider class. Upon loading the class, we try to invoke {@code create(Map<String, | ||
| * String>)} static method. If that fails, we fall back to {@code create()}. If credential | ||
| * provider class wasn't set, we fall back to default credentials provider. | ||
| * | ||
| * @param accessKeyId the AWS access key ID | ||
| * @param secretAccessKey the AWS secret access key | ||
|
|
@@ -136,6 +154,12 @@ public <T extends AwsClientBuilder> void applyClientCredentialConfigurations(T b | |
| @SuppressWarnings("checkstyle:HiddenField") | ||
| public AwsCredentialsProvider credentialsProvider( | ||
| String accessKeyId, String secretAccessKey, String sessionToken) { | ||
| if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: Should we log a warning if the endpoint is set but refreshCredentialsEnabled is false? I don't think we should fail but this is probably something a user would want to be aware of.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure adding a warning adds a lot of value. It's valid to have the server send you back an endpoint + refresh enabled flag that you then override for cases like Kafka connect. I'd say let's go without a warning for now unless this becomes a place of confusion |
||
| clientCredentialsProviderProperties.put( | ||
| VendedCredentialsProvider.URI, refreshCredentialsEndpoint); | ||
| return credentialsProvider(VendedCredentialsProvider.class.getName()); | ||
| } | ||
|
|
||
| if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { | ||
| if (Strings.isNullOrEmpty(sessionToken)) { | ||
| return StaticCredentialsProvider.create( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,138 @@ | ||
| /* | ||
| * 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.iceberg.aws.s3; | ||
|
|
||
| import java.time.Instant; | ||
| import java.time.temporal.ChronoUnit; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.stream.Collectors; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.apache.iceberg.rest.ErrorHandlers; | ||
| import org.apache.iceberg.rest.HTTPClient; | ||
| import org.apache.iceberg.rest.RESTClient; | ||
| import org.apache.iceberg.rest.auth.OAuth2Properties; | ||
| import org.apache.iceberg.rest.auth.OAuth2Util; | ||
| import org.apache.iceberg.rest.credentials.Credential; | ||
| import org.apache.iceberg.rest.responses.LoadCredentialsResponse; | ||
| import software.amazon.awssdk.auth.credentials.AwsCredentials; | ||
| import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; | ||
| import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; | ||
| import software.amazon.awssdk.utils.IoUtils; | ||
| import software.amazon.awssdk.utils.SdkAutoCloseable; | ||
| import software.amazon.awssdk.utils.cache.CachedSupplier; | ||
| import software.amazon.awssdk.utils.cache.RefreshResult; | ||
|
|
||
| public class VendedCredentialsProvider implements AwsCredentialsProvider, SdkAutoCloseable { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @jackye1995 is this credential provider similar to the one you mentioned a while ago where you guys had a custom credential provider that would always call loadTable()?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes this looks similar. |
||
| public static final String URI = "credentials.uri"; | ||
| private volatile HTTPClient client; | ||
| private final Map<String, String> properties; | ||
| private final CachedSupplier<AwsCredentials> credentialCache; | ||
|
|
||
| private VendedCredentialsProvider(Map<String, String> properties) { | ||
| Preconditions.checkArgument(null != properties, "Invalid properties: null"); | ||
| Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); | ||
nastra marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| this.properties = properties; | ||
| this.credentialCache = | ||
| CachedSupplier.builder(this::refreshCredential) | ||
| .cachedValueName(VendedCredentialsProvider.class.getName()) | ||
| .build(); | ||
| } | ||
|
|
||
| @Override | ||
| public AwsCredentials resolveCredentials() { | ||
| return credentialCache.get(); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| IoUtils.closeQuietly(client, null); | ||
| credentialCache.close(); | ||
| } | ||
|
|
||
| public static VendedCredentialsProvider create(Map<String, String> properties) { | ||
| return new VendedCredentialsProvider(properties); | ||
| } | ||
|
|
||
| private RESTClient httpClient() { | ||
| if (null == client) { | ||
| synchronized (this) { | ||
| if (null == client) { | ||
| client = HTTPClient.builder(properties).uri(properties.get(URI)).build(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| return client; | ||
| } | ||
|
|
||
| private LoadCredentialsResponse fetchCredentials() { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think there are 2 concepts that would be beneficial to be added, a
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is already being done further below: |
||
| return httpClient() | ||
| .get( | ||
| properties.get(URI), | ||
| null, | ||
nastra marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| LoadCredentialsResponse.class, | ||
| OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)), | ||
|
Comment on lines
+90
to
+91
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Assuming the Token here is same short lived token used by RestCatalog instance, are we planning to handle token refresh post its expiration with in the FileIO?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is indeed an issue that I was planning to address when I wrote the first version of this many months ago but simply forgot to get back to. thanks for raising this @ChaladiMohanVamsi
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think for the initial support for refresh, we can assume that we're bound by the token lifetime for refreshes. It's not perfect, but I'd like to see the resolution of some of the AuthManger refactor before settling on a solution. We're not regressing at this point, so I'm ok with leaving this as is and addressing in the future.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. FYI, this is on my todo list for after the AuthManager refactor. |
||
| ErrorHandlers.defaultErrorHandler()); | ||
| } | ||
|
|
||
| private RefreshResult<AwsCredentials> refreshCredential() { | ||
| LoadCredentialsResponse response = fetchCredentials(); | ||
|
|
||
| List<Credential> s3Credentials = | ||
| response.credentials().stream() | ||
| .filter(c -> c.prefix().startsWith("s3")) | ||
| .collect(Collectors.toList()); | ||
|
Comment on lines
+98
to
+101
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This imho still doesn't addresses the problem of wiring the credential for the right prefix ? I would recommend rather than using starting with "S3" let make it equal to "S3" for now to make sure the client doesn't mess around, what do you think ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @singhpk234 for how the implementation enforces that there's really only a single credential being sent back by the server. I'll be working on supporting and selecting the "right" credential when the server sents back multiple in a follow-up
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
[doubt] how are we enforcing this ? for ex a rest server can send a credential for only one prefix but still for a diff prefix than what being asked for, are you suggesting that for now, rest server should only send back for exactly "s3" prefix ? if yes how are we enforcing this in rest for the meanwhile ?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The catalog has the responsibility of returning a credential with a scoped policy that provides the appropriate access for all prefixes. This isn't about the spec or client trying to enforce that behavior. Either the client will have access or not, that's up to the catalog.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I see, so that fact that only one credential is being returned from the catalog, it itself means that its best prefix that could fit in into all the request the client is allowed make. Make sense ! so we would indeed fail at client trying for an un-acessible prefix from S3 end. |
||
|
|
||
| Preconditions.checkState(!s3Credentials.isEmpty(), "Invalid S3 Credentials: empty"); | ||
| Preconditions.checkState( | ||
| s3Credentials.size() == 1, "Invalid S3 Credentials: only one S3 credential should exist"); | ||
|
|
||
| Credential s3Credential = s3Credentials.get(0); | ||
| checkCredential(s3Credential, S3FileIOProperties.ACCESS_KEY_ID); | ||
| checkCredential(s3Credential, S3FileIOProperties.SECRET_ACCESS_KEY); | ||
| checkCredential(s3Credential, S3FileIOProperties.SESSION_TOKEN); | ||
| checkCredential(s3Credential, S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS); | ||
|
|
||
| String accessKeyId = s3Credential.config().get(S3FileIOProperties.ACCESS_KEY_ID); | ||
| String secretAccessKey = s3Credential.config().get(S3FileIOProperties.SECRET_ACCESS_KEY); | ||
| String sessionToken = s3Credential.config().get(S3FileIOProperties.SESSION_TOKEN); | ||
| String tokenExpiresAtMillis = | ||
| s3Credential.config().get(S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS); | ||
| Instant expiresAt = Instant.ofEpochMilli(Long.parseLong(tokenExpiresAtMillis)); | ||
| Instant prefetchAt = expiresAt.minus(5, ChronoUnit.MINUTES); | ||
|
|
||
| return RefreshResult.builder( | ||
| (AwsCredentials) | ||
| AwsSessionCredentials.builder() | ||
| .accessKeyId(accessKeyId) | ||
| .secretAccessKey(secretAccessKey) | ||
| .sessionToken(sessionToken) | ||
| .expirationTime(expiresAt) | ||
| .build()) | ||
| .staleTime(expiresAt) | ||
| .prefetchTime(prefetchAt) | ||
| .build(); | ||
| } | ||
|
|
||
| private void checkCredential(Credential credential, String property) { | ||
| Preconditions.checkState( | ||
| credential.config().containsKey(property), "Invalid S3 Credentials: %s not set", property); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.