kbendick commented on code in PR #4830: URL: https://github.com/apache/iceberg/pull/4830#discussion_r878601787
########## core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java: ########## @@ -0,0 +1,626 @@ +/* + * 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.rest; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.io.Closeable; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.MetadataUpdate; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.Transactions; +import org.apache.iceberg.catalog.BaseSessionCatalog; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.ResolvingFileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.rest.requests.CreateNamespaceRequest; +import org.apache.iceberg.rest.requests.CreateTableRequest; +import org.apache.iceberg.rest.requests.RenameTableRequest; +import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest; +import org.apache.iceberg.rest.responses.ConfigResponse; +import org.apache.iceberg.rest.responses.CreateNamespaceResponse; +import org.apache.iceberg.rest.responses.GetNamespaceResponse; +import org.apache.iceberg.rest.responses.ListNamespacesResponse; +import org.apache.iceberg.rest.responses.ListTablesResponse; +import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.rest.responses.OAuthTokenResponse; +import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RESTSessionCatalog extends BaseSessionCatalog implements Configurable<Configuration>, Closeable { + private static final Logger LOG = LoggerFactory.getLogger(RESTSessionCatalog.class); + private final Function<Map<String, String>, RESTClient> clientBuilder; + private final Cache<String, AuthSession> sessions = Caffeine.newBuilder().build(); + private AuthSession catalogAuth = null; + private RESTClient client = null; + private ResourcePaths paths = null; + private Object conf = null; + private FileIO io = null; + + public RESTSessionCatalog() { + this(new HTTPClientFactory()); + } + + RESTSessionCatalog(Function<Map<String, String>, RESTClient> clientBuilder) { + this.clientBuilder = clientBuilder; + } + + private void scheduleTokenRefresh(AuthSession session, int expiresInSeconds) { + // TODO: Add token refresh here + } + + @Override + public void initialize(String name, Map<String, String> props) { + Preconditions.checkArgument(props != null, "Invalid configuration: null"); + + String initToken = props.get(RESTCatalogProperties.TOKEN); + + // fetch auth and config to complete initialization + ConfigResponse config; + OAuthTokenResponse authResponse; + try (RESTClient initClient = clientBuilder.apply(props)) { + Map<String, String> initHeaders = merge(configHeaders(props), authHeaders(initToken)); + String credential = props.get(RESTCatalogProperties.CREDENTIAL); + if (credential != null && !credential.isEmpty()) { + String scope = props.getOrDefault(RESTCatalogProperties.SCOPE, RESTCatalogProperties.CATALOG_SCOPE); + authResponse = fetchClientCredentials(initClient, initHeaders, credential, scope); + config = fetchConfig(initClient, merge(initHeaders, 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); + if (authResponse != null) { + this.catalogAuth = new AuthSession( + configHeaders(mergedProps), authResponse.token(), authResponse.issuedTokenType()); + if (authResponse.expiresInSeconds() != null) { + scheduleTokenRefresh(catalogAuth, authResponse.expiresInSeconds()); + } + } else if (initToken != null) { + this.catalogAuth = new AuthSession(configHeaders(mergedProps), initToken, null); + // TODO: add expiration/refresh from a config property + } else { + this.catalogAuth = null; + } + + this.client = clientBuilder.apply(mergedProps); + this.paths = ResourcePaths.forCatalogProperties(mergedProps); + + String ioImpl = mergedProps.get(CatalogProperties.FILE_IO_IMPL); + this.io = CatalogUtil.loadFileIO(ioImpl != null ? ioImpl : ResolvingFileIO.class.getName(), mergedProps, conf); + + super.initialize(name, mergedProps); + } + + Supplier<Map<String, String>> headers(SessionContext context) { + AuthSession session = sessions.get(context.sessionId(), id -> { + Map<String, String> headers = catalogAuth.headers(); + + OAuthTokenResponse response; + if (context.credential() != null) { + response = fetchClientCredentials( + client, headers, context.credential(), RESTCatalogProperties.CATALOG_SCOPE); + } else if (context.properties().containsKey(RESTCatalogProperties.TOKEN)) { + response = exchangeToken( + client, + headers, + context.properties().get(RESTCatalogProperties.TOKEN), + "urn:ietf:params:oauth:token-type:access_token", Review Comment: +1. There's a number of these long strings (which are defined in the OAuth2 spec) that would be easier to read with names. -- 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]
