This is an automated email from the ASF dual-hosted git repository.

snazy pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/polaris.git


The following commit(s) were added to refs/heads/main by this push:
     new c38d0b067 Runtime/service: move getConfig() down to 
`IcebergCatalogHandler` (#3231)
c38d0b067 is described below

commit c38d0b067daeab5851163e36b47efce1ba8a0bf0
Author: Robert Stupp <[email protected]>
AuthorDate: Wed Dec 10 07:14:40 2025 +0100

    Runtime/service: move getConfig() down to `IcebergCatalogHandler` (#3231)
    
    All catalog specific functionality is implemented in 
`IcebergCatalogHandler`, whereas `IcebergCatalogAdapter` is meant to act as a 
"REST wrapper" to it.
    
    This change moves the implementation of `getConfig` down to the handler, no 
functional changes.
---
 .../catalog/iceberg/IcebergCatalogAdapter.java     | 97 +++-------------------
 .../catalog/iceberg/IcebergCatalogHandler.java     | 82 ++++++++++++++++++
 .../iceberg/IcebergCatalogHandlerAuthzTest.java    | 10 +++
 ...ebergCatalogHandlerFineGrainedDisabledTest.java |  7 +-
 4 files changed, 109 insertions(+), 87 deletions(-)

diff --git 
a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java
 
b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java
index 1101ccbee..6c30afb9e 100644
--- 
a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java
+++ 
b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java
@@ -23,9 +23,6 @@ import static 
org.apache.polaris.service.catalog.validation.IcebergPropertiesVal
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-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;
@@ -34,17 +31,12 @@ import jakarta.ws.rs.core.HttpHeaders;
 import jakarta.ws.rs.core.Response;
 import jakarta.ws.rs.core.SecurityContext;
 import java.util.EnumSet;
-import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
 import java.util.function.Function;
 import org.apache.iceberg.MetadataUpdate;
 import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.iceberg.exceptions.BadRequestException;
-import org.apache.iceberg.exceptions.NotAuthorizedException;
-import org.apache.iceberg.exceptions.NotFoundException;
-import org.apache.iceberg.rest.Endpoint;
 import org.apache.iceberg.rest.RESTUtil;
 import org.apache.iceberg.rest.requests.CommitTransactionRequest;
 import org.apache.iceberg.rest.requests.CreateNamespaceRequest;
@@ -56,7 +48,6 @@ import org.apache.iceberg.rest.requests.RenameTableRequest;
 import org.apache.iceberg.rest.requests.ReportMetricsRequest;
 import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest;
 import org.apache.iceberg.rest.requests.UpdateTableRequest;
-import org.apache.iceberg.rest.responses.ConfigResponse;
 import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse;
 import org.apache.iceberg.rest.responses.LoadTableResponse;
 import org.apache.polaris.core.PolarisDiagnostics;
@@ -67,14 +58,9 @@ import org.apache.polaris.core.config.RealmConfig;
 import org.apache.polaris.core.context.CallContext;
 import org.apache.polaris.core.context.RealmContext;
 import org.apache.polaris.core.credentials.PolarisCredentialManager;
-import org.apache.polaris.core.entity.PolarisEntity;
 import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
-import org.apache.polaris.core.persistence.ResolvedPolarisEntity;
 import org.apache.polaris.core.persistence.resolver.ResolutionManifestFactory;
-import org.apache.polaris.core.persistence.resolver.Resolver;
 import org.apache.polaris.core.persistence.resolver.ResolverFactory;
-import org.apache.polaris.core.persistence.resolver.ResolverStatus;
-import org.apache.polaris.core.rest.PolarisEndpoints;
 import org.apache.polaris.core.rest.PolarisResourcePaths;
 import org.apache.polaris.service.catalog.AccessDelegationMode;
 import org.apache.polaris.service.catalog.CatalogPrefixParser;
@@ -103,37 +89,6 @@ public class IcebergCatalogAdapter
 
   private static final Logger LOGGER = 
LoggerFactory.getLogger(IcebergCatalogAdapter.class);
 
-  private static final Set<Endpoint> DEFAULT_ENDPOINTS =
-      ImmutableSet.<Endpoint>builder()
-          .add(Endpoint.V1_LIST_NAMESPACES)
-          .add(Endpoint.V1_LOAD_NAMESPACE)
-          .add(Endpoint.V1_NAMESPACE_EXISTS)
-          .add(Endpoint.V1_CREATE_NAMESPACE)
-          .add(Endpoint.V1_UPDATE_NAMESPACE)
-          .add(Endpoint.V1_DELETE_NAMESPACE)
-          .add(Endpoint.V1_LIST_TABLES)
-          .add(Endpoint.V1_LOAD_TABLE)
-          .add(Endpoint.V1_TABLE_EXISTS)
-          .add(Endpoint.V1_CREATE_TABLE)
-          .add(Endpoint.V1_UPDATE_TABLE)
-          .add(Endpoint.V1_DELETE_TABLE)
-          .add(Endpoint.V1_RENAME_TABLE)
-          .add(Endpoint.V1_REGISTER_TABLE)
-          .add(Endpoint.V1_REPORT_METRICS)
-          .add(Endpoint.V1_COMMIT_TRANSACTION)
-          .build();
-
-  private static final Set<Endpoint> VIEW_ENDPOINTS =
-      ImmutableSet.<Endpoint>builder()
-          .add(Endpoint.V1_LIST_VIEWS)
-          .add(Endpoint.V1_LOAD_VIEW)
-          .add(Endpoint.V1_VIEW_EXISTS)
-          .add(Endpoint.V1_CREATE_VIEW)
-          .add(Endpoint.V1_UPDATE_VIEW)
-          .add(Endpoint.V1_DELETE_VIEW)
-          .add(Endpoint.V1_RENAME_VIEW)
-          .build();
-
   private final PolarisDiagnostics diagnostics;
   private final RealmContext realmContext;
   private final CallContext callContext;
@@ -195,6 +150,13 @@ public class IcebergCatalogAdapter
       String prefix,
       Function<IcebergCatalogHandler, Response> action) {
     String catalogName = prefixParser.prefixToCatalogName(realmContext, 
prefix);
+    return withCatalogByName(securityContext, catalogName, action);
+  }
+
+  private Response withCatalogByName(
+      SecurityContext securityContext,
+      String catalogName,
+      Function<IcebergCatalogHandler, Response> action) {
     try (IcebergCatalogHandler wrapper = newHandlerWrapper(securityContext, 
catalogName)) {
       return action.apply(wrapper);
     } catch (RuntimeException e) {
@@ -213,6 +175,8 @@ public class IcebergCatalogAdapter
     return new IcebergCatalogHandler(
         diagnostics,
         callContext,
+        prefixParser,
+        resolverFactory,
         resolutionManifestFactory,
         metaStoreManager,
         credentialManager,
@@ -785,46 +749,7 @@ public class IcebergCatalogAdapter
   @Override
   public Response getConfig(
       String warehouse, RealmContext realmContext, SecurityContext 
securityContext) {
-    // 'warehouse' as an input here is catalogName.
-    // 'warehouse' as an output will be treated by the client as a default 
catalog
-    // storage
-    //    base location.
-    // 'prefix' as an output is the REST subpath that routes to the catalog
-    // resource,
-    //    which may be URL-escaped catalogName or potentially a different 
unique
-    // identifier for
-    //    the catalog being accessed.
-    // TODO: Push this down into PolarisCatalogHandlerWrapper for authorizing 
"any" catalog
-    // role in this catalog.
-    PolarisPrincipal authenticatedPrincipal = (PolarisPrincipal) 
securityContext.getUserPrincipal();
-    if (authenticatedPrincipal == null) {
-      throw new NotAuthorizedException("Failed to find authenticatedPrincipal 
in SecurityContext");
-    }
-    if (warehouse == null) {
-      throw new BadRequestException("Please specify a warehouse");
-    }
-    Resolver resolver = resolverFactory.createResolver(authenticatedPrincipal, 
warehouse);
-    ResolverStatus resolverStatus = resolver.resolveAll();
-    if (!resolverStatus.getStatus().equals(ResolverStatus.StatusEnum.SUCCESS)) 
{
-      throw new NotFoundException("Unable to find warehouse %s", warehouse);
-    }
-    ResolvedPolarisEntity resolvedReferenceCatalog = 
resolver.getResolvedReferenceCatalog();
-    Map<String, String> properties =
-        
PolarisEntity.of(resolvedReferenceCatalog.getEntity()).getPropertiesAsMap();
-
-    String prefix = prefixParser.catalogNameToPrefix(realmContext, warehouse);
-    return Response.ok(
-            ConfigResponse.builder()
-                .withDefaults(properties) // catalog properties are defaults
-                .withOverrides(ImmutableMap.of("prefix", prefix))
-                .withEndpoints(
-                    ImmutableList.<Endpoint>builder()
-                        .addAll(DEFAULT_ENDPOINTS)
-                        .addAll(VIEW_ENDPOINTS)
-                        
.addAll(PolarisEndpoints.getSupportedGenericTableEndpoints(realmConfig))
-                        
.addAll(PolarisEndpoints.getSupportedPolicyEndpoints(realmConfig))
-                        .build())
-                .build())
-        .build();
+    return withCatalogByName(
+        securityContext, warehouse, catalog -> 
Response.ok(catalog.getConfig()).build());
   }
 }
diff --git 
a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java
 
b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java
index 4f4ab0d1d..c5276ef6b 100644
--- 
a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java
+++ 
b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java
@@ -23,6 +23,9 @@ import static 
org.apache.polaris.core.config.FeatureConfiguration.LIST_PAGINATIO
 import static 
org.apache.polaris.service.catalog.AccessDelegationMode.VENDED_CREDENTIALS;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
 import io.smallrye.common.annotation.Identifier;
 import jakarta.annotation.Nonnull;
@@ -60,6 +63,8 @@ import org.apache.iceberg.exceptions.BadRequestException;
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.ForbiddenException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.rest.Endpoint;
 import org.apache.iceberg.rest.credentials.ImmutableCredential;
 import org.apache.iceberg.rest.requests.CommitTransactionRequest;
 import org.apache.iceberg.rest.requests.CreateNamespaceRequest;
@@ -69,6 +74,7 @@ import org.apache.iceberg.rest.requests.RegisterTableRequest;
 import org.apache.iceberg.rest.requests.RenameTableRequest;
 import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest;
 import org.apache.iceberg.rest.requests.UpdateTableRequest;
+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;
@@ -93,16 +99,22 @@ import org.apache.polaris.core.entity.PolarisEntityType;
 import org.apache.polaris.core.entity.table.IcebergTableLikeEntity;
 import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
 import org.apache.polaris.core.persistence.PolarisResolvedPathWrapper;
+import org.apache.polaris.core.persistence.ResolvedPolarisEntity;
 import 
org.apache.polaris.core.persistence.TransactionWorkspaceMetaStoreManager;
 import org.apache.polaris.core.persistence.dao.entity.EntitiesResult;
 import org.apache.polaris.core.persistence.dao.entity.EntityWithPath;
 import org.apache.polaris.core.persistence.pagination.Page;
 import org.apache.polaris.core.persistence.pagination.PageToken;
 import org.apache.polaris.core.persistence.resolver.ResolutionManifestFactory;
+import org.apache.polaris.core.persistence.resolver.Resolver;
+import org.apache.polaris.core.persistence.resolver.ResolverFactory;
+import org.apache.polaris.core.persistence.resolver.ResolverStatus;
+import org.apache.polaris.core.rest.PolarisEndpoints;
 import org.apache.polaris.core.storage.PolarisStorageActions;
 import org.apache.polaris.core.storage.StorageAccessConfig;
 import org.apache.polaris.core.storage.StorageUtil;
 import org.apache.polaris.service.catalog.AccessDelegationMode;
+import org.apache.polaris.service.catalog.CatalogPrefixParser;
 import org.apache.polaris.service.catalog.SupportsNotifications;
 import org.apache.polaris.service.catalog.common.CatalogHandler;
 import org.apache.polaris.service.catalog.common.CatalogUtils;
@@ -133,6 +145,39 @@ import org.slf4j.LoggerFactory;
 public class IcebergCatalogHandler extends CatalogHandler implements 
AutoCloseable {
   private static final Logger LOGGER = 
LoggerFactory.getLogger(IcebergCatalogHandler.class);
 
+  private static final Set<Endpoint> DEFAULT_ENDPOINTS =
+      ImmutableSet.<Endpoint>builder()
+          .add(Endpoint.V1_LIST_NAMESPACES)
+          .add(Endpoint.V1_LOAD_NAMESPACE)
+          .add(Endpoint.V1_NAMESPACE_EXISTS)
+          .add(Endpoint.V1_CREATE_NAMESPACE)
+          .add(Endpoint.V1_UPDATE_NAMESPACE)
+          .add(Endpoint.V1_DELETE_NAMESPACE)
+          .add(Endpoint.V1_LIST_TABLES)
+          .add(Endpoint.V1_LOAD_TABLE)
+          .add(Endpoint.V1_TABLE_EXISTS)
+          .add(Endpoint.V1_CREATE_TABLE)
+          .add(Endpoint.V1_UPDATE_TABLE)
+          .add(Endpoint.V1_DELETE_TABLE)
+          .add(Endpoint.V1_RENAME_TABLE)
+          .add(Endpoint.V1_REGISTER_TABLE)
+          .add(Endpoint.V1_REPORT_METRICS)
+          .add(Endpoint.V1_COMMIT_TRANSACTION)
+          .build();
+
+  private static final Set<Endpoint> VIEW_ENDPOINTS =
+      ImmutableSet.<Endpoint>builder()
+          .add(Endpoint.V1_LIST_VIEWS)
+          .add(Endpoint.V1_LOAD_VIEW)
+          .add(Endpoint.V1_VIEW_EXISTS)
+          .add(Endpoint.V1_CREATE_VIEW)
+          .add(Endpoint.V1_UPDATE_VIEW)
+          .add(Endpoint.V1_DELETE_VIEW)
+          .add(Endpoint.V1_RENAME_VIEW)
+          .build();
+
+  private final CatalogPrefixParser prefixParser;
+  private final ResolverFactory resolverFactory;
   private final PolarisMetaStoreManager metaStoreManager;
   private final CallContextCatalogFactory catalogFactory;
   private final ReservedProperties reservedProperties;
@@ -151,6 +196,8 @@ public class IcebergCatalogHandler extends CatalogHandler 
implements AutoCloseab
   public IcebergCatalogHandler(
       PolarisDiagnostics diagnostics,
       CallContext callContext,
+      CatalogPrefixParser prefixParser,
+      ResolverFactory resolverFactory,
       ResolutionManifestFactory resolutionManifestFactory,
       PolarisMetaStoreManager metaStoreManager,
       PolarisCredentialManager credentialManager,
@@ -171,6 +218,8 @@ public class IcebergCatalogHandler extends CatalogHandler 
implements AutoCloseab
         authorizer,
         credentialManager,
         externalCatalogFactories);
+    this.prefixParser = prefixParser;
+    this.resolverFactory = resolverFactory;
     this.metaStoreManager = metaStoreManager;
     this.catalogFactory = catalogFactory;
     this.reservedProperties = reservedProperties;
@@ -1265,4 +1314,37 @@ public class IcebergCatalogHandler extends 
CatalogHandler implements AutoCloseab
       closeable.close();
     }
   }
+
+  public ConfigResponse getConfig() {
+    // 'catalogName' is taken from the REST request's 'warehouse' query 
parameter.
+    // 'warehouse' as an output will be treated by the client as a default 
catalog
+    //   storage base location.
+    // 'prefix' as an output is the REST subpath that routes to the catalog
+    //   resource, which may be URL-escaped catalogName or potentially a 
different
+    //   unique identifier for the catalog being accessed.
+    if (catalogName == null) {
+      throw new BadRequestException("Please specify a warehouse");
+    }
+    Resolver resolver = resolverFactory.createResolver(polarisPrincipal, 
catalogName);
+    ResolverStatus resolverStatus = resolver.resolveAll();
+    if (!resolverStatus.getStatus().equals(ResolverStatus.StatusEnum.SUCCESS)) 
{
+      throw new NotFoundException("Unable to find warehouse %s", catalogName);
+    }
+    ResolvedPolarisEntity resolvedReferenceCatalog = 
resolver.getResolvedReferenceCatalog();
+    Map<String, String> properties =
+        
PolarisEntity.of(resolvedReferenceCatalog.getEntity()).getPropertiesAsMap();
+
+    String prefix = 
prefixParser.catalogNameToPrefix(callContext.getRealmContext(), catalogName);
+    return ConfigResponse.builder()
+        .withDefaults(properties) // catalog properties are defaults
+        .withOverrides(ImmutableMap.of("prefix", prefix))
+        .withEndpoints(
+            ImmutableList.<Endpoint>builder()
+                .addAll(DEFAULT_ENDPOINTS)
+                .addAll(VIEW_ENDPOINTS)
+                
.addAll(PolarisEndpoints.getSupportedGenericTableEndpoints(realmConfig))
+                
.addAll(PolarisEndpoints.getSupportedPolicyEndpoints(realmConfig))
+                .build())
+        .build();
+  }
 }
diff --git 
a/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerAuthzTest.java
 
b/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerAuthzTest.java
index dd4bc57de..613d37e4d 100644
--- 
a/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerAuthzTest.java
+++ 
b/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerAuthzTest.java
@@ -69,6 +69,7 @@ import org.apache.polaris.core.entity.PrincipalEntity;
 import org.apache.polaris.core.persistence.dao.entity.CreatePrincipalResult;
 import org.apache.polaris.core.persistence.resolver.PolarisResolutionManifest;
 import org.apache.polaris.service.admin.PolarisAuthzTestBase;
+import org.apache.polaris.service.catalog.CatalogPrefixParser;
 import org.apache.polaris.service.context.catalog.CallContextCatalogFactory;
 import 
org.apache.polaris.service.context.catalog.PolarisCallContextCatalogFactory;
 import org.apache.polaris.service.http.IfNoneMatch;
@@ -98,6 +99,7 @@ public class IcebergCatalogHandlerAuthzTest extends 
PolarisAuthzTestBase {
 
   @Inject CallContextCatalogFactory callContextCatalogFactory;
   @Inject Instance<ExternalCatalogFactory> externalCatalogFactories;
+  @Inject CatalogPrefixParser prefixParser;
 
   @SuppressWarnings("unchecked")
   private static Instance<ExternalCatalogFactory> 
emptyExternalCatalogFactory() {
@@ -122,6 +124,8 @@ public class IcebergCatalogHandlerAuthzTest extends 
PolarisAuthzTestBase {
     return new IcebergCatalogHandler(
         diagServices,
         callContext,
+        prefixParser,
+        resolverFactory,
         resolutionManifestFactory,
         metaStoreManager,
         credentialManager,
@@ -261,6 +265,8 @@ public class IcebergCatalogHandlerAuthzTest extends 
PolarisAuthzTestBase {
         new IcebergCatalogHandler(
             diagServices,
             callContext,
+            prefixParser,
+            resolverFactory,
             resolutionManifestFactory,
             metaStoreManager,
             credentialManager,
@@ -298,6 +304,8 @@ public class IcebergCatalogHandlerAuthzTest extends 
PolarisAuthzTestBase {
         new IcebergCatalogHandler(
             diagServices,
             callContext,
+            prefixParser,
+            resolverFactory,
             resolutionManifestFactory,
             metaStoreManager,
             credentialManager,
@@ -1185,6 +1193,8 @@ public class IcebergCatalogHandlerAuthzTest extends 
PolarisAuthzTestBase {
     return new IcebergCatalogHandler(
         diagServices,
         mockCallContext,
+        prefixParser,
+        resolverFactory,
         resolutionManifestFactory,
         metaStoreManager,
         credentialManager,
diff --git 
a/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerFineGrainedDisabledTest.java
 
b/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerFineGrainedDisabledTest.java
index cc59fb088..adeaa38a1 100644
--- 
a/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerFineGrainedDisabledTest.java
+++ 
b/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerFineGrainedDisabledTest.java
@@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableMap;
 import io.quarkus.test.junit.QuarkusTest;
 import io.quarkus.test.junit.TestProfile;
 import jakarta.enterprise.inject.Instance;
+import jakarta.inject.Inject;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -31,6 +32,7 @@ import org.apache.iceberg.rest.requests.UpdateTableRequest;
 import org.apache.polaris.core.auth.PolarisPrincipal;
 import org.apache.polaris.core.entity.PolarisPrivilege;
 import org.apache.polaris.service.admin.PolarisAuthzTestBase;
+import org.apache.polaris.service.catalog.CatalogPrefixParser;
 import org.apache.polaris.service.context.catalog.CallContextCatalogFactory;
 import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
@@ -43,7 +45,8 @@ import org.mockito.Mockito;
 @TestProfile(IcebergCatalogHandlerFineGrainedDisabledTest.Profile.class)
 public class IcebergCatalogHandlerFineGrainedDisabledTest extends 
PolarisAuthzTestBase {
 
-  @jakarta.inject.Inject CallContextCatalogFactory callContextCatalogFactory;
+  @Inject CallContextCatalogFactory callContextCatalogFactory;
+  @Inject CatalogPrefixParser prefixParser;
 
   @SuppressWarnings("unchecked")
   private static 
Instance<org.apache.polaris.core.catalog.ExternalCatalogFactory>
@@ -60,6 +63,8 @@ public class IcebergCatalogHandlerFineGrainedDisabledTest 
extends PolarisAuthzTe
     return new IcebergCatalogHandler(
         diagServices,
         callContext,
+        prefixParser,
+        resolverFactory,
         resolutionManifestFactory,
         metaStoreManager,
         credentialManager,

Reply via email to