danielcweeks commented on code in PR #4830:
URL: https://github.com/apache/iceberg/pull/4830#discussion_r880049767
##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -87,18 +106,54 @@ public RESTSessionCatalog() {
@Override
public void initialize(String name, Map<String, String> props) {
Preconditions.checkArgument(props != null, "Invalid configuration: null");
- ConfigResponse config = fetchConfig(props);
- Map<String, String> properties = config.merge(props);
- this.client = clientBuilder.apply(properties);
- this.baseHeaders = RESTUtil.extractPrefixMap(properties, "header.");
- this.paths = ResourcePaths.forCatalogProperties(properties);
- String ioImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
- this.io = CatalogUtil.loadFileIO(ioImpl != null ? ioImpl :
ResolvingFileIO.class.getName(), properties, conf);
- super.initialize(name, properties);
+
+ long startTimeMillis = System.currentTimeMillis(); // keep track of the
init start time for token refresh
+ String initToken = props.get(OAuth2Properties.TOKEN);
+
+ // fetch auth and config to complete initialization
+ ConfigResponse config;
+ OAuthTokenResponse authResponse;
+ try (RESTClient initClient = clientBuilder.apply(props)) {
+ Map<String, String> initHeaders = RESTUtil.merge(configHeaders(props),
OAuth2Util.authHeaders(initToken));
+ String credential = props.get(OAuth2Properties.CREDENTIAL);
+ if (credential != null && !credential.isEmpty()) {
+ String scope = props.getOrDefault(OAuth2Properties.SCOPE,
OAuth2Properties.CATALOG_SCOPE);
+ authResponse = OAuth2Util.fetchToken(initClient, initHeaders,
credential, scope);
+ config = fetchConfig(initClient, RESTUtil.merge(initHeaders,
OAuth2Util.authHeaders(authResponse.token())));
+ } else {
+ authResponse = null;
+ config = fetchConfig(initClient, initHeaders);
+ }
+ } catch (IOException e) {
+ throw new UncheckedIOException("Failed to close HTTP client", e);
+ }
+
+ // build the final configuration and set up the catalog's auth
+ Map<String, String> mergedProps = config.merge(props);
+ Map<String, String> baseHeaders = configHeaders(mergedProps);
+ this.catalogAuth = new AuthSession(baseHeaders, null, null);
+ if (authResponse != null) {
+ this.catalogAuth = newSession(authResponse, startTimeMillis,
catalogAuth);
+ } else if (initToken != null) {
+ this.catalogAuth = newSession(initToken, expiresInMs(mergedProps),
catalogAuth);
Review Comment:
This feels a little awkward because if there's a token initially, the init
doesn't go through the token exchange flow, so we don't actually know when the
token is supposed to expire but still scheduling to refresh. It feels like we
should either exchange immediately and use the expires response to schedule the
interval or assume that a set token has no expiration.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]