-
Notifications
You must be signed in to change notification settings - Fork 332
Modularize calls to federated catalogs #2301
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 12 commits
afe30fa
63f25b1
5cafd63
d9cdb71
91e1807
e036e9f
2c4aa58
c73966c
284ad40
790f81b
05dbe0c
6e78c72
7c7e257
4fb7967
1cbafb3
1902d21
103fe6f
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 |
|---|---|---|
| @@ -0,0 +1,57 @@ | ||
| /* | ||
| * 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. | ||
| */ | ||
|
|
||
| plugins { | ||
| id("polaris-client") | ||
| alias(libs.plugins.jandex) | ||
| } | ||
|
|
||
| dependencies { | ||
| // Polaris dependencies | ||
| implementation(project(":polaris-core")) | ||
|
|
||
| implementation(platform(libs.iceberg.bom)) | ||
| implementation("org.apache.iceberg:iceberg-api") | ||
| implementation("org.apache.iceberg:iceberg-core") | ||
| implementation("org.apache.iceberg:iceberg-common") | ||
|
|
||
| // Hadoop dependencies (for Hadoop catalog support) | ||
| implementation(libs.hadoop.common) { | ||
| exclude("org.slf4j", "slf4j-reload4j") | ||
| exclude("org.slf4j", "slf4j-log4j12") | ||
| exclude("ch.qos.reload4j", "reload4j") | ||
| exclude("log4j", "log4j") | ||
| exclude("org.apache.zookeeper", "zookeeper") | ||
| exclude("org.apache.hadoop.thirdparty", "hadoop-shaded-protobuf_3_25") | ||
| exclude("com.github.pjfanning", "jersey-json") | ||
| exclude("com.sun.jersey", "jersey-core") | ||
| exclude("com.sun.jersey", "jersey-server") | ||
| exclude("com.sun.jersey", "jersey-servlet") | ||
| exclude("io.dropwizard.metrics", "metrics-core") | ||
| } | ||
| implementation(libs.hadoop.client.api) | ||
| implementation(libs.hadoop.client.runtime) | ||
|
|
||
| // CDI dependencies for runtime discovery | ||
| implementation(libs.jakarta.enterprise.cdi.api) | ||
| implementation(libs.smallrye.common.annotation) | ||
|
|
||
| // Logging | ||
| implementation(libs.slf4j.api) | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,61 @@ | ||
| /* | ||
| * 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.polaris.extensions.federation.hadoop; | ||
|
|
||
| import io.smallrye.common.annotation.Identifier; | ||
| import jakarta.enterprise.context.ApplicationScoped; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.iceberg.catalog.Catalog; | ||
| import org.apache.iceberg.hadoop.HadoopCatalog; | ||
| import org.apache.polaris.core.catalog.ExternalCatalogFactory; | ||
| import org.apache.polaris.core.connection.AuthenticationParametersDpo; | ||
| import org.apache.polaris.core.connection.AuthenticationType; | ||
| import org.apache.polaris.core.connection.ConnectionConfigInfoDpo; | ||
| import org.apache.polaris.core.connection.ConnectionType; | ||
| import org.apache.polaris.core.connection.hadoop.HadoopConnectionConfigInfoDpo; | ||
| import org.apache.polaris.core.secrets.UserSecretsManager; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| /** Factory class for creating a Hadoop catalog handle based on connection configuration. */ | ||
| @ApplicationScoped | ||
| @Identifier(ConnectionType.HADOOP_FACTORY_IDENTIFIER) | ||
| public class HadoopFederatedCatalogFactory implements ExternalCatalogFactory { | ||
| private static final Logger LOGGER = LoggerFactory.getLogger(HadoopFederatedCatalogFactory.class); | ||
|
|
||
| @Override | ||
| public Catalog createCatalog( | ||
| ConnectionConfigInfoDpo connectionConfigInfoDpo, UserSecretsManager userSecretsManager) { | ||
| // Currently, Polaris supports Hadoop federation only via IMPLICIT authentication. | ||
| // Hence, prior to initializing the configuration, ensure that the catalog uses | ||
| // IMPLICIT authentication. | ||
| AuthenticationParametersDpo authenticationParametersDpo = | ||
| connectionConfigInfoDpo.getAuthenticationParameters(); | ||
| if (authenticationParametersDpo.getAuthenticationTypeCode() | ||
| != AuthenticationType.IMPLICIT.getCode()) { | ||
| throw new IllegalStateException("Hadoop federation only supports IMPLICIT authentication."); | ||
| } | ||
| Configuration conf = new Configuration(); | ||
| String warehouse = ((HadoopConnectionConfigInfoDpo) connectionConfigInfoDpo).getWarehouse(); | ||
| HadoopCatalog hadoopCatalog = new HadoopCatalog(conf, warehouse); | ||
| hadoopCatalog.initialize( | ||
| warehouse, connectionConfigInfoDpo.asIcebergCatalogProperties(userSecretsManager)); | ||
| return hadoopCatalog; | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,43 @@ | ||
| /* | ||
| * 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.polaris.core.catalog; | ||
|
|
||
| import org.apache.iceberg.catalog.Catalog; | ||
| import org.apache.polaris.core.connection.ConnectionConfigInfoDpo; | ||
| import org.apache.polaris.core.secrets.UserSecretsManager; | ||
|
|
||
| /** | ||
| * Factory interface for creating external catalog handles based on connection configuration. | ||
| * | ||
| * <p>Implementations should be annotated with CDI annotations and use the @Identifier annotation to | ||
| * specify which connection type they support. | ||
| */ | ||
| public interface ExternalCatalogFactory { | ||
|
|
||
| /** | ||
| * Creates a catalog handle for the given connection configuration. | ||
| * | ||
| * @param connectionConfig the connection configuration | ||
| * @param userSecretsManager the user secrets manager for handling credentials | ||
| * @return the initialized catalog | ||
| * @throws IllegalStateException if the connection configuration is invalid | ||
| */ | ||
| Catalog createCatalog( | ||
| ConnectionConfigInfoDpo connectionConfig, UserSecretsManager userSecretsManager); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,7 +20,9 @@ | |
|
|
||
| import com.google.common.base.Preconditions; | ||
| import com.google.common.collect.Maps; | ||
| import io.smallrye.common.annotation.Identifier; | ||
| import jakarta.annotation.Nonnull; | ||
| import jakarta.enterprise.inject.Instance; | ||
| import jakarta.ws.rs.core.SecurityContext; | ||
| import java.io.Closeable; | ||
| import java.time.OffsetDateTime; | ||
|
|
@@ -33,7 +35,6 @@ | |
| import java.util.Optional; | ||
| import java.util.Set; | ||
| import java.util.stream.Collectors; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.iceberg.BaseMetadataTable; | ||
| import org.apache.iceberg.BaseTable; | ||
| import org.apache.iceberg.MetadataUpdate; | ||
|
|
@@ -46,7 +47,6 @@ | |
| import org.apache.iceberg.UpdateRequirement; | ||
| import org.apache.iceberg.catalog.Catalog; | ||
| import org.apache.iceberg.catalog.Namespace; | ||
| import org.apache.iceberg.catalog.SessionCatalog; | ||
| import org.apache.iceberg.catalog.SupportsNamespaces; | ||
| import org.apache.iceberg.catalog.TableIdentifier; | ||
| import org.apache.iceberg.catalog.ViewCatalog; | ||
|
|
@@ -55,9 +55,6 @@ | |
| import org.apache.iceberg.exceptions.CommitFailedException; | ||
| import org.apache.iceberg.exceptions.ForbiddenException; | ||
| import org.apache.iceberg.exceptions.NoSuchTableException; | ||
| import org.apache.iceberg.hadoop.HadoopCatalog; | ||
| import org.apache.iceberg.rest.HTTPClient; | ||
| import org.apache.iceberg.rest.RESTCatalog; | ||
| import org.apache.iceberg.rest.credentials.ImmutableCredential; | ||
| import org.apache.iceberg.rest.requests.CommitTransactionRequest; | ||
| import org.apache.iceberg.rest.requests.CreateNamespaceRequest; | ||
|
|
@@ -76,13 +73,10 @@ | |
| import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; | ||
| import org.apache.polaris.core.auth.PolarisAuthorizableOperation; | ||
| import org.apache.polaris.core.auth.PolarisAuthorizer; | ||
| import org.apache.polaris.core.catalog.ExternalCatalogFactory; | ||
| import org.apache.polaris.core.config.FeatureConfiguration; | ||
| import org.apache.polaris.core.connection.AuthenticationParametersDpo; | ||
| import org.apache.polaris.core.connection.AuthenticationType; | ||
| import org.apache.polaris.core.connection.ConnectionConfigInfoDpo; | ||
| import org.apache.polaris.core.connection.ConnectionType; | ||
| import org.apache.polaris.core.connection.hadoop.HadoopConnectionConfigInfoDpo; | ||
| import org.apache.polaris.core.connection.iceberg.IcebergRestConnectionConfigInfoDpo; | ||
| import org.apache.polaris.core.context.CallContext; | ||
| import org.apache.polaris.core.entity.CatalogEntity; | ||
| import org.apache.polaris.core.entity.PolarisEntitySubType; | ||
|
|
@@ -132,6 +126,8 @@ public class IcebergCatalogHandler extends CatalogHandler implements AutoCloseab | |
| private final ReservedProperties reservedProperties; | ||
| private final CatalogHandlerUtils catalogHandlerUtils; | ||
|
|
||
| private final Instance<ExternalCatalogFactory> externalCatalogFactories; | ||
|
|
||
| // Catalog instance will be initialized after authorizing resolver successfully resolves | ||
| // the catalog entity. | ||
| protected Catalog baseCatalog = null; | ||
|
|
@@ -151,13 +147,15 @@ public IcebergCatalogHandler( | |
| String catalogName, | ||
| PolarisAuthorizer authorizer, | ||
| ReservedProperties reservedProperties, | ||
| CatalogHandlerUtils catalogHandlerUtils) { | ||
| CatalogHandlerUtils catalogHandlerUtils, | ||
| Instance<ExternalCatalogFactory> externalCatalogFactories) { | ||
| super(callContext, resolutionManifestFactory, securityContext, catalogName, authorizer); | ||
| this.metaStoreManager = metaStoreManager; | ||
| this.userSecretsManager = userSecretsManager; | ||
| this.catalogFactory = catalogFactory; | ||
| this.reservedProperties = reservedProperties; | ||
| this.catalogHandlerUtils = catalogHandlerUtils; | ||
| this.externalCatalogFactories = externalCatalogFactories; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -220,42 +218,18 @@ protected void initializeCatalog() { | |
| ConnectionType connectionType = | ||
| ConnectionType.fromCode(connectionConfigInfoDpo.getConnectionTypeCode()); | ||
|
|
||
| switch (connectionType) { | ||
| case ICEBERG_REST: | ||
| SessionCatalog.SessionContext context = SessionCatalog.SessionContext.createEmpty(); | ||
| federatedCatalog = | ||
| new RESTCatalog( | ||
| context, | ||
| (config) -> | ||
| HTTPClient.builder(config) | ||
| .uri(config.get(org.apache.iceberg.CatalogProperties.URI)) | ||
| .build()); | ||
| federatedCatalog.initialize( | ||
| ((IcebergRestConnectionConfigInfoDpo) connectionConfigInfoDpo).getRemoteCatalogName(), | ||
| connectionConfigInfoDpo.asIcebergCatalogProperties(getUserSecretsManager())); | ||
| break; | ||
| case HADOOP: | ||
| // Currently, Polaris supports Hadoop federation only via IMPLICIT authentication. | ||
| // Hence, prior to initializing the configuration, ensure that the catalog uses | ||
| // IMPLICIT authentication. | ||
| AuthenticationParametersDpo authenticationParametersDpo = | ||
| connectionConfigInfoDpo.getAuthenticationParameters(); | ||
| if (authenticationParametersDpo.getAuthenticationTypeCode() | ||
| != AuthenticationType.IMPLICIT.getCode()) { | ||
| throw new IllegalStateException( | ||
| "Hadoop federation only supports IMPLICIT authentication."); | ||
| } | ||
| Configuration conf = new Configuration(); | ||
| String warehouse = | ||
| ((HadoopConnectionConfigInfoDpo) connectionConfigInfoDpo).getWarehouse(); | ||
| federatedCatalog = new HadoopCatalog(conf, warehouse); | ||
| federatedCatalog.initialize( | ||
| warehouse, | ||
| connectionConfigInfoDpo.asIcebergCatalogProperties(getUserSecretsManager())); | ||
| break; | ||
| default: | ||
| throw new UnsupportedOperationException( | ||
| "Connection type not supported: " + connectionType); | ||
| // Use the unified factory pattern for all external catalog types | ||
| Instance<ExternalCatalogFactory> externalCatalogFactory = | ||
| externalCatalogFactories.select( | ||
| Identifier.Literal.of(connectionType.getFactoryIdentifier())); | ||
| if (!externalCatalogFactory.isUnsatisfied()) { | ||
|
||
| federatedCatalog = | ||
| externalCatalogFactory | ||
| .get() | ||
| .createCatalog(connectionConfigInfoDpo, getUserSecretsManager()); | ||
| } else { | ||
| throw new UnsupportedOperationException( | ||
| "External catalog factory for type '" + connectionType + "' is unavailable."); | ||
| } | ||
| this.baseCatalog = federatedCatalog; | ||
| } else { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a breaking change to Polaris that effectively removes functionality in the Polaris server.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
IIUC the functionality is still there, but the way you need to build Polaris to use that functionality does change (across versions).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In any case, shouldn't this block be declared rather in
runtime/server?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Regarding runtime/server module: We reference the ExternalCatalogFactory in the IcebergCatalogHandler which is in runtime/service module. So isn't this the correct location?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But afaict
ExternalCatalogFactoryis an interface inpolaris-core. We don't referenceHadoopFederatedCatalogFactorydirectly in code.In
runtime/serverwe already have other similarruntimeOnlydeclarations:There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I had that until c73966c
I can do either, from the Polaris OSS sync last week (discussion about regarding hive federation), my takeaway was that we wanted to avoid having the default Polaris JAR depend on anything hadoop.
However, if you'd prefer compiling it each time (and only loading if necessary), I can revert that change. I will send out a separate PR without dynamic compilation and update a README.md for this PR. Please pick the option that's best suited according to you.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 to adding something to the README/getting-started or similar to better make it an easy copy/paste command to decide whether to compile with or without the extended dependencies.
My understanding is indeed that this check was directly to address the concern others had about having Hadoop (or Hive in the future) compile-time dependencies be always present for all Polaris builds.
Personally I don't feel too strongly either way, so I'm okay with or without the additional compilation property.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also, per offline discussion, +1 to what @adutra said about putting the Hadoop extension into
runtime/serverif it works correctly for Quarkus finding it in the runtime assembly.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.