diff --git a/extensions/auth/opa/impl/src/test/java/org/apache/polaris/extension/auth/opa/OpaPolarisAuthorizerTest.java b/extensions/auth/opa/impl/src/test/java/org/apache/polaris/extension/auth/opa/OpaPolarisAuthorizerTest.java
index 7401004a8a..b0e6ce8681 100644
--- a/extensions/auth/opa/impl/src/test/java/org/apache/polaris/extension/auth/opa/OpaPolarisAuthorizerTest.java
+++ b/extensions/auth/opa/impl/src/test/java/org/apache/polaris/extension/auth/opa/OpaPolarisAuthorizerTest.java
@@ -36,6 +36,8 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
import org.apache.hc.client5.http.classic.methods.HttpPost;
import org.apache.hc.client5.http.impl.classic.CloseableHttpClient;
import org.apache.hc.client5.http.impl.classic.HttpClients;
@@ -108,6 +110,50 @@ void testOpaInputJsonFormat() throws Exception {
}
}
+ @Test
+ void testExternalPrincipalRolesAndNameAreSentToOpa() throws Exception {
+ final String[] capturedRequestBody = new String[1];
+
+ HttpServer server = createServerWithRequestCapture(capturedRequestBody);
+ try {
+ URI policyUri =
+ URI.create(
+ "http://localhost:" + server.getAddress().getPort() + "/v1/data/polaris/allow");
+ OpaPolarisAuthorizer authorizer =
+ new OpaPolarisAuthorizer(
+ policyUri, HttpClients.createDefault(), new ObjectMapper(), null);
+
+ PolarisPrincipal principal =
+ PolarisPrincipal.of(
+ "external-user",
+ Map.of("external", "true", "principalId", "42"),
+ Set.of("ext-role", "common-role"));
+
+ PolarisResolvedPathWrapper target = new PolarisResolvedPathWrapper(List.of());
+
+ assertThatNoException()
+ .isThrownBy(
+ () ->
+ authorizer.authorizeOrThrow(
+ principal,
+ Set.of(),
+ PolarisAuthorizableOperation.LIST_NAMESPACES,
+ target,
+ null));
+
+ JsonNode actorNode =
+ new ObjectMapper().readTree(capturedRequestBody[0]).path("input").path("actor");
+ assertThat(actorNode.get("principal").asText()).isEqualTo("external-user");
+ assertThat(
+ StreamSupport.stream(actorNode.get("roles").spliterator(), false)
+ .map(JsonNode::asText)
+ .collect(Collectors.toSet()))
+ .containsExactlyInAnyOrder("ext-role", "common-role");
+ } finally {
+ server.stop(0);
+ }
+ }
+
@Test
void testOpaRequestJsonWithHierarchicalResource() throws Exception {
// Capture the request body for verification
diff --git a/extensions/auth/opa/tests/build.gradle.kts b/extensions/auth/opa/tests/build.gradle.kts
index e752d808bc..37276c3cc3 100644
--- a/extensions/auth/opa/tests/build.gradle.kts
+++ b/extensions/auth/opa/tests/build.gradle.kts
@@ -38,8 +38,12 @@ dependencies {
testImplementation(project(":polaris-runtime-test-common"))
// Test dependencies
+ intTestImplementation(platform(libs.iceberg.bom))
+ intTestImplementation("org.apache.iceberg:iceberg-api")
+ intTestImplementation(project(":polaris-core"))
intTestImplementation("io.quarkus:quarkus-junit5")
intTestImplementation("io.rest-assured:rest-assured")
+ intTestImplementation("io.quarkus:quarkus-security")
// Test container dependencies
intTestImplementation(platform(libs.testcontainers.bom))
diff --git a/extensions/auth/opa/tests/src/intTest/java/org/apache/polaris/extension/auth/opa/test/OpaExternalAuthIntegrationTest.java b/extensions/auth/opa/tests/src/intTest/java/org/apache/polaris/extension/auth/opa/test/OpaExternalAuthIntegrationTest.java
new file mode 100644
index 0000000000..b42f6efa5c
--- /dev/null
+++ b/extensions/auth/opa/tests/src/intTest/java/org/apache/polaris/extension/auth/opa/test/OpaExternalAuthIntegrationTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.extension.auth.opa.test;
+
+import static io.restassured.RestAssured.given;
+
+import io.quarkus.test.junit.QuarkusTest;
+import io.quarkus.test.junit.QuarkusTestProfile;
+import io.quarkus.test.junit.QuarkusTestProfile.TestResourceEntry;
+import io.quarkus.test.junit.TestProfile;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Integration test that exercises OPA authorization when Polaris uses the external authenticator.
+ *
+ *
Authentication is driven entirely by test headers via {@link
+ * TestExternalHeaderAuthenticationMechanism}, ensuring no internal principal lookups occur.
+ */
+@QuarkusTest
+@TestProfile(OpaExternalAuthIntegrationTest.ExternalOpaProfile.class)
+public class OpaExternalAuthIntegrationTest extends OpaIntegrationTestBase {
+
+ public static class ExternalOpaProfile implements QuarkusTestProfile {
+ @Override
+ public Map getConfigOverrides() {
+ Map config = new HashMap<>();
+ config.put("polaris.authorization.type", "opa");
+ config.put("polaris.authorization.opa.auth.type", "bearer");
+ config.put(
+ "polaris.authorization.opa.auth.bearer.static-token.value",
+ "test-opa-bearer-token-12345");
+
+ // Enable external authentication and skip internal token services
+ config.put("polaris.authentication.type", "external");
+ config.put("polaris.authentication.authenticator.type", "external");
+ config.put("polaris.authentication.token-broker.type", "none");
+ config.put("polaris.authentication.token-service.type", "disabled");
+
+ // OIDC not used in this flow
+ config.put("quarkus.oidc.enabled", "false");
+
+ return config;
+ }
+
+ @Override
+ public List testResources() {
+ return List.of(new TestResourceEntry(OpaTestResource.class));
+ }
+ }
+
+ @Test
+ void testExternalPrincipalAllowed() {
+ given()
+ .header(TestExternalHeaderAuthenticationMechanism.PRINCIPAL_HEADER, "admin")
+ .header(TestExternalHeaderAuthenticationMechanism.ROLES_HEADER, "ext-role")
+ .when()
+ .get("/api/management/v1/catalogs")
+ .then()
+ .statusCode(200);
+ }
+
+ @Test
+ void testExternalPrincipalDenied() {
+ given()
+ .header(TestExternalHeaderAuthenticationMechanism.PRINCIPAL_HEADER, "stranger")
+ .header(TestExternalHeaderAuthenticationMechanism.ROLES_HEADER, "unknown-role")
+ .when()
+ .get("/api/management/v1/catalogs")
+ .then()
+ .statusCode(403);
+ }
+}
diff --git a/extensions/auth/opa/tests/src/intTest/java/org/apache/polaris/extension/auth/opa/test/TestExternalHeaderAuthenticationMechanism.java b/extensions/auth/opa/tests/src/intTest/java/org/apache/polaris/extension/auth/opa/test/TestExternalHeaderAuthenticationMechanism.java
new file mode 100644
index 0000000000..0f9bf8b806
--- /dev/null
+++ b/extensions/auth/opa/tests/src/intTest/java/org/apache/polaris/extension/auth/opa/test/TestExternalHeaderAuthenticationMechanism.java
@@ -0,0 +1,105 @@
+/*
+ * 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.extension.auth.opa.test;
+
+import io.quarkus.security.identity.IdentityProviderManager;
+import io.quarkus.security.identity.SecurityIdentity;
+import io.quarkus.security.runtime.QuarkusSecurityIdentity;
+import io.quarkus.vertx.http.runtime.security.ChallengeData;
+import io.quarkus.vertx.http.runtime.security.HttpAuthenticationMechanism;
+import io.quarkus.vertx.http.runtime.security.HttpCredentialTransport;
+import io.smallrye.mutiny.Uni;
+import io.vertx.ext.web.RoutingContext;
+import jakarta.annotation.Priority;
+import jakarta.enterprise.context.ApplicationScoped;
+import jakarta.inject.Inject;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.polaris.core.auth.PolarisPrincipal;
+import org.apache.polaris.service.auth.Authenticator;
+import org.apache.polaris.service.auth.PolarisCredential;
+import org.jboss.logging.Logger;
+
+/**
+ * Test-only authentication mechanism that turns test headers into external {@link
+ * PolarisCredential} instances. This avoids any DB lookups and exercises the external authenticator
+ * flow.
+ */
+@ApplicationScoped
+@Priority(HttpAuthenticationMechanism.DEFAULT_PRIORITY + 200)
+public class TestExternalHeaderAuthenticationMechanism implements HttpAuthenticationMechanism {
+
+ private static final Logger LOG =
+ Logger.getLogger(TestExternalHeaderAuthenticationMechanism.class);
+
+ static final String PRINCIPAL_HEADER = "X-External-Principal";
+ static final String ROLES_HEADER = "X-External-Roles";
+
+ @Inject Authenticator authenticator;
+
+ @Override
+ public Uni authenticate(
+ RoutingContext context, IdentityProviderManager identityProviderManager) {
+ String principal = context.request().getHeader(PRINCIPAL_HEADER);
+ if (principal == null || principal.isBlank()) {
+ return Uni.createFrom().nullItem();
+ }
+ Set roles = parseRoles(context.request().getHeader(ROLES_HEADER));
+ PolarisCredential credential = PolarisCredential.of(null, principal, roles, true);
+ PolarisPrincipal polarisPrincipal = authenticator.authenticate(credential);
+ QuarkusSecurityIdentity identity =
+ QuarkusSecurityIdentity.builder()
+ .setPrincipal(polarisPrincipal)
+ .addCredential(credential)
+ .addRoles(polarisPrincipal.getRoles())
+ .setAnonymous(false)
+ .build();
+ LOG.debugf(
+ "Authenticated external principal from headers principal=%s roles=%s", principal, roles);
+ return Uni.createFrom().item(identity);
+ }
+
+ @Override
+ public Uni getChallenge(RoutingContext context) {
+ return Uni.createFrom().nullItem();
+ }
+
+ @Override
+ public Set>
+ getCredentialTypes() {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public Uni getCredentialTransport(RoutingContext context) {
+ return Uni.createFrom().nullItem();
+ }
+
+ private Set parseRoles(String header) {
+ if (header == null || header.isBlank()) {
+ return Set.of();
+ }
+ return Arrays.stream(header.split(","))
+ .map(String::trim)
+ .filter(s -> !s.isEmpty())
+ .collect(Collectors.toSet());
+ }
+}
diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/resolver/Resolver.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/resolver/Resolver.java
index 50c4a4f193..a1d5b30a18 100644
--- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/resolver/Resolver.java
+++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/resolver/Resolver.java
@@ -41,6 +41,7 @@
import org.apache.polaris.core.entity.PolarisEntityType;
import org.apache.polaris.core.entity.PolarisGrantRecord;
import org.apache.polaris.core.entity.PolarisPrivilege;
+import org.apache.polaris.core.entity.PrincipalEntity;
import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
import org.apache.polaris.core.persistence.ResolvedPolarisEntity;
import org.apache.polaris.core.persistence.cache.EntityCache;
@@ -744,6 +745,20 @@ private ResolverStatus resolvePaths(
private ResolverStatus resolveCallerPrincipalAndPrincipalRoles(
List toValidate) {
+ if (isExternalPrincipal()) {
+ PrincipalEntity externalPrincipal = createExternalPrincipalEntity();
+ this.resolvedCallerPrincipal =
+ new ResolvedPolarisEntity(
+ diagnostics,
+ externalPrincipal,
+ List.of(),
+ externalPrincipal.getGrantRecordsVersion());
+ this.resolvedEntriesById.put(
+ this.resolvedCallerPrincipal.getEntity().getId(), this.resolvedCallerPrincipal);
+ this.resolvedCallerPrincipalRoles = List.of();
+ return new ResolverStatus(ResolverStatus.StatusEnum.SUCCESS);
+ }
+
// resolve the principal, by name or id
this.resolvedCallerPrincipal =
this.resolveByName(toValidate, PolarisEntityType.PRINCIPAL, polarisPrincipal.getName());
@@ -751,6 +766,21 @@ private ResolverStatus resolveCallerPrincipalAndPrincipalRoles(
// if the principal was not found, we can end right there
if (this.resolvedCallerPrincipal == null
|| this.resolvedCallerPrincipal.getEntity().isDropped()) {
+ if (isExternalPrincipal()) {
+ // For external principals we do not maintain principal entities in the metastore,
+ // so synthesize a placeholder entry and continue without resolving grants.
+ PrincipalEntity externalPrincipal = createExternalPrincipalEntity();
+ this.resolvedCallerPrincipal =
+ new ResolvedPolarisEntity(
+ diagnostics,
+ externalPrincipal,
+ List.of(),
+ externalPrincipal.getGrantRecordsVersion());
+ this.resolvedEntriesById.put(
+ this.resolvedCallerPrincipal.getEntity().getId(), this.resolvedCallerPrincipal);
+ this.resolvedCallerPrincipalRoles = List.of();
+ return new ResolverStatus(ResolverStatus.StatusEnum.SUCCESS);
+ }
return new ResolverStatus(ResolverStatus.StatusEnum.CALLER_PRINCIPAL_DOES_NOT_EXIST);
}
@@ -764,6 +794,17 @@ private ResolverStatus resolveCallerPrincipalAndPrincipalRoles(
return new ResolverStatus(ResolverStatus.StatusEnum.SUCCESS);
}
+ private boolean isExternalPrincipal() {
+ return Boolean.parseBoolean(polarisPrincipal.getProperties().getOrDefault("external", "false"));
+ }
+
+ private PrincipalEntity createExternalPrincipalEntity() {
+ return new PrincipalEntity.Builder()
+ .setName(polarisPrincipal.getName())
+ .setProperties(polarisPrincipal.getProperties())
+ .build();
+ }
+
/**
* Resolve all principal roles that the principal has grants for
*
diff --git a/runtime/service/src/main/java/org/apache/polaris/service/auth/ExternalAuthenticator.java b/runtime/service/src/main/java/org/apache/polaris/service/auth/ExternalAuthenticator.java
new file mode 100644
index 0000000000..ac63ba3920
--- /dev/null
+++ b/runtime/service/src/main/java/org/apache/polaris/service/auth/ExternalAuthenticator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.service.auth;
+
+import io.smallrye.common.annotation.Identifier;
+import jakarta.enterprise.context.RequestScoped;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+import org.apache.polaris.core.auth.PolarisPrincipal;
+
+/**
+ * Authenticator for principals supplied by an external identity provider.
+ *
+ * Builds an in-memory {@link PolarisPrincipal} directly from credential claims without any
+ * persistence lookups.
+ */
+@RequestScoped
+@Identifier("external")
+public class ExternalAuthenticator implements Authenticator {
+
+ @Override
+ public PolarisPrincipal authenticate(PolarisCredential credentials) {
+ PolarisCredential externalCredentials = ensureExternal(credentials);
+
+ String principalName = externalCredentials.getPrincipalName();
+ Long principalId = externalCredentials.getPrincipalId();
+ if (principalName == null && principalId != null) {
+ principalName = String.valueOf(principalId);
+ }
+ if (principalName == null) {
+ throw new NotAuthorizedException("Unable to authenticate external principal");
+ }
+
+ Set principalRoles = externalCredentials.getPrincipalRoles();
+ Map properties = new HashMap<>();
+ if (principalId != null) {
+ properties.put("principalId", principalId.toString());
+ }
+ properties.put("external", Boolean.toString(true));
+ if (externalCredentials.getPrincipalName() != null) {
+ properties.put("principalName", externalCredentials.getPrincipalName());
+ }
+
+ return PolarisPrincipal.of(principalName, properties, principalRoles);
+ }
+
+ private PolarisCredential ensureExternal(PolarisCredential credentials) {
+ Objects.requireNonNull(credentials, "credentials");
+ if (credentials.isExternal()) {
+ return credentials;
+ }
+ return PolarisCredential.of(
+ credentials.getPrincipalId(),
+ credentials.getPrincipalName(),
+ credentials.getPrincipalRoles(),
+ true);
+ }
+}
diff --git a/runtime/service/src/main/java/org/apache/polaris/service/auth/PolarisCredential.java b/runtime/service/src/main/java/org/apache/polaris/service/auth/PolarisCredential.java
index f86565e669..207aee03dd 100644
--- a/runtime/service/src/main/java/org/apache/polaris/service/auth/PolarisCredential.java
+++ b/runtime/service/src/main/java/org/apache/polaris/service/auth/PolarisCredential.java
@@ -22,6 +22,7 @@
import jakarta.annotation.Nullable;
import java.util.Set;
import org.apache.polaris.immutables.PolarisImmutable;
+import org.immutables.value.Value;
/**
* A Quarkus Security {@link Credential} exposing Polaris-specific attributes: the principal id,
@@ -32,10 +33,19 @@ public interface PolarisCredential extends Credential {
static PolarisCredential of(
@Nullable Long principalId, @Nullable String principalName, Set principalRoles) {
+ return of(principalId, principalName, principalRoles, false);
+ }
+
+ static PolarisCredential of(
+ @Nullable Long principalId,
+ @Nullable String principalName,
+ Set principalRoles,
+ boolean external) {
return ImmutablePolarisCredential.builder()
.principalId(principalId)
.principalName(principalName)
.principalRoles(principalRoles)
+ .external(external)
.build();
}
@@ -49,4 +59,10 @@ static PolarisCredential of(
/** The principal roles, or empty if the principal has no roles. */
Set getPrincipalRoles();
+
+ /** True if the credential represents an external principal. */
+ @Value.Default
+ default boolean isExternal() {
+ return false;
+ }
}
diff --git a/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/InternalAuthenticationMechanism.java b/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/InternalAuthenticationMechanism.java
index 51d910d6da..9c006d4b9f 100644
--- a/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/InternalAuthenticationMechanism.java
+++ b/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/InternalAuthenticationMechanism.java
@@ -25,7 +25,6 @@
import io.quarkus.security.identity.IdentityProviderManager;
import io.quarkus.security.identity.SecurityIdentity;
import io.quarkus.security.identity.request.AuthenticationRequest;
-import io.quarkus.security.identity.request.TokenAuthenticationRequest;
import io.quarkus.vertx.http.runtime.security.ChallengeData;
import io.quarkus.vertx.http.runtime.security.HttpAuthenticationMechanism;
import io.quarkus.vertx.http.runtime.security.HttpCredentialTransport;
@@ -118,7 +117,7 @@ public Uni getChallenge(RoutingContext context) {
@Override
public Set> getCredentialTypes() {
- return Collections.singleton(TokenAuthenticationRequest.class);
+ return Collections.singleton(InternalAuthenticationRequest.class);
}
@Override
diff --git a/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/broker/InternalPolarisToken.java b/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/broker/InternalPolarisToken.java
index 8047d33180..494c037c27 100644
--- a/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/broker/InternalPolarisToken.java
+++ b/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/broker/InternalPolarisToken.java
@@ -42,11 +42,17 @@ abstract class InternalPolarisToken implements PolarisCredential {
static InternalPolarisToken of(
String principalName, Long principalId, String clientId, String scope) {
+ return of(principalName, principalId, clientId, scope, false);
+ }
+
+ static InternalPolarisToken of(
+ String principalName, Long principalId, String clientId, String scope, boolean external) {
return ImmutableInternalPolarisToken.builder()
.principalName(principalName)
.principalId(principalId)
.clientId(clientId)
.scope(scope)
+ .external(external)
.build();
}
@@ -60,6 +66,12 @@ static InternalPolarisToken of(
@SuppressWarnings("NullableProblems")
public abstract Long getPrincipalId();
+ @Value.Default
+ @Override
+ public boolean isExternal() {
+ return false;
+ }
+
@Value.Lazy
@Override
public Set getPrincipalRoles() {
diff --git a/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/broker/NoneTokenBrokerFactory.java b/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/broker/NoneTokenBrokerFactory.java
new file mode 100644
index 0000000000..7c1382309f
--- /dev/null
+++ b/runtime/service/src/main/java/org/apache/polaris/service/auth/internal/broker/NoneTokenBrokerFactory.java
@@ -0,0 +1,78 @@
+/*
+ * 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.service.auth.internal.broker;
+
+import io.smallrye.common.annotation.Identifier;
+import jakarta.enterprise.context.ApplicationScoped;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+import org.apache.polaris.core.PolarisCallContext;
+import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
+import org.apache.polaris.service.auth.PolarisCredential;
+import org.apache.polaris.service.auth.internal.service.OAuthError;
+import org.apache.polaris.service.types.TokenType;
+
+/** A no-op token broker factory used when authentication is delegated to an external IdP. */
+@ApplicationScoped
+@Identifier("none")
+public class NoneTokenBrokerFactory implements TokenBrokerFactory {
+
+ private static final TokenBroker DISABLED_TOKEN_BROKER =
+ new TokenBroker() {
+ @Override
+ public boolean supportsGrantType(String grantType) {
+ return false;
+ }
+
+ @Override
+ public boolean supportsRequestedTokenType(TokenType tokenType) {
+ return false;
+ }
+
+ @Override
+ public TokenResponse generateFromClientSecrets(
+ String clientId,
+ String clientSecret,
+ String grantType,
+ String scope,
+ TokenType requestedTokenType) {
+ return TokenResponse.of(OAuthError.invalid_request);
+ }
+
+ @Override
+ public PolarisCredential verify(String token) {
+ throw new NotAuthorizedException("Token broker is disabled for external authentication");
+ }
+
+ @Override
+ public TokenResponse generateFromToken(
+ TokenType subjectTokenType,
+ String subjectToken,
+ String grantType,
+ String scope,
+ TokenType requestedTokenType) {
+ return TokenResponse.of(OAuthError.invalid_request);
+ }
+ };
+
+ @Override
+ public TokenBroker create(
+ PolarisMetaStoreManager polarisMetaStoreManager, PolarisCallContext polarisCallContext) {
+ return DISABLED_TOKEN_BROKER;
+ }
+}
diff --git a/runtime/service/src/test/java/org/apache/polaris/service/auth/DefaultAuthenticatorTest.java b/runtime/service/src/test/java/org/apache/polaris/service/auth/DefaultAuthenticatorTest.java
index 5924745d96..6a246e8a01 100644
--- a/runtime/service/src/test/java/org/apache/polaris/service/auth/DefaultAuthenticatorTest.java
+++ b/runtime/service/src/test/java/org/apache/polaris/service/auth/DefaultAuthenticatorTest.java
@@ -70,6 +70,11 @@ public void before(TestInfo testInfo) {
metaStoreManager, PRINCIPAL_NO_ROLES, principal.getCredentials(), polarisContext);
}
+ @Test
+ void credentialsDefaultToInternal() {
+ assertThat(PolarisCredential.of(null, PRINCIPAL_NAME, Set.of()).isExternal()).isFalse();
+ }
+
@Test
void testNullPrincipalIdAndName() {
// Given: credentials with both null principal ID and name
diff --git a/runtime/service/src/test/java/org/apache/polaris/service/auth/ExternalAuthenticatorTest.java b/runtime/service/src/test/java/org/apache/polaris/service/auth/ExternalAuthenticatorTest.java
new file mode 100644
index 0000000000..6902bbe03b
--- /dev/null
+++ b/runtime/service/src/test/java/org/apache/polaris/service/auth/ExternalAuthenticatorTest.java
@@ -0,0 +1,124 @@
+/*
+ * 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.service.auth;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import io.quarkus.test.junit.QuarkusMock;
+import io.quarkus.test.junit.QuarkusTest;
+import io.quarkus.test.junit.QuarkusTestProfile;
+import io.quarkus.test.junit.TestProfile;
+import io.smallrye.common.annotation.Identifier;
+import jakarta.enterprise.inject.Any;
+import jakarta.enterprise.inject.Instance;
+import jakarta.inject.Inject;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.exceptions.NotAuthorizedException;
+import org.apache.polaris.core.auth.PolarisPrincipal;
+import org.apache.polaris.core.context.RealmContext;
+import org.apache.polaris.service.catalog.Profiles;
+import org.apache.polaris.service.context.catalog.RealmContextHolder;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+
+@QuarkusTest
+@TestProfile(ExternalAuthenticatorTest.Profile.class)
+class ExternalAuthenticatorTest {
+
+ private static final String REALM_NAME = "external-realm";
+
+ @Inject AuthenticationRealmConfiguration realmConfiguration;
+ @Inject @Any Instance authenticators;
+ @Inject RealmContextHolder realmContextHolder;
+ private Authenticator authenticator;
+
+ @BeforeEach
+ void setUp(TestInfo testInfo) {
+ RealmContext realmContext = () -> REALM_NAME + "-" + testInfo.getDisplayName();
+ QuarkusMock.installMockForType(realmContext, RealmContext.class);
+ realmContextHolder.set(realmContext);
+ authenticator =
+ authenticators
+ .select(Identifier.Literal.of(realmConfiguration.authenticator().type()))
+ .get();
+ }
+
+ @Test
+ void selectsExternalAuthenticator() {
+ assertThat(authenticator).isInstanceOf(ExternalAuthenticator.class);
+ }
+
+ @Test
+ void buildsPrincipalFromClaims() {
+ PolarisCredential credential =
+ PolarisCredential.of(42L, "external-user", Set.of("roleA", "roleB"), true);
+
+ PolarisPrincipal principal = authenticator.authenticate(credential);
+
+ assertThat(principal.getName()).isEqualTo("external-user");
+ assertThat(principal.getRoles()).containsExactlyInAnyOrder("roleA", "roleB");
+ assertThat(principal.getProperties())
+ .containsEntry("external", "true")
+ .containsEntry("principalId", "42")
+ .containsEntry("principalName", "external-user");
+ }
+
+ @Test
+ void fallsBackToPrincipalIdWhenNameMissing() {
+ PolarisCredential credential = PolarisCredential.of(7L, null, Set.of("roleA"), true);
+
+ PolarisPrincipal principal = authenticator.authenticate(credential);
+
+ assertThat(principal.getName()).isEqualTo("7");
+ assertThat(principal.getRoles()).containsExactly("roleA");
+ }
+
+ @Test
+ void rejectsMissingIdentifiers() {
+ PolarisCredential credential = PolarisCredential.of(null, null, Set.of(), true);
+
+ assertThatThrownBy(() -> authenticator.authenticate(credential))
+ .isInstanceOf(NotAuthorizedException.class);
+ }
+
+ @Test
+ void marksCredentialAsExternalWhenFlagMissing() {
+ PolarisCredential credential = PolarisCredential.of(9L, "fallback", Set.of("roleA"));
+
+ PolarisPrincipal principal = authenticator.authenticate(credential);
+
+ assertThat(principal.getProperties()).containsEntry("external", "true");
+ }
+
+ public static class Profile extends Profiles.DefaultProfile implements QuarkusTestProfile {
+ @Override
+ public Map getConfigOverrides() {
+ Map overrides = new HashMap<>(super.getConfigOverrides());
+ overrides.put("polaris.authentication.type", "external");
+ overrides.put("polaris.authentication.authenticator.type", "external");
+ overrides.put("polaris.authentication.token-broker.type", "none");
+ overrides.put("polaris.authentication.token-service.type", "disabled");
+ return overrides;
+ }
+ }
+}