Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
57 changes: 57 additions & 0 deletions extensions/federation/hadoop/build.gradle.kts
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;
}
}
1 change: 1 addition & 0 deletions gradle/projects.main.properties
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ polaris-minio-testcontainer=tools/minio-testcontainer
polaris-version=tools/version
polaris-misc-types=tools/misc-types
polaris-persistence-varint=nosql/persistence/varint
polaris-extensions-federation-hadoop=extensions/federation/hadoop

polaris-config-docs-annotations=tools/config-docs/annotations
polaris-config-docs-generator=tools/config-docs/generator
Expand Down
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
Expand Up @@ -35,6 +35,9 @@ public enum ConnectionType {
HADOOP(2),
;

public static final String ICEBERG_REST_FACTORY_IDENTIFIER = "iceberg_rest";
public static final String HADOOP_FACTORY_IDENTIFIER = "hadoop";

private static final ConnectionType[] REVERSE_MAPPING_ARRAY;

static {
Expand Down Expand Up @@ -77,4 +80,22 @@ public enum ConnectionType {
public int getCode() {
return this.code;
}

/**
* Get the factory identifier string used for CDI injection of the appropriate
* ExternalCatalogFactory.
*
* @return the factory identifier string
*/
public String getFactoryIdentifier() {
switch (this) {
case ICEBERG_REST:
return ICEBERG_REST_FACTORY_IDENTIFIER;
case HADOOP:
return HADOOP_FACTORY_IDENTIFIER;
default:
throw new UnsupportedOperationException(
"No factory identifier for connection type: " + this);
}
}
Comment on lines +90 to +100
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You may want to just return name().toLowerCase(Locale.ROOT) – since most CDI identifiers in Polaris (if not all) are lower-case.

Suggested change
public String getFactoryIdentifier() {
switch (this) {
case ICEBERG_REST:
return ICEBERG_REST_FACTORY_IDENTIFIER;
case HADOOP:
return HADOOP_FACTORY_IDENTIFIER;
default:
throw new UnsupportedOperationException(
"No factory identifier for connection type: " + this);
}
}
public String getFactoryIdentifier() {
return name().toLowerCase(Locale.ROOT);
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can't use the name().toLowerCase() directly because of two reasons:

  1. We need the annotations to be constants. (Please see my comment above)
  2. We need to explicitly handle the NULL_TYPE/default case.

To make things consistent, I've changed the declarations to use lower case.

}
1 change: 1 addition & 0 deletions runtime/server/build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ dependencies {
runtimeOnly("org.postgresql:postgresql")
runtimeOnly(project(":polaris-relational-jdbc"))
runtimeOnly("io.quarkus:quarkus-jdbc-postgresql")
runtimeOnly(project(":polaris-extensions-federation-hadoop"))

// enforce the Quarkus _platform_ here, to get a consistent and validated set of dependencies
implementation(enforcedPlatform(libs.quarkus.bom))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import jakarta.enterprise.context.RequestScoped;
import jakarta.enterprise.inject.Any;
import jakarta.enterprise.inject.Instance;
import jakarta.inject.Inject;
import jakarta.ws.rs.WebApplicationException;
import jakarta.ws.rs.core.HttpHeaders;
Expand Down Expand Up @@ -61,6 +63,7 @@
import org.apache.iceberg.rest.responses.LoadTableResponse;
import org.apache.polaris.core.auth.AuthenticatedPolarisPrincipal;
import org.apache.polaris.core.auth.PolarisAuthorizer;
import org.apache.polaris.core.catalog.ExternalCatalogFactory;
import org.apache.polaris.core.config.RealmConfig;
import org.apache.polaris.core.context.CallContext;
import org.apache.polaris.core.context.RealmContext;
Expand Down Expand Up @@ -146,6 +149,7 @@ public class IcebergCatalogAdapter
private final CatalogPrefixParser prefixParser;
private final ReservedProperties reservedProperties;
private final CatalogHandlerUtils catalogHandlerUtils;
private final Instance<ExternalCatalogFactory> externalCatalogFactories;

@Inject
public IcebergCatalogAdapter(
Expand All @@ -159,7 +163,8 @@ public IcebergCatalogAdapter(
PolarisAuthorizer polarisAuthorizer,
CatalogPrefixParser prefixParser,
ReservedProperties reservedProperties,
CatalogHandlerUtils catalogHandlerUtils) {
CatalogHandlerUtils catalogHandlerUtils,
@Any Instance<ExternalCatalogFactory> externalCatalogFactories) {
this.realmContext = realmContext;
this.callContext = callContext;
this.realmConfig = callContext.getRealmConfig();
Expand All @@ -172,6 +177,7 @@ public IcebergCatalogAdapter(
this.prefixParser = prefixParser;
this.reservedProperties = reservedProperties;
this.catalogHandlerUtils = catalogHandlerUtils;
this.externalCatalogFactories = externalCatalogFactories;
}

/**
Expand Down Expand Up @@ -208,7 +214,8 @@ IcebergCatalogHandler newHandlerWrapper(SecurityContext securityContext, String
catalogName,
polarisAuthorizer,
reservedProperties,
catalogHandlerUtils);
catalogHandlerUtils,
externalCatalogFactories);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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;
Expand All @@ -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;
}

/**
Expand Down Expand Up @@ -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.isResolvable()) {
federatedCatalog =
externalCatalogFactory
.get()
.createCatalog(connectionConfigInfoDpo, getUserSecretsManager());
} else {
throw new UnsupportedOperationException(
"External catalog factory for type '" + connectionType + "' is unavailable.");
}
this.baseCatalog = federatedCatalog;
} else {
Expand Down
Loading