Skip to content

Implement federation to HadoopCatalog #1466

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

Merged
merged 18 commits into from
May 9, 2025
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.List;
import java.util.Optional;
import org.apache.polaris.core.admin.model.StorageConfigInfo;
import org.apache.polaris.core.connection.ConnectionType;
import org.apache.polaris.core.context.CallContext;
import org.apache.polaris.core.persistence.cache.EntityWeigher;

Expand Down Expand Up @@ -241,4 +242,11 @@ public static void enforceFeatureEnabledOrThrow(
.description("If true, the policy-store endpoints are enabled")
.defaultValue(true)
.buildFeatureConfiguration();

public static final FeatureConfiguration<List<String>> SUPPORTED_CATALOG_CONNECTION_TYPES =
PolarisConfiguration.<List<String>>builder()
.key("SUPPORTED_CATALOG_CONNECTION_TYPES")
.description("The list of supported catalog connection types for federation")
.defaultValue(List.of(ConnectionType.ICEBERG_REST.name()))
.buildFeatureConfiguration();
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.polaris.core.admin.model.AuthenticationParameters;
import org.apache.polaris.core.admin.model.BearerAuthenticationParameters;
import org.apache.polaris.core.admin.model.OAuthClientCredentialsParameters;
import org.apache.polaris.core.connection.iceberg.IcebergCatalogPropertiesProvider;
import org.apache.polaris.core.secrets.UserSecretReference;

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,11 @@
import java.util.Map;
import org.apache.polaris.core.PolarisDiagnostics;
import org.apache.polaris.core.admin.model.ConnectionConfigInfo;
import org.apache.polaris.core.admin.model.HadoopConnectionConfigInfo;
import org.apache.polaris.core.admin.model.IcebergRestConnectionConfigInfo;
import org.apache.polaris.core.connection.hadoop.HadoopConnectionConfigInfoDpo;
import org.apache.polaris.core.connection.iceberg.IcebergCatalogPropertiesProvider;
import org.apache.polaris.core.connection.iceberg.IcebergRestConnectionConfigInfoDpo;
import org.apache.polaris.core.secrets.UserSecretReference;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -48,6 +52,7 @@
property = "connectionTypeCode")
@JsonSubTypes({
@JsonSubTypes.Type(value = IcebergRestConnectionConfigInfoDpo.class, name = "1"),
@JsonSubTypes.Type(value = HadoopConnectionConfigInfoDpo.class, name = "2"),
})
public abstract class ConnectionConfigInfoDpo implements IcebergCatalogPropertiesProvider {
private static final Logger logger = LoggerFactory.getLogger(ConnectionConfigInfoDpo.class);
Expand Down Expand Up @@ -140,11 +145,12 @@ public static ConnectionConfigInfoDpo fromConnectionConfigInfoModelWithSecrets(
ConnectionConfigInfo connectionConfigurationModel,
Map<String, UserSecretReference> secretReferences) {
ConnectionConfigInfoDpo config = null;
final AuthenticationParametersDpo authenticationParameters;
switch (connectionConfigurationModel.getConnectionType()) {
case ICEBERG_REST:
IcebergRestConnectionConfigInfo icebergRestConfigModel =
(IcebergRestConnectionConfigInfo) connectionConfigurationModel;
AuthenticationParametersDpo authenticationParameters =
authenticationParameters =
AuthenticationParametersDpo.fromAuthenticationParametersModelWithSecrets(
icebergRestConfigModel.getAuthenticationParameters(), secretReferences);
config =
Expand All @@ -153,6 +159,18 @@ public static ConnectionConfigInfoDpo fromConnectionConfigInfoModelWithSecrets(
authenticationParameters,
icebergRestConfigModel.getRemoteCatalogName());
break;
case HADOOP:
HadoopConnectionConfigInfo hadoopConfigModel =
(HadoopConnectionConfigInfo) connectionConfigurationModel;
authenticationParameters =
AuthenticationParametersDpo.fromAuthenticationParametersModelWithSecrets(
hadoopConfigModel.getAuthenticationParameters(), secretReferences);
config =
new HadoopConnectionConfigInfoDpo(
hadoopConfigModel.getUri(),
authenticationParameters,
hadoopConfigModel.getWarehouse());
break;
default:
throw new IllegalStateException(
"Unsupported connection type: " + connectionConfigurationModel.getConnectionType());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
public enum ConnectionType {
NULL_TYPE(0),
ICEBERG_REST(1),
HADOOP(2),
Copy link
Member

Choose a reason for hiding this comment

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

Do we plan to federate to Hadoop through a non-Iceberg Catalog?
Just curious how generic tables would work with Catalog Federation in that case.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is for Hadoop's IcebergCatalog, but good point. For UC, we might have a separate connection for UC's IRC endpoints compared to the non-IRC ones and they might be used for different table types

;

private static final ConnectionType[] REVERSE_MAPPING_ARRAY;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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.connection.hadoop;

import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.MoreObjects;
import jakarta.annotation.Nonnull;
import jakarta.annotation.Nullable;
import java.util.HashMap;
import java.util.Map;
import org.apache.iceberg.CatalogProperties;
import org.apache.polaris.core.admin.model.ConnectionConfigInfo;
import org.apache.polaris.core.admin.model.HadoopConnectionConfigInfo;
import org.apache.polaris.core.connection.AuthenticationParametersDpo;
import org.apache.polaris.core.connection.ConnectionConfigInfoDpo;
import org.apache.polaris.core.connection.ConnectionType;
import org.apache.polaris.core.secrets.UserSecretsManager;

/**
* The internal persistence-object counterpart to {@link
* org.apache.polaris.core.admin.model.HadoopConnectionConfigInfo} defined in the API model.
*/
public class HadoopConnectionConfigInfoDpo extends ConnectionConfigInfoDpo {

private final String warehouse;

public HadoopConnectionConfigInfoDpo(
@JsonProperty(value = "uri", required = true) @Nonnull String uri,
@JsonProperty(value = "authenticationParameters", required = true) @Nonnull
Copy link
Member

Choose a reason for hiding this comment

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

QQ: What type of authentication method does HadoopCatalog typically use? I tried looking for some docs but couldn't find much.

Should we add a no-op authentication method?

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 essentially just forward whatever properties into the catalog itself, which I think functions as the auth mechanism

Copy link
Contributor

Choose a reason for hiding this comment

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

If we extend or refactor HadoopCatalog then the auth could be more properly injectable; unfortunately right now it appears to just hard-code using HadoopFileIO as its fileIOImpl and also independently instantiates a org.apache.hadoop.fs.FileSystem, which relies on the org.apache.hadoop.conf.Configuration which appears to be null in this PR since we call the no-arg constructor of HadoopCatalog instead of the 2-arg constructor that takes a Configuration.

From what I can tell this means it'll just fallback to Hadoop's "default" Configuration loading where it would normally load things from e.g. hdfs-site.xml.

If we're going to really support running this federation mode properly, we should at least add a TODO to expand a HadoopCatalog-specific AuthenticationParameters subclass. One way would be to express similar delegation settings as StorageConfigInfo, or alternatively, HadoopCatalog is a special-case where the real storage credential is the catalog credential, so maybe we could just have something that "points" at the Catalog's StorageConfigInfo as the effective AuthenticationParameters.

Copy link
Contributor Author

@eric-maynard eric-maynard Apr 29, 2025

Choose a reason for hiding this comment

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

I actually think supporting HadoopCatalog as a sort of "dumb" federation mode that just uses the properties you give it (and loads from hdfs-site.xml by default) is fine... if we wanted to add another module that brings in a POLARIS_HADOOP mode or something in the future that could be nice, too.

In practice, the way you would probably deploy this is by creating a second Polaris instance dedicated to wrapping the Hadoop catalog, where you can specify env variables and a hdfs-site.xml

Copy link
Contributor Author

@eric-maynard eric-maynard Apr 29, 2025

Choose a reason for hiding this comment

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

it appears to just hard-code using HadoopFileIO

I think the properties we pass into initialize should take effect right? That would be the fallback if io-impl is not specified.

Copy link
Contributor

Choose a reason for hiding this comment

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

Should we restrict HadoopCatalog backed by cloud-providers only (like S3) and not HDFS clusters for phase 1, as opening up HDFS will add more cases like HDFS backed by kreberos, its not just hdfs-site.xml that Configuration contains, thoughts ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think the concept of federation means that Polaris itself is completely agnostic to the configuration of the remote catalog. Otherwise, when we delegate to IRCs, should we give admins some kind of knobs about which IRCs we can and cannot delegate to? Should we enforce certain settings for those IRCs?

In my view, it makes sense to just create a HadoopCatalog and pass along whatever properties the user provides when the user has configured Polaris to delegate to a Hadoop catalog.

Copy link
Contributor

Choose a reason for hiding this comment

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

In my view, it makes sense to just create a HadoopCatalog and pass along whatever properties the user provides when the user has configured Polaris to delegate to a Hadoop catalog.

I agree in principal for this, I was mostly coming from introducing / opening up for some use-case that polaris doesn't support in the moment itself, hence was my consideration, but totally fine if we move ahead.

I think the concept of federation means that Polaris itself is completely agnostic to the configuration of the remote catalog.

I am not sure if I fully agree with this though, my reasoning is following because Polaris can't just be a pass through, the identity that the federated catalog sees is much more powerful so Polaris should IMHO need to some down scoping or never the less make a way to provide the federated catalog the requester's identity, which in Kreberos world is way different and have to go hand in had with downstream catalog so some co-ordination is required. mostly talking of this : #1026 (comment)
May be I am making it more complicated, than its supposed to be at-least this is chain of thoughts that come to my mind, on thinking about this.

Copy link
Contributor Author

@eric-maynard eric-maynard May 4, 2025

Choose a reason for hiding this comment

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

Polaris should IMHO need to some down scoping or never the less make a way to provide the federated catalog the requester's identity

I agree with you that this should be a goal. I would point out however that the same would be true for even the simplest case -- Polaris -> Polaris federation. This problem isn't unique to federation to Hadoop. It is, however, a very hard problem. What if the two systems don't have the same concept of identity, or they don't trust the same IdP?

There are cases where identity federation may unlock new solutions for us, where the two catalogs can federate to the same trusted IdP. But there are also going to be cases where this is not so.

The way Polaris -> IRC federation currently works, and the way that I think federation will have to continue working for the time being, is that Polaris simply communicates with the remote catalog via the connection config. This can be set on the catalog level today, but in the future we will be able to set this on the namespace or the table level, similar to a storage config.

Copy link
Contributor

Choose a reason for hiding this comment

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

Right, identity-passthrough/delegation is important long-term, but has a lot of complicated prerequisites and other dependencies on how we express federated identities and user-isolated caching of metadata or storage credentials, so in the meantime we should just clearly document what the "powerful connectionconfig" model implies.

AuthenticationParametersDpo authenticationParameters,
@JsonProperty(value = "warehouse", required = false) @Nullable String remoteCatalogName) {
super(ConnectionType.HADOOP.getCode(), uri, authenticationParameters);
this.warehouse = remoteCatalogName;
}

public String getWarehouse() {
return warehouse;
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("connectionTypeCode", getConnectionTypeCode())
.add("uri", getUri())
.add("warehouse", getWarehouse())
.add("authenticationParameters", getAuthenticationParameters().toString())
.toString();
}

@Override
public @Nonnull Map<String, String> asIcebergCatalogProperties(
UserSecretsManager secretsManager) {
HashMap<String, String> properties = new HashMap<>();
properties.put(CatalogProperties.URI, getUri());
if (getWarehouse() != null) {
properties.put(CatalogProperties.WAREHOUSE_LOCATION, getWarehouse());
}
properties.putAll(getAuthenticationParameters().asIcebergCatalogProperties(secretsManager));
return properties;
}

@Override
public ConnectionConfigInfo asConnectionConfigInfoModel() {
return HadoopConnectionConfigInfo.builder()
.setConnectionType(ConnectionConfigInfo.ConnectionTypeEnum.HADOOP)
.setUri(getUri())
.setWarehouse(getWarehouse())
.setAuthenticationParameters(
getAuthenticationParameters().asAuthenticationParametersModel())
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.polaris.core.connection;
package org.apache.polaris.core.connection.iceberg;

import jakarta.annotation.Nonnull;
import java.util.Map;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.polaris.core.connection;
package org.apache.polaris.core.connection.iceberg;

import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.MoreObjects;
Expand All @@ -27,6 +27,9 @@
import org.apache.iceberg.CatalogProperties;
import org.apache.polaris.core.admin.model.ConnectionConfigInfo;
import org.apache.polaris.core.admin.model.IcebergRestConnectionConfigInfo;
import org.apache.polaris.core.connection.AuthenticationParametersDpo;
import org.apache.polaris.core.connection.ConnectionConfigInfoDpo;
import org.apache.polaris.core.connection.ConnectionType;
import org.apache.polaris.core.secrets.UserSecretsManager;

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,7 @@ polaris.realm-context.require-header=false
polaris.features.defaults."ENFORCE_PRINCIPAL_CREDENTIAL_ROTATION_REQUIRED_CHECKING"=false
polaris.features.defaults."SUPPORTED_CATALOG_STORAGE_TYPES"=["S3","GCS","AZURE","FILE"]
# polaris.features.defaults."ENABLE_CATALOG_FEDERATION"=true
polaris.features.defaults."SUPPORTED_CATALOG_CONNECTION_TYPES"=["ICEBERG_REST"]

# realm overrides
# polaris.features.realm-overrides."my-realm"."INITIALIZE_DEFAULT_CATALOG_FILEIO_FOR_TEST"=true
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,8 +87,8 @@
import org.apache.polaris.service.config.DefaultConfigurationStore;
import org.apache.polaris.service.config.RealmEntityManagerFactory;
import org.apache.polaris.service.config.ReservedProperties;
import org.apache.polaris.service.context.CallContextCatalogFactory;
import org.apache.polaris.service.context.PolarisCallContextCatalogFactory;
import org.apache.polaris.service.context.catalog.CallContextCatalogFactory;
import org.apache.polaris.service.context.catalog.PolarisCallContextCatalogFactory;
import org.apache.polaris.service.events.PolarisEventListener;
import org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl;
import org.apache.polaris.service.task.TaskExecutor;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,8 +68,8 @@
import org.apache.polaris.service.catalog.iceberg.IcebergCatalogHandler;
import org.apache.polaris.service.catalog.io.DefaultFileIOFactory;
import org.apache.polaris.service.config.RealmEntityManagerFactory;
import org.apache.polaris.service.context.CallContextCatalogFactory;
import org.apache.polaris.service.context.PolarisCallContextCatalogFactory;
import org.apache.polaris.service.context.catalog.CallContextCatalogFactory;
import org.apache.polaris.service.context.catalog.PolarisCallContextCatalogFactory;
import org.apache.polaris.service.http.IfNoneMatch;
import org.apache.polaris.service.quarkus.admin.PolarisAuthzTestBase;
import org.apache.polaris.service.types.NotificationRequest;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import jakarta.ws.rs.core.Response;
import jakarta.ws.rs.core.SecurityContext;
import java.util.List;
import java.util.Locale;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.exceptions.NotAuthorizedException;
Expand All @@ -37,6 +38,7 @@
import org.apache.polaris.core.admin.model.CreateCatalogRoleRequest;
import org.apache.polaris.core.admin.model.CreatePrincipalRequest;
import org.apache.polaris.core.admin.model.CreatePrincipalRoleRequest;
import org.apache.polaris.core.admin.model.ExternalCatalog;
import org.apache.polaris.core.admin.model.GrantCatalogRoleRequest;
import org.apache.polaris.core.admin.model.GrantPrincipalRoleRequest;
import org.apache.polaris.core.admin.model.GrantResource;
Expand Down Expand Up @@ -141,6 +143,7 @@ public Response createCatalog(
PolarisAdminService adminService = newAdminService(realmContext, securityContext);
Catalog catalog = request.getCatalog();
validateStorageConfig(catalog.getStorageConfigInfo());
validateConnectionConfigInfo(catalog);
Catalog newCatalog = new CatalogEntity(adminService.createCatalog(request)).asCatalog();
LOGGER.info("Created new catalog {}", newCatalog);
return Response.status(Response.Status.CREATED).build();
Expand All @@ -163,6 +166,30 @@ private void validateStorageConfig(StorageConfigInfo storageConfigInfo) {
}
}

private void validateConnectionConfigInfo(Catalog catalog) {
if (catalog.getType() == Catalog.TypeEnum.EXTERNAL) {
if (catalog instanceof ExternalCatalog externalCatalog) {
if (externalCatalog.getConnectionConfigInfo() != null) {
String connectionType =
externalCatalog.getConnectionConfigInfo().getConnectionType().name();
List<String> supportedConnectionTypes =
callContext
.getPolarisCallContext()
.getConfigurationStore()
.getConfiguration(
callContext.getPolarisCallContext(),
FeatureConfiguration.SUPPORTED_CATALOG_CONNECTION_TYPES)
.stream()
.map(s -> s.toUpperCase(Locale.ROOT))
.toList();
if (!supportedConnectionTypes.contains(connectionType)) {
throw new IllegalStateException("Unsupported connection type: " + connectionType);
}
}
}
}
}

/** From PolarisCatalogsApiService */
@Override
public Response deleteCatalog(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@
import org.apache.polaris.service.catalog.api.IcebergRestConfigurationApiService;
import org.apache.polaris.service.catalog.common.CatalogAdapter;
import org.apache.polaris.service.config.ReservedProperties;
import org.apache.polaris.service.context.CallContextCatalogFactory;
import org.apache.polaris.service.context.catalog.CallContextCatalogFactory;
import org.apache.polaris.service.http.IcebergHttpUtil;
import org.apache.polaris.service.http.IfNoneMatch;
import org.apache.polaris.service.types.CommitTableRequest;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@
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.CatalogHandlers;
import org.apache.iceberg.rest.HTTPClient;
import org.apache.iceberg.rest.RESTCatalog;
Expand All @@ -79,7 +80,8 @@
import org.apache.polaris.core.config.PolarisConfigurationStore;
import org.apache.polaris.core.connection.ConnectionConfigInfoDpo;
import org.apache.polaris.core.connection.ConnectionType;
import org.apache.polaris.core.connection.IcebergRestConnectionConfigInfoDpo;
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 All @@ -97,7 +99,7 @@
import org.apache.polaris.service.catalog.SupportsNotifications;
import org.apache.polaris.service.catalog.common.CatalogHandler;
import org.apache.polaris.service.config.ReservedProperties;
import org.apache.polaris.service.context.CallContextCatalogFactory;
import org.apache.polaris.service.context.catalog.CallContextCatalogFactory;
import org.apache.polaris.service.http.IcebergHttpUtil;
import org.apache.polaris.service.http.IfNoneMatch;
import org.apache.polaris.service.types.NotificationRequest;
Expand Down Expand Up @@ -212,6 +214,7 @@ protected void initializeCatalog() {
Catalog federatedCatalog;
ConnectionType connectionType =
ConnectionType.fromCode(connectionConfigInfoDpo.getConnectionTypeCode());

switch (connectionType) {
case ICEBERG_REST:
SessionCatalog.SessionContext context = SessionCatalog.SessionContext.createEmpty();
Expand All @@ -226,6 +229,12 @@ protected void initializeCatalog() {
((IcebergRestConnectionConfigInfoDpo) connectionConfigInfoDpo).getRemoteCatalogName(),
connectionConfigInfoDpo.asIcebergCatalogProperties(getUserSecretsManager()));
break;
case HADOOP:
federatedCatalog = new HadoopCatalog();
federatedCatalog.initialize(
((HadoopConnectionConfigInfoDpo) connectionConfigInfoDpo).getWarehouse(),
connectionConfigInfoDpo.asIcebergCatalogProperties(getUserSecretsManager()));
break;
default:
throw new UnsupportedOperationException(
"Connection type not supported: " + connectionType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.polaris.service.context;
package org.apache.polaris.service.context.catalog;

import jakarta.ws.rs.core.SecurityContext;
import org.apache.iceberg.catalog.Catalog;
Expand All @@ -25,6 +25,9 @@
import org.apache.polaris.core.persistence.resolver.PolarisResolutionManifest;

public interface CallContextCatalogFactory {

static final String WAREHOUSE_LOCATION_BASEDIR = "/tmp/iceberg_rest_server_warehouse_data/";

Catalog createCallContextCatalog(
CallContext context,
AuthenticatedPolarisPrincipal authenticatedPrincipal,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.polaris.service.context;
package org.apache.polaris.service.context.catalog;

import jakarta.enterprise.context.ApplicationScoped;
import jakarta.inject.Inject;
Expand Down Expand Up @@ -48,9 +48,6 @@ public class PolarisCallContextCatalogFactory implements CallContextCatalogFacto
private static final Logger LOGGER =
LoggerFactory.getLogger(PolarisCallContextCatalogFactory.class);

private static final String WAREHOUSE_LOCATION_BASEDIR =
"/tmp/iceberg_rest_server_warehouse_data/";

private final RealmEntityManagerFactory entityManagerFactory;
private final TaskExecutor taskExecutor;
private final FileIOFactory fileIOFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,8 +55,8 @@
import org.apache.polaris.service.config.DefaultConfigurationStore;
import org.apache.polaris.service.config.RealmEntityManagerFactory;
import org.apache.polaris.service.config.ReservedProperties;
import org.apache.polaris.service.context.CallContextCatalogFactory;
import org.apache.polaris.service.context.PolarisCallContextCatalogFactory;
import org.apache.polaris.service.context.catalog.CallContextCatalogFactory;
import org.apache.polaris.service.context.catalog.PolarisCallContextCatalogFactory;
import org.apache.polaris.service.events.PolarisEventListener;
import org.apache.polaris.service.events.TestPolarisEventListener;
import org.apache.polaris.service.persistence.InMemoryPolarisMetaStoreManagerFactory;
Expand Down
Loading