From e877e7e333421337c7ef47c17cb932825cdb5806 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 12 May 2026 17:49:27 -0400 Subject: [PATCH 01/24] cloud credential provider JVM side --- .../cloud/CometCloudCredentialDispatcher.java | 174 ++++++++++++++++++ .../cloud/CometCloudCredentialProvider.java | 48 +++++ .../apache/comet/cloud/CometCredentials.java | 74 ++++++++ .../CometCloudCredentialDispatcherTest.java | 93 ++++++++++ .../TestCometCloudCredentialProvider.java | 58 ++++++ ...e.comet.cloud.CometCloudCredentialProvider | 1 + 6 files changed, 448 insertions(+) create mode 100644 common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java create mode 100644 common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java create mode 100644 common/src/main/java/org/apache/comet/cloud/CometCredentials.java create mode 100644 common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java create mode 100644 common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java create mode 100644 common/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider diff --git a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java new file mode 100644 index 0000000000..3f9241c16a --- /dev/null +++ b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java @@ -0,0 +1,174 @@ +/* + * 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.comet.cloud; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.ServiceLoader; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// spotless:off +/* + * Architecture Overview: + * + * JVM Side | Native Side + * ┌──────────────────────────────────────────┐ | ┌──────────────────────────────────────────┐ + * │ CometCloudCredentialDispatcher │ | │ S3 Object Reading │ + * │ │ | │ │ + * │ ┌────────────────────────────────────┐ │ | │ ┌────────────────────────────────────┐ │ + * │ │ ServiceLoader discovery: │ │ | │ │ DataFusion iceberg-rust │ │ + * │ │ META-INF/services/ │ │ | │ │ object_store opendal │ │ + * │ │ o.a.c.cloud.CometCloudCred... │ │ | │ └────────────────────────────────────┘ │ + * │ └────────────────────────────────────┘ │ | │ │ │ │ + * │ │ │ | │ ▼ ▼ │ + * │ ▼ │ | │ ┌────────────────────────────────────┐ │ + * │ ┌────────────────────────────────────┐ │ | │ │ CometCredentialBridge (Rust) │ │ + * │ │ CometCloudCredentialProvider │ │ | │ │ impl object_store:: │ │ + * │ │ (single instance, cached) │ │ | │ │ CredentialProvider │ │ + * │ └────────────────────────────────────┘ │ | │ │ impl reqsign_core:: │ │ + * │ │ │ | │ │ ProvideCredential │ │ + * │ ▼ │ | │ └────────────────────────────────────┘ │ + * │ ┌────────────────────────────────────┐ │ | │ │ │ + * │ │ .getCredentialsForPath(...) │◄─┼───────┼─────┼──╗ ▼ │ + * │ └────────────────────────────────────┘ │ | │ ╔════════════════════════════════════╗ │ + * │ │ │ | │ ║ JNI CALL: ║ │ + * │ ▼ │ | │ ║ getCredentialsForPath( ║ │ + * │ ┌────────────────────────────────────┐ │ | │ ║ bucket, path) ║ │ + * │ │ return CometCredentials POJO │──┼───────┼─────┼─►║ ║ │ + * │ │ (access key, secret, token, │ │ | │ ╚════════════════════════════════════╝ │ + * │ │ region, expiration) │ │ | │ │ │ + * │ └────────────────────────────────────┘ │ | │ ▼ │ + * │ │ | │ ┌────────────────────────────────────┐ │ + * │ │ | │ │ AwsCredential │ │ + * │ │ | │ │ used to sign S3 requests │ │ + * │ │ | │ └────────────────────────────────────┘ │ + * └──────────────────────────────────────────┘ | └──────────────────────────────────────────┘ + * | + * JNI Boundary + * + * Setup Phase (one-time per executor): + * 1. Vendor JAR ships an impl of CometCloudCredentialProvider via META-INF/services. + * 2. CometCloudCredentialDispatcher resolves it via ServiceLoader on first class-load. + * 3. Native side caches dispatcher class + static method ID in OnceCell. + * + * Runtime Phase (per S3 request): + * 4. object_store / opendal calls its async credential trait on CometCredentialBridge. + * 5. Bridge enters JNI, invokes dispatcher.getCredentialsForPath(bucket, path). + * 6. Provider returns a CometCredentials POJO; vendor may call its own STS / authz service. + * 7. Rust reads fields via JNI accessors, returns AwsCredential for request signing. + */ +// spotless:on + +/** + * Static entry point invoked from Comet's native code (via JNI) to fetch AWS credentials for an S3 + * request. + * + *

Resolution rules at first class-load: + * + *

+ * + *

Discovery is via classpath only; there is no Comet-specific config knob for selecting a + * provider. This keeps the credentials Comet uses identical to whatever the same JVM would use if a + * query fell back to Spark execution mid-flight. + */ +public final class CometCloudCredentialDispatcher { + + private static final Logger LOG = + LoggerFactory.getLogger(CometCloudCredentialDispatcher.class); + + /* + * Process-lifetime singleton, justified per the contributor guide's "Global singletons" + * section. + * + * Why static is the right lifetime: ServiceLoader discovers the impl from the executor + * classpath, which is fixed once Spark has launched the JVM. The same instance must serve + * every credential request from native code so that a query falling back from Comet to + * Spark mid-execution sees identical credentials. + * + * Bounded: a single reference, not a cache. + * + * Credential refresh: this dispatcher does NOT cache credentials. Each call to + * getCredentialsForPath delegates straight to the provider, which is responsible for any + * STS / token refresh logic. Stale-credential failure modes therefore live in the provider + * impl, not here. + */ + private static final CometCloudCredentialProvider PROVIDER = resolve(); + + private CometCloudCredentialDispatcher() {} + + /** Returns true if a provider was discovered on the classpath. */ + public static boolean isProviderRegistered() { + return PROVIDER != null; + } + + /** + * Invoked by native code via JNI. Delegates to the registered provider. + * + * @throws IllegalStateException if no provider is registered (callers should check {@link + * #isProviderRegistered()} first) + */ + public static CometCredentials getCredentialsForPath(String bucket, String path) + throws Exception { + if (PROVIDER == null) { + throw new IllegalStateException( + "No CometCloudCredentialProvider registered; check META-INF/services on the classpath"); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Fetching credentials for bucket={} path={}", bucket, path); + } + return PROVIDER.getCredentialsForPath(bucket, path); + } + + private static CometCloudCredentialProvider resolve() { + List impls = new ArrayList<>(); + Iterator it = + ServiceLoader.load(CometCloudCredentialProvider.class).iterator(); + while (it.hasNext()) { + impls.add(it.next()); + } + if (impls.isEmpty()) { + LOG.info( + "No CometCloudCredentialProvider registered; native S3 readers will use the default " + + "AWS credential chain"); + return null; + } + if (impls.size() > 1) { + List names = new ArrayList<>(impls.size()); + for (CometCloudCredentialProvider impl : impls) { + names.add(impl.getClass().getName()); + } + LOG.error("Multiple CometCloudCredentialProvider impls on classpath: {}", names); + throw new IllegalStateException( + "Multiple CometCloudCredentialProvider impls on classpath: " + names); + } + CometCloudCredentialProvider provider = impls.get(0); + LOG.info("Registered CometCloudCredentialProvider: {}", provider.getClass().getName()); + return provider; + } +} diff --git a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java new file mode 100644 index 0000000000..c66c6de9dd --- /dev/null +++ b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java @@ -0,0 +1,48 @@ +/* + * 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.comet.cloud; + +/** + * SPI for supplying AWS credentials to Comet's native S3 readers. + * + *

Comet's native scan paths ({@code object_store} for raw Parquet, {@code opendal} via {@code + * iceberg-rust} for Iceberg) bypass Spark's Hadoop S3A code path. The standard {@code + * AWSCredentialsProvider.getCredentials()} contract has no path argument, so vendors that issue + * per-path STS credentials cannot expose them through that interface. This SPI fills the gap. + * + *

Vendors register an implementation via {@code + * META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider}. Comet discovers it at + * executor startup and routes every per-request credential fetch through it. + * + *

Implementations must be thread-safe; {@link #getCredentialsForPath} may be invoked + * concurrently from many native tokio tasks. + */ +public interface CometCloudCredentialProvider { + + /** + * Returns credentials usable to sign an S3 request for the given path. + * + * @param bucket the S3 bucket name (no scheme, no path) + * @param path the object key being accessed (no leading slash) + * @return credentials, or {@code null} if this provider does not handle the given path + * @throws Exception any failure surfaces to the native caller and aborts the request + */ + CometCredentials getCredentialsForPath(String bucket, String path) throws Exception; +} diff --git a/common/src/main/java/org/apache/comet/cloud/CometCredentials.java b/common/src/main/java/org/apache/comet/cloud/CometCredentials.java new file mode 100644 index 0000000000..1df349fa8a --- /dev/null +++ b/common/src/main/java/org/apache/comet/cloud/CometCredentials.java @@ -0,0 +1,74 @@ +/* + * 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.comet.cloud; + +import java.util.Objects; + +/** + * Credentials returned by a {@link CometCloudCredentialProvider}, consumed by Comet's native code + * via JNI field accessors. + * + *

{@code sessionToken} is null for non-STS credentials; {@code region} is null when the provider + * has no opinion (the native side falls back to its configured region). {@code + * expirationEpochMillis} is {@code 0} when the provider does not track expiration; in that case + * Comet will not pre-emptively refresh and relies on the provider to return fresh credentials on + * each call. + */ +public final class CometCredentials { + + private final String accessKeyId; + private final String secretAccessKey; + private final String sessionToken; + private final String region; + private final long expirationEpochMillis; + + public CometCredentials( + String accessKeyId, + String secretAccessKey, + String sessionToken, + String region, + long expirationEpochMillis) { + this.accessKeyId = Objects.requireNonNull(accessKeyId, "accessKeyId"); + this.secretAccessKey = Objects.requireNonNull(secretAccessKey, "secretAccessKey"); + this.sessionToken = sessionToken; + this.region = region; + this.expirationEpochMillis = expirationEpochMillis; + } + + public String getAccessKeyId() { + return accessKeyId; + } + + public String getSecretAccessKey() { + return secretAccessKey; + } + + public String getSessionToken() { + return sessionToken; + } + + public String getRegion() { + return region; + } + + public long getExpirationEpochMillis() { + return expirationEpochMillis; + } +} diff --git a/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java b/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java new file mode 100644 index 0000000000..e55dc6b527 --- /dev/null +++ b/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java @@ -0,0 +1,93 @@ +/* + * 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.comet.cloud; + +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class CometCloudCredentialDispatcherTest { + + @Before + public void resetTestProvider() { + TestCometCloudCredentialProvider.reset(); + } + + @Test + public void providerIsRegisteredFromTestClasspath() { + assertTrue(CometCloudCredentialDispatcher.isProviderRegistered()); + } + + @Test + public void getCredentialsRoundTripsThroughProvider() throws Exception { + CometCredentials creds = + CometCloudCredentialDispatcher.getCredentialsForPath("my-bucket", "path/to/object"); + + assertNotNull(creds); + assertEquals("AKIATEST", creds.getAccessKeyId()); + assertEquals("secret", creds.getSecretAccessKey()); + assertEquals("session-tok", creds.getSessionToken()); + assertEquals("us-east-1", creds.getRegion()); + assertEquals(0L, creds.getExpirationEpochMillis()); + + assertEquals(1, TestCometCloudCredentialProvider.callCount.get()); + assertEquals("my-bucket", TestCometCloudCredentialProvider.lastBucket); + assertEquals("path/to/object", TestCometCloudCredentialProvider.lastPath); + } + + @Test + public void providerExceptionsPropagate() { + IllegalStateException boom = new IllegalStateException("simulated STS failure"); + TestCometCloudCredentialProvider.throwOnNext = boom; + + Exception thrown = + assertThrows( + Exception.class, () -> CometCloudCredentialDispatcher.getCredentialsForPath("b", "k")); + assertSame(boom, thrown); + } + + @Test + public void nullSessionTokenAndRegionAreAllowed() throws Exception { + TestCometCloudCredentialProvider.nextResult = + new CometCredentials("AKIA", "sec", null, null, 0L); + + CometCredentials creds = CometCloudCredentialDispatcher.getCredentialsForPath("b", "k"); + + assertNull(creds.getSessionToken()); + assertNull(creds.getRegion()); + } + + @Test + public void providerReceivesEachCallSeparately() throws Exception { + CometCloudCredentialDispatcher.getCredentialsForPath("b1", "k1"); + CometCloudCredentialDispatcher.getCredentialsForPath("b2", "k2"); + CometCloudCredentialDispatcher.getCredentialsForPath("b3", "k3"); + + assertEquals(3, TestCometCloudCredentialProvider.callCount.get()); + assertEquals("b3", TestCometCloudCredentialProvider.lastBucket); + assertEquals("k3", TestCometCloudCredentialProvider.lastPath); + } +} diff --git a/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java b/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java new file mode 100644 index 0000000000..59776efad3 --- /dev/null +++ b/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java @@ -0,0 +1,58 @@ +/* + * 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.comet.cloud; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Test-only provider registered via {@code META-INF/services} in the test classpath. State is + * static because {@link CometCloudCredentialDispatcher} caches a single instance in a {@code static + * final} field for the JVM lifetime; tests reset state via {@link #reset()}. + */ +public class TestCometCloudCredentialProvider implements CometCloudCredentialProvider { + + static final AtomicInteger callCount = new AtomicInteger(0); + static volatile String lastBucket; + static volatile String lastPath; + static volatile RuntimeException throwOnNext; + static volatile CometCredentials nextResult = + new CometCredentials("AKIATEST", "secret", "session-tok", "us-east-1", 0L); + + static void reset() { + callCount.set(0); + lastBucket = null; + lastPath = null; + throwOnNext = null; + nextResult = new CometCredentials("AKIATEST", "secret", "session-tok", "us-east-1", 0L); + } + + @Override + public CometCredentials getCredentialsForPath(String bucket, String path) { + callCount.incrementAndGet(); + lastBucket = bucket; + lastPath = path; + RuntimeException toThrow = throwOnNext; + if (toThrow != null) { + throwOnNext = null; + throw toThrow; + } + return nextResult; + } +} diff --git a/common/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider b/common/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider new file mode 100644 index 0000000000..fb1c912058 --- /dev/null +++ b/common/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider @@ -0,0 +1 @@ +org.apache.comet.cloud.TestCometCloudCredentialProvider From 892697bca0b41e806ddfd6c9b591c0070e33fa5e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 12 May 2026 18:11:47 -0400 Subject: [PATCH 02/24] cloud credential provider native side --- .../objectstore/comet_credential_bridge.rs | 356 ++++++++++++++++++ native/core/src/parquet/objectstore/mod.rs | 1 + native/core/src/parquet/objectstore/s3.rs | 16 +- 3 files changed, 370 insertions(+), 3 deletions(-) create mode 100644 native/core/src/parquet/objectstore/comet_credential_bridge.rs diff --git a/native/core/src/parquet/objectstore/comet_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_credential_bridge.rs new file mode 100644 index 0000000000..d1f2c1368e --- /dev/null +++ b/native/core/src/parquet/objectstore/comet_credential_bridge.rs @@ -0,0 +1,356 @@ +// 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. + +//! JNI bridge to the Java `CometCloudCredentialDispatcher` SPI for per-request AWS credentials. +//! +//! See `common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java` for the +//! Java side and the architecture diagram. + +use crate::execution::operators::ExecutionError; +use crate::jvm_bridge::{check_exception, JVMClasses}; +use async_trait::async_trait; +use jni::objects::{JClass, JFieldID, JStaticMethodID, JString}; +use jni::signature::{Primitive, ReturnType}; +use jni::strings::JNIString; +use log::debug; +use object_store::aws::AwsCredential; +use object_store::CredentialProvider; +use once_cell::sync::OnceCell; +use std::sync::Arc; + +const DISPATCHER_CLASS: &str = "org/apache/comet/cloud/CometCloudCredentialDispatcher"; +const CREDENTIALS_CLASS: &str = "org/apache/comet/cloud/CometCredentials"; + +/// Process-lifetime cache of the JNI handles needed to call into the dispatcher. +/// +/// ## Why static / process lifetime? +/// +/// `JStaticMethodID` and `JFieldID` are tied to their owning `JClass`; the class itself must +/// remain reachable for the IDs to stay valid. Resolving the class and IDs requires a JNI +/// round-trip, so caching them once per executor avoids per-request overhead. +/// +/// ## Bounded +/// +/// One entry, populated exactly once. `OnceCell` enforces single initialization. +/// +/// ## Credential refresh +/// +/// This cache holds *no* credentials, only the JNI handles needed to invoke the Java provider. +/// Every `get_credential` call dispatches through JNI; the Java provider owns all token / STS +/// refresh logic. There is no Rust-side credential staleness window. +static BRIDGE_HANDLE: OnceCell = OnceCell::new(); + +enum BridgeHandleState { + /// Java dispatcher reported a registered provider; cached handles are usable. + Registered(BridgeHandle), + /// Java dispatcher reported no provider, or initialization could not reach the JVM. + /// Native callers should fall back to the default AWS credential chain. + NotRegistered, +} + +struct BridgeHandle { + /// Used by `get_credential` to invoke the static dispatcher method. + dispatcher_class: JClass<'static>, + /// Kept alive so that the cached field IDs remain valid. + _credentials_class: JClass<'static>, + method_get_credentials: JStaticMethodID, + field_access_key_id: JFieldID, + field_secret_access_key: JFieldID, + field_session_token: JFieldID, +} + +// SAFETY: The cached `JClass`, `JStaticMethodID`, and `JFieldID` are all global identifiers +// that may be used from any thread once acquired. JVMClasses applies the same reasoning to its +// own cached classes/methods. +unsafe impl Send for BridgeHandle {} +unsafe impl Sync for BridgeHandle {} + +/// Attempt to initialize the JNI handles. Returns `Some(handle)` if the dispatcher reports a +/// registered provider, `None` otherwise. Cached after the first call. +fn get_handle() -> Option<&'static BridgeHandle> { + let state = BRIDGE_HANDLE.get_or_init(init_handle); + match state { + BridgeHandleState::Registered(h) => Some(h), + BridgeHandleState::NotRegistered => None, + } +} + +fn init_handle() -> BridgeHandleState { + let result: Result = JVMClasses::with_env(|env| { + // SAFETY: Match the transmute trick in `JVMClasses::init` so that classes acquired here + // can be cached for process lifetime. + let env_static = + unsafe { std::mem::transmute::<&mut jni::Env, &'static mut jni::Env>(env) }; + + let dispatcher_class = env_static + .find_class(JNIString::new(DISPATCHER_CLASS)) + .map_err(|e| { + ExecutionError::GeneralError(format!("Failed to find {DISPATCHER_CLASS}: {e}")) + })?; + + let is_registered_method = env_static + .get_static_method_id( + JNIString::new(DISPATCHER_CLASS), + jni::jni_str!("isProviderRegistered"), + jni::jni_sig!("()Z"), + ) + .map_err(|e| { + ExecutionError::GeneralError(format!("Failed to resolve isProviderRegistered: {e}")) + })?; + + let registered = unsafe { + env_static.call_static_method_unchecked( + &dispatcher_class, + is_registered_method, + ReturnType::Primitive(Primitive::Boolean), + &[], + ) + } + .map_err(|e| { + ExecutionError::GeneralError(format!("isProviderRegistered call failed: {e}")) + })? + .z() + .map_err(|e| { + ExecutionError::GeneralError(format!( + "isProviderRegistered did not return boolean: {e}" + )) + })?; + + if !registered { + debug!("CometCloudCredentialDispatcher reports no registered provider"); + return Ok(BridgeHandleState::NotRegistered); + } + + let method_get_credentials = env_static + .get_static_method_id( + JNIString::new(DISPATCHER_CLASS), + jni::jni_str!("getCredentialsForPath"), + jni::jni_sig!( + "(Ljava/lang/String;Ljava/lang/String;)Lorg/apache/comet/cloud/CometCredentials;" + ), + ) + .map_err(|e| { + ExecutionError::GeneralError(format!( + "Failed to resolve getCredentialsForPath: {e}" + )) + })?; + + let credentials_class = env_static + .find_class(JNIString::new(CREDENTIALS_CLASS)) + .map_err(|e| { + ExecutionError::GeneralError(format!("Failed to find {CREDENTIALS_CLASS}: {e}")) + })?; + + let field_access_key_id = env_static + .get_field_id( + &credentials_class, + jni::jni_str!("accessKeyId"), + jni::jni_sig!("Ljava/lang/String;"), + ) + .map_err(|e| { + ExecutionError::GeneralError(format!("Failed to resolve accessKeyId field: {e}")) + })?; + let field_secret_access_key = env_static + .get_field_id( + &credentials_class, + jni::jni_str!("secretAccessKey"), + jni::jni_sig!("Ljava/lang/String;"), + ) + .map_err(|e| { + ExecutionError::GeneralError(format!( + "Failed to resolve secretAccessKey field: {e}" + )) + })?; + let field_session_token = env_static + .get_field_id( + &credentials_class, + jni::jni_str!("sessionToken"), + jni::jni_sig!("Ljava/lang/String;"), + ) + .map_err(|e| { + ExecutionError::GeneralError(format!("Failed to resolve sessionToken field: {e}")) + })?; + + Ok(BridgeHandleState::Registered(BridgeHandle { + dispatcher_class, + _credentials_class: credentials_class, + method_get_credentials, + field_access_key_id, + field_secret_access_key, + field_session_token, + })) + }); + + match result { + Ok(state) => { + if matches!(state, BridgeHandleState::Registered(_)) { + debug!("CometCredentialBridge initialized; will route credentials through JNI"); + } + state + } + Err(e) => { + // Initialization is best-effort. If the JVM isn't reachable or the dispatcher + // class isn't on the classpath, fall back silently to the default chain. + debug!("CometCredentialBridge unavailable, falling back to default chain: {e}"); + BridgeHandleState::NotRegistered + } + } +} + +/// Returns true if a `CometCloudCredentialProvider` is registered on the JVM classpath. +/// Used by `s3.rs::create_store` to decide whether to construct a [`CometCredentialBridge`]. +pub fn is_provider_registered() -> bool { + get_handle().is_some() +} + +/// Per-request credential provider that delegates to the Java SPI via JNI. +/// +/// One instance is constructed per S3 store (which is itself per-URL in `create_store`), so the +/// `bucket` and `path` fields uniquely identify the access. The Java provider receives both on +/// every credential fetch and is free to return different credentials for different paths. +#[derive(Debug)] +pub struct CometCredentialBridge { + bucket: String, + path: String, +} + +impl CometCredentialBridge { + pub fn new(bucket: impl Into, path: impl Into) -> Self { + Self { + bucket: bucket.into(), + path: path.into(), + } + } +} + +#[async_trait] +impl CredentialProvider for CometCredentialBridge { + type Credential = AwsCredential; + + async fn get_credential(&self) -> object_store::Result> { + let handle = get_handle().ok_or_else(|| object_store::Error::Generic { + store: "S3", + source: "CometCredentialBridge invoked but no Java provider is registered".into(), + })?; + + let bucket = self.bucket.clone(); + let path = self.path.clone(); + + let cred = JVMClasses::with_env(|env| -> Result { + let bucket_jstr = env.new_string(&bucket).map_err(|e| { + ExecutionError::GeneralError(format!("Failed to create bucket JString: {e}")) + })?; + let path_jstr = env.new_string(&path).map_err(|e| { + ExecutionError::GeneralError(format!("Failed to create path JString: {e}")) + })?; + + let result = unsafe { + env.call_static_method_unchecked( + &handle.dispatcher_class, + handle.method_get_credentials, + ReturnType::Object, + &[ + jni::objects::JValue::from(&bucket_jstr).as_jni(), + jni::objects::JValue::from(&path_jstr).as_jni(), + ], + ) + }; + + if let Some(exception) = check_exception(env).map_err(|e| { + ExecutionError::GeneralError(format!("Failed to check Java exception: {e}")) + })? { + return Err(ExecutionError::GeneralError(format!( + "Java exception in CometCloudCredentialDispatcher.getCredentialsForPath: \ + {exception}" + ))); + } + + let creds_obj = result + .map_err(|e| { + ExecutionError::GeneralError(format!( + "getCredentialsForPath JNI call failed: {e}" + )) + })? + .l() + .map_err(|e| { + ExecutionError::GeneralError(format!( + "getCredentialsForPath did not return an object: {e}" + )) + })?; + + if creds_obj.is_null() { + return Err(ExecutionError::GeneralError( + "getCredentialsForPath returned null".to_string(), + )); + } + + let access_key_id = + read_string_field(env, &creds_obj, handle.field_access_key_id, "accessKeyId")? + .ok_or_else(|| { + ExecutionError::GeneralError("accessKeyId was null".to_string()) + })?; + let secret_access_key = read_string_field( + env, + &creds_obj, + handle.field_secret_access_key, + "secretAccessKey", + )? + .ok_or_else(|| ExecutionError::GeneralError("secretAccessKey was null".to_string()))?; + let session_token = + read_string_field(env, &creds_obj, handle.field_session_token, "sessionToken")?; + + Ok(AwsCredential { + key_id: access_key_id, + secret_key: secret_access_key, + token: session_token, + }) + }) + .map_err(|e: ExecutionError| object_store::Error::Generic { + store: "S3", + source: e.to_string().into(), + })?; + + Ok(Arc::new(cred)) + } +} + +/// Read a Java `String` instance field via cached `JFieldID`. Returns `Ok(None)` if the field +/// value is null (allowed for nullable fields like `sessionToken`). +fn read_string_field( + env: &mut jni::Env, + instance: &jni::objects::JObject, + field_id: JFieldID, + field_name: &str, +) -> Result, ExecutionError> { + let value = unsafe { env.get_field_unchecked(instance, field_id, ReturnType::Object) } + .map_err(|e| { + ExecutionError::GeneralError(format!("Failed to read field {field_name}: {e}")) + })? + .l() + .map_err(|e| { + ExecutionError::GeneralError(format!("Field {field_name} was not an Object: {e}")) + })?; + + if value.is_null() { + return Ok(None); + } + let jstr = unsafe { JString::from_raw(env, value.into_raw()) }; + let s = jstr.try_to_string(env).map_err(|e| { + ExecutionError::GeneralError(format!("Failed to read field {field_name} as String: {e}")) + })?; + Ok(Some(s)) +} diff --git a/native/core/src/parquet/objectstore/mod.rs b/native/core/src/parquet/objectstore/mod.rs index bedae08f69..5f7bc3b133 100644 --- a/native/core/src/parquet/objectstore/mod.rs +++ b/native/core/src/parquet/objectstore/mod.rs @@ -15,4 +15,5 @@ // specific language governing permissions and limitations // under the License. +pub mod comet_credential_bridge; pub mod s3; diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index a427ad8ad5..9fd05390d6 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -21,6 +21,7 @@ use std::sync::OnceLock; use url::Url; use crate::execution::jni_api::get_runtime; +use crate::parquet::objectstore::comet_credential_bridge; use async_trait::async_trait; use aws_config::{ ecs::EcsCredentialsProvider, environment::EnvironmentVariableCredentialsProvider, @@ -80,9 +81,18 @@ pub fn create_store( let credential_provider = get_runtime().block_on(build_credential_provider(configs, bucket, min_ttl))?; - builder = match credential_provider { - Some(provider) => builder.with_credentials(Arc::new(provider)), - None => builder.with_skip_signature(true), + builder = if comet_credential_bridge::is_provider_registered() { + // A vendor `CometCloudCredentialProvider` is registered on the JVM classpath. Route + // every credential fetch through it; the Java provider owns any STS / token refresh + // and may return different credentials per S3 path. + debug!("Using CometCredentialBridge for bucket {bucket}"); + let bridge = comet_credential_bridge::CometCredentialBridge::new(bucket, url.path()); + builder.with_credentials(Arc::new(bridge)) + } else { + match credential_provider { + Some(provider) => builder.with_credentials(Arc::new(provider)), + None => builder.with_skip_signature(true), + } }; let s3_configs = extract_s3_config_options(configs, bucket); From 67b6f9b115e9e8df5edb1adc4f34cebf56a2038f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 12 May 2026 18:31:24 -0400 Subject: [PATCH 03/24] hook up iceberg-rust, had to bump iceberg-rust --- native/Cargo.lock | 308 +++++++++++++----- native/Cargo.toml | 5 +- native/core/Cargo.toml | 1 + .../src/execution/operators/iceberg_scan.rs | 25 +- native/core/src/parquet/mod.rs | 2 +- .../objectstore/comet_credential_bridge.rs | 154 +++++++-- 6 files changed, 385 insertions(+), 110 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index df3c3b03c0..7f2317de37 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -674,6 +674,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ec6fb3fe69024a75fa7e1bfb48aa6cf59706a101658ea01bfd33b2b248a038f" dependencies = [ "aws-lc-sys", + "untrusted 0.7.1", "zeroize", ] @@ -1994,7 +1995,7 @@ dependencies = [ "object_store", "object_store_opendal", "once_cell", - "opendal 0.56.0", + "opendal", "parking_lot", "parquet", "paste", @@ -2002,6 +2003,7 @@ dependencies = [ "procfs", "prost", "rand 0.10.1", + "reqsign-core", "reqwest 0.12.28", "serde_json", "tempfile", @@ -3514,7 +3516,6 @@ dependencies = [ "tokio", "tokio-rustls", "tower-service", - "webpki-roots", ] [[package]] @@ -3567,7 +3568,7 @@ dependencies = [ [[package]] name = "iceberg" version = "0.9.0" -source = "git+https://github.com/apache/iceberg-rust?rev=1ad4bfd#1ad4bfd39319508e79960d16dad1b1cdf965c5f4" +source = "git+https://github.com/apache/iceberg-rust?rev=83b4595#83b4595e2b5f522974d24d51c8ecbd09a093fa92" dependencies = [ "aes-gcm", "anyhow", @@ -3622,7 +3623,7 @@ dependencies = [ [[package]] name = "iceberg-storage-opendal" version = "0.9.0" -source = "git+https://github.com/apache/iceberg-rust?rev=1ad4bfd#1ad4bfd39319508e79960d16dad1b1cdf965c5f4" +source = "git+https://github.com/apache/iceberg-rust?rev=83b4595#83b4595e2b5f522974d24d51c8ecbd09a093fa92" dependencies = [ "anyhow", "async-trait", @@ -3630,9 +3631,9 @@ dependencies = [ "cfg-if", "futures", "iceberg", - "opendal 0.55.0", - "reqsign", - "reqwest 0.12.28", + "opendal", + "reqsign-aws-v4", + "reqsign-core", "serde", "typetag", "url", @@ -4028,17 +4029,20 @@ dependencies = [ [[package]] name = "jsonwebtoken" -version = "9.3.1" +version = "10.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a87cc7a48537badeae96744432de36f4be2b4a34a05a5ef32e9dd8a1c169dde" +checksum = "eba32bfb4ffdeaca3e34431072faf01745c9b26d25504aa7a6cf5684334fc4fc" dependencies = [ + "aws-lc-rs", "base64", + "getrandom 0.2.17", "js-sys", "pem", - "ring", "serde", "serde_json", + "signature", "simple_asn1", + "zeroize", ] [[package]] @@ -4569,7 +4573,7 @@ dependencies = [ "futures", "mea", "object_store", - "opendal 0.56.0", + "opendal", "pin-project", "tokio", ] @@ -4598,35 +4602,6 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c08d65885ee38876c4f86fa503fb49d7b507c2b62552df7c70b2fce627e06381" -[[package]] -name = "opendal" -version = "0.55.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d075ab8a203a6ab4bc1bce0a4b9fe486a72bf8b939037f4b78d95386384bc80a" -dependencies = [ - "anyhow", - "backon", - "base64", - "bytes", - "crc32c", - "futures", - "getrandom 0.2.17", - "http 1.4.0", - "http-body 1.0.1", - "jiff", - "log", - "md-5", - "percent-encoding", - "quick-xml 0.38.4", - "reqsign", - "reqwest 0.12.28", - "serde", - "serde_json", - "tokio", - "url", - "uuid", -] - [[package]] name = "opendal" version = "0.56.0" @@ -4639,7 +4614,12 @@ dependencies = [ "opendal-layer-logging", "opendal-layer-retry", "opendal-layer-timeout", + "opendal-service-azdls", + "opendal-service-fs", + "opendal-service-gcs", "opendal-service-hdfs", + "opendal-service-oss", + "opendal-service-s3", ] [[package]] @@ -4713,6 +4693,70 @@ dependencies = [ "tokio", ] +[[package]] +name = "opendal-service-azdls" +version = "0.56.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f9884c2d8cf8ba2bb077d79c877dac5863ba3bab9e2c9c1e41a2e0491404772" +dependencies = [ + "bytes", + "http 1.4.0", + "log", + "opendal-core", + "opendal-service-azure-common", + "quick-xml 0.38.4", + "reqsign-azure-storage", + "reqsign-core", + "reqsign-file-read-tokio", + "serde", + "serde_json", +] + +[[package]] +name = "opendal-service-azure-common" +version = "0.56.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffb0e45d6c8dcf66ce2da20e241bcb80e6e540e109a4ff20f318f6c9b4c54e0c" +dependencies = [ + "http 1.4.0", + "opendal-core", +] + +[[package]] +name = "opendal-service-fs" +version = "0.56.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf0be0417abeeb0053376d816b90fceb9ca98f20dfb54ebf1f2a282729f83663" +dependencies = [ + "bytes", + "log", + "opendal-core", + "serde", + "tokio", + "xattr", +] + +[[package]] +name = "opendal-service-gcs" +version = "0.56.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70a49477a10163431896d106136117f5670717f9c9e49cf6f710528800c6633a" +dependencies = [ + "async-trait", + "bytes", + "http 1.4.0", + "log", + "opendal-core", + "percent-encoding", + "quick-xml 0.38.4", + "reqsign-core", + "reqsign-file-read-tokio", + "reqsign-google", + "serde", + "serde_json", + "tokio", +] + [[package]] name = "opendal-service-hdfs" version = "0.56.0" @@ -4728,6 +4772,44 @@ dependencies = [ "tokio", ] +[[package]] +name = "opendal-service-oss" +version = "0.56.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29c8a917829ad06d21b639558532cb0101fe49b040d946d673a73018683fac05" +dependencies = [ + "bytes", + "http 1.4.0", + "log", + "opendal-core", + "quick-xml 0.38.4", + "reqsign-aliyun-oss", + "reqsign-core", + "reqsign-file-read-tokio", + "serde", +] + +[[package]] +name = "opendal-service-s3" +version = "0.56.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dadddeb9bb50b0d30927dd914c298c4ddca47e4c1cfa7674d311f0cf9b051c8" +dependencies = [ + "base64", + "bytes", + "crc32c", + "http 1.4.0", + "log", + "md-5", + "opendal-core", + "quick-xml 0.38.4", + "reqsign-aws-v4", + "reqsign-core", + "reqsign-file-read-tokio", + "serde", + "url", +] + [[package]] name = "openssl-probe" version = "0.2.1" @@ -5270,16 +5352,6 @@ dependencies = [ "memchr", ] -[[package]] -name = "quick-xml" -version = "0.37.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "331e97a1af0bf59823e6eadffe373d7b27f485be8748f71471c662c1f269b7fb" -dependencies = [ - "memchr", - "serde", -] - [[package]] name = "quick-xml" version = "0.38.4" @@ -5383,7 +5455,6 @@ version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5ca0ecfa931c29007047d1bc58e623ab12e5590e8c7cc53200d5202b69266d8a" dependencies = [ - "libc", "rand_chacha 0.3.1", "rand_core 0.6.4", ] @@ -5538,35 +5609,63 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dc897dd8d9e8bd1ed8cdad82b5966c3e0ecae09fb1907d58efaa013543185d0a" [[package]] -name = "reqsign" -version = "0.16.5" +name = "reqsign-aliyun-oss" +version = "3.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43451dbf3590a7590684c25fb8d12ecdcc90ed3ac123433e500447c7d77ed701" +checksum = "57ac2757f3140aa2e213b554148ae0b52733e624fc6723f0cc6bb3d440176c95" +dependencies = [ + "anyhow", + "form_urlencoded", + "http 1.4.0", + "log", + "percent-encoding", + "reqsign-core", + "rust-ini", + "serde", + "serde_json", +] + +[[package]] +name = "reqsign-aws-v4" +version = "3.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44eaca382e94505a49f1a4849658d153aebf79d9c1a58e5dd3b10361511e9f43" +dependencies = [ + "anyhow", + "bytes", + "form_urlencoded", + "http 1.4.0", + "log", + "percent-encoding", + "quick-xml 0.39.2", + "reqsign-core", + "rust-ini", + "serde", + "serde_json", + "serde_urlencoded", + "sha1", +] + +[[package]] +name = "reqsign-azure-storage" +version = "3.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a321980405d596bd34aaf95c4722a3de4128a67fd19e74a81a83aa3fdf082e6" dependencies = [ "anyhow", - "async-trait", "base64", - "chrono", + "bytes", "form_urlencoded", - "getrandom 0.2.17", - "hex", - "hmac 0.12.1", - "home", "http 1.4.0", "jsonwebtoken", "log", - "once_cell", + "pem", "percent-encoding", - "quick-xml 0.37.5", - "rand 0.8.6", - "reqwest 0.12.28", + "reqsign-core", "rsa", - "rust-ini", "serde", "serde_json", "sha1", - "sha2 0.10.9", - "tokio", ] [[package]] @@ -5591,6 +5690,37 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "reqsign-file-read-tokio" +version = "3.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2d89295b3d17abea31851cc8de55d843d89c52132c864963c38d41920613dc5" +dependencies = [ + "anyhow", + "reqsign-core", + "tokio", +] + +[[package]] +name = "reqsign-google" +version = "3.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35cc609b49c69e76ecaceb775a03f792d1ed3e7755ab3548d4534fd801e3242e" +dependencies = [ + "form_urlencoded", + "http 1.4.0", + "jsonwebtoken", + "log", + "percent-encoding", + "reqsign-aws-v4", + "reqsign-core", + "rsa", + "serde", + "serde_json", + "sha2 0.10.9", + "tokio", +] + [[package]] name = "reqwest" version = "0.12.28" @@ -5631,7 +5761,6 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams 0.4.2", "web-sys", - "webpki-roots", ] [[package]] @@ -5691,7 +5820,7 @@ dependencies = [ "cfg-if", "getrandom 0.2.17", "libc", - "untrusted", + "untrusted 0.9.0", "windows-sys 0.52.0", ] @@ -5862,7 +5991,7 @@ dependencies = [ "aws-lc-rs", "ring", "rustls-pki-types", - "untrusted", + "untrusted 0.9.0", ] [[package]] @@ -6857,6 +6986,12 @@ version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" +[[package]] +name = "untrusted" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" + [[package]] name = "untrusted" version = "0.9.0" @@ -7110,15 +7245,6 @@ dependencies = [ "rustls-pki-types", ] -[[package]] -name = "webpki-roots" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52f5ee44c96cf55f1b349600768e3ece3a8f26010c05265ab73f945bb1a2eb9d" -dependencies = [ - "rustls-pki-types", -] - [[package]] name = "which" version = "4.4.2" @@ -7546,6 +7672,16 @@ version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1ffae5123b2d3fc086436f8834ae3ab053a283cfac8fe0a0b8eaae044768a4c4" +[[package]] +name = "xattr" +version = "1.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32e45ad4206f6d2479085147f02bc2ef834ac85886624a23575ae137c8aa8156" +dependencies = [ + "libc", + "rustix 1.1.4", +] + [[package]] name = "xmlparser" version = "0.13.6" @@ -7621,6 +7757,20 @@ name = "zeroize" version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" +dependencies = [ + "zeroize_derive", +] + +[[package]] +name = "zeroize_derive" +version = "1.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85a5b4158499876c763cb03bc4e49185d3cccbabb15b33c627f7884f43db852e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.117", +] [[package]] name = "zerotrie" diff --git a/native/Cargo.toml b/native/Cargo.toml index d1b5c74af9..c6e07f25e2 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -58,8 +58,9 @@ object_store = { version = "0.13.1", features = ["gcp", "azure", "aws", "http"] url = "2.2" aws-config = "1.8.16" aws-credential-types = "1.2.13" -iceberg = { git = "https://github.com/apache/iceberg-rust", rev = "1ad4bfd" } -iceberg-storage-opendal = { git = "https://github.com/apache/iceberg-rust", rev = "1ad4bfd", features = ["opendal-all"] } +iceberg = { git = "https://github.com/apache/iceberg-rust", rev = "83b4595" } +iceberg-storage-opendal = { git = "https://github.com/apache/iceberg-rust", rev = "83b4595", features = ["opendal-all"] } +reqsign-core = "3" [profile.release] debug = true diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 4fb3ed4c5d..1c8c49d09e 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -75,6 +75,7 @@ hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3"]} opendal = { version = "0.56.0", optional = true, features = ["services-hdfs"] } iceberg = { workspace = true } iceberg-storage-opendal = { workspace = true } +reqsign-core = { workspace = true } serde_json = "1.0" uuid = "1.23.0" diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 55bcbef349..eb19623ca2 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -43,11 +43,13 @@ use iceberg::io::{FileIO, FileIOBuilder, StorageFactory}; use iceberg_storage_opendal::OpenDalStorageFactory; use crate::execution::operators::ExecutionError; +use crate::parquet::objectstore::comet_credential_bridge::{self, CometCredentialBridge}; use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkPhysicalExprAdapterFactory; use datafusion_comet_spark_expr::EvalMode; use datafusion_physical_expr_adapter::{PhysicalExprAdapter, PhysicalExprAdapterFactory}; use iceberg::scan::FileScanTask; +use iceberg_storage_opendal::CustomAwsCredentialLoader; /// Iceberg table scan operator that uses iceberg-rust to read Iceberg tables. /// @@ -207,9 +209,12 @@ impl IcebergScanExec { }; match scheme { "file" => Ok(Arc::new(OpenDalStorageFactory::Fs)), - "s3" | "s3a" => Ok(Arc::new(OpenDalStorageFactory::S3 { - customized_credential_load: None, - })), + "s3" | "s3a" => { + let customized_credential_load = build_s3_credential_loader(path); + Ok(Arc::new(OpenDalStorageFactory::S3 { + customized_credential_load, + })) + } "gs" => Ok(Arc::new(OpenDalStorageFactory::Gcs)), "oss" => Ok(Arc::new(OpenDalStorageFactory::Oss)), _ => Err(DataFusionError::Execution(format!( @@ -233,6 +238,20 @@ impl IcebergScanExec { } } +/// Build a `CustomAwsCredentialLoader` from the registered Comet cloud credential provider, if +/// any. Returns `None` when no Java provider is registered, when the metadata URL is malformed, +/// or when no bucket can be extracted - in those cases opendal falls back to its default +/// credential resolution. +fn build_s3_credential_loader(metadata_location: &str) -> Option { + if !comet_credential_bridge::is_provider_registered() { + return None; + } + let url = url::Url::parse(metadata_location).ok()?; + let bucket = url.host_str()?.to_string(); + let bridge = CometCredentialBridge::new(bucket, metadata_location.to_string()); + Some(CustomAwsCredentialLoader::new(bridge)) +} + /// Metrics for IcebergScanExec struct IcebergScanMetrics { /// Baseline metrics (output rows, elapsed compute time) diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs index 5de14aa610..06b7190fb0 100644 --- a/native/core/src/parquet/mod.rs +++ b/native/core/src/parquet/mod.rs @@ -29,7 +29,7 @@ pub mod read; pub mod schema_adapter; mod cast_column; -mod objectstore; +pub(crate) mod objectstore; use std::collections::HashMap; use std::task::Poll; diff --git a/native/core/src/parquet/objectstore/comet_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_credential_bridge.rs index d1f2c1368e..93f6e6e8c0 100644 --- a/native/core/src/parquet/objectstore/comet_credential_bridge.rs +++ b/native/core/src/parquet/objectstore/comet_credential_bridge.rs @@ -23,14 +23,29 @@ use crate::execution::operators::ExecutionError; use crate::jvm_bridge::{check_exception, JVMClasses}; use async_trait::async_trait; +use iceberg_storage_opendal::AwsCredential as IcebergAwsCredential; use jni::objects::{JClass, JFieldID, JStaticMethodID, JString}; use jni::signature::{Primitive, ReturnType}; use jni::strings::JNIString; -use log::debug; +use log::{debug, warn}; use object_store::aws::AwsCredential; use object_store::CredentialProvider; use once_cell::sync::OnceCell; +use reqsign_core::time::Timestamp; +use reqsign_core::{ + Context, Error as ReqsignError, ErrorKind as ReqsignErrorKind, + ProvideCredential as IcebergProvideCredential, +}; use std::sync::Arc; +use std::time::Duration; + +/// Default expiration to attach when the Java provider returns +/// `CometCredentials.expirationEpochMillis == 0` ("unknown"). Without a non-None expiry, opendal +/// would cache the credential for the entire executor lifetime - a silent footgun for Spark +/// jobs that run for hours. Five minutes is short enough to limit blast radius and long enough +/// to avoid per-request JNI overhead. Vendors that want a different cadence should set +/// `expirationEpochMillis` on every returned POJO. +const DEFAULT_EXPIRY_WHEN_UNKNOWN: Duration = Duration::from_secs(300); const DISPATCHER_CLASS: &str = "org/apache/comet/cloud/CometCloudCredentialDispatcher"; const CREDENTIALS_CLASS: &str = "org/apache/comet/cloud/CometCredentials"; @@ -71,6 +86,7 @@ struct BridgeHandle { field_access_key_id: JFieldID, field_secret_access_key: JFieldID, field_session_token: JFieldID, + field_expiration_epoch_millis: JFieldID, } // SAFETY: The cached `JClass`, `JStaticMethodID`, and `JFieldID` are all global identifiers @@ -184,6 +200,17 @@ fn init_handle() -> BridgeHandleState { .map_err(|e| { ExecutionError::GeneralError(format!("Failed to resolve sessionToken field: {e}")) })?; + let field_expiration_epoch_millis = env_static + .get_field_id( + &credentials_class, + jni::jni_str!("expirationEpochMillis"), + jni::jni_sig!("J"), + ) + .map_err(|e| { + ExecutionError::GeneralError(format!( + "Failed to resolve expirationEpochMillis field: {e}" + )) + })?; Ok(BridgeHandleState::Registered(BridgeHandle { dispatcher_class, @@ -192,6 +219,7 @@ fn init_handle() -> BridgeHandleState { field_access_key_id, field_secret_access_key, field_session_token, + field_expiration_epoch_millis, })) }); @@ -219,9 +247,11 @@ pub fn is_provider_registered() -> bool { /// Per-request credential provider that delegates to the Java SPI via JNI. /// -/// One instance is constructed per S3 store (which is itself per-URL in `create_store`), so the -/// `bucket` and `path` fields uniquely identify the access. The Java provider receives both on -/// every credential fetch and is free to return different credentials for different paths. +/// One instance is constructed per S3 store for the `object_store` path (per-URL in +/// `s3.rs::create_store`) or per FileIO for the iceberg-rust path. The Java provider receives +/// `(bucket, path)` on every credential fetch and is free to return different credentials per +/// path; on the iceberg-rust path the path is the table's metadata location, so credentials are +/// effectively per-table. #[derive(Debug)] pub struct CometCredentialBridge { bucket: String, @@ -235,26 +265,22 @@ impl CometCredentialBridge { path: path.into(), } } -} -#[async_trait] -impl CredentialProvider for CometCredentialBridge { - type Credential = AwsCredential; - - async fn get_credential(&self) -> object_store::Result> { - let handle = get_handle().ok_or_else(|| object_store::Error::Generic { - store: "S3", - source: "CometCredentialBridge invoked but no Java provider is registered".into(), + /// Invoke the Java dispatcher and extract the three string fields off the returned POJO. + /// Shared between the `object_store::CredentialProvider` and `reqsign_core::ProvideCredential` + /// impls. + fn fetch_raw(&self) -> Result { + let handle = get_handle().ok_or_else(|| { + ExecutionError::GeneralError( + "CometCredentialBridge invoked but no Java provider is registered".to_string(), + ) })?; - let bucket = self.bucket.clone(); - let path = self.path.clone(); - - let cred = JVMClasses::with_env(|env| -> Result { - let bucket_jstr = env.new_string(&bucket).map_err(|e| { + JVMClasses::with_env(|env| -> Result { + let bucket_jstr = env.new_string(&self.bucket).map_err(|e| { ExecutionError::GeneralError(format!("Failed to create bucket JString: {e}")) })?; - let path_jstr = env.new_string(&path).map_err(|e| { + let path_jstr = env.new_string(&self.path).map_err(|e| { ExecutionError::GeneralError(format!("Failed to create path JString: {e}")) })?; @@ -312,19 +338,97 @@ impl CredentialProvider for CometCredentialBridge { .ok_or_else(|| ExecutionError::GeneralError("secretAccessKey was null".to_string()))?; let session_token = read_string_field(env, &creds_obj, handle.field_session_token, "sessionToken")?; + let expiration_epoch_millis = unsafe { + env.get_field_unchecked( + &creds_obj, + handle.field_expiration_epoch_millis, + ReturnType::Primitive(Primitive::Long), + ) + } + .map_err(|e| { + ExecutionError::GeneralError(format!("Failed to read expirationEpochMillis: {e}")) + })? + .j() + .map_err(|e| { + ExecutionError::GeneralError(format!("expirationEpochMillis was not a long: {e}")) + })?; - Ok(AwsCredential { - key_id: access_key_id, - secret_key: secret_access_key, - token: session_token, + Ok(RawCredentials { + access_key_id, + secret_access_key, + session_token, + expiration_epoch_millis, }) }) - .map_err(|e: ExecutionError| object_store::Error::Generic { + } +} + +struct RawCredentials { + access_key_id: String, + secret_access_key: String, + session_token: Option, + /// Provider-supplied absolute expiry. `0` means "unknown"; callers translate that into a + /// short fallback so opendal cannot cache the credential for the entire executor lifetime. + expiration_epoch_millis: i64, +} + +#[async_trait] +impl CredentialProvider for CometCredentialBridge { + type Credential = AwsCredential; + + async fn get_credential(&self) -> object_store::Result> { + let raw = self.fetch_raw().map_err(|e| object_store::Error::Generic { store: "S3", source: e.to_string().into(), })?; + Ok(Arc::new(AwsCredential { + key_id: raw.access_key_id, + secret_key: raw.secret_access_key, + token: raw.session_token, + })) + } +} - Ok(Arc::new(cred)) +impl IcebergProvideCredential for CometCredentialBridge { + type Credential = IcebergAwsCredential; + + async fn provide_credential( + &self, + _ctx: &Context, + ) -> reqsign_core::Result> { + let raw = self + .fetch_raw() + .map_err(|e| ReqsignError::new(ReqsignErrorKind::CredentialInvalid, e.to_string()))?; + + let expires_in = if raw.expiration_epoch_millis > 0 { + Some( + Timestamp::from_millisecond(raw.expiration_epoch_millis).map_err(|e| { + ReqsignError::new( + ReqsignErrorKind::CredentialInvalid, + format!( + "Invalid expirationEpochMillis {}: {e}", + raw.expiration_epoch_millis + ), + ) + })?, + ) + } else { + // Provider did not set an expiration. Opendal would otherwise cache this credential + // for the entire executor lifetime; force a refresh after DEFAULT_EXPIRY_WHEN_UNKNOWN. + warn!( + "CometCloudCredentialProvider returned credentials with no expiration; \ + defaulting to {}s expiry to bound opendal caching", + DEFAULT_EXPIRY_WHEN_UNKNOWN.as_secs() + ); + Some(Timestamp::now() + DEFAULT_EXPIRY_WHEN_UNKNOWN) + }; + + Ok(Some(IcebergAwsCredential { + access_key_id: raw.access_key_id, + secret_access_key: raw.secret_access_key, + session_token: raw.session_token, + expires_in, + })) } } From 22e30b0bb5b7090de13a1b2c34e830da00a6923b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 12 May 2026 18:34:57 -0400 Subject: [PATCH 04/24] docs --- .../cloud-credential-providers.md | 200 ++++++++++++++++++ docs/source/contributor-guide/index.md | 1 + .../latest/cloud-credential-providers.md | 120 +++++++++++ docs/source/user-guide/latest/index.rst | 1 + 4 files changed, 322 insertions(+) create mode 100644 docs/source/contributor-guide/cloud-credential-providers.md create mode 100644 docs/source/user-guide/latest/cloud-credential-providers.md diff --git a/docs/source/contributor-guide/cloud-credential-providers.md b/docs/source/contributor-guide/cloud-credential-providers.md new file mode 100644 index 0000000000..5ab7fc4676 --- /dev/null +++ b/docs/source/contributor-guide/cloud-credential-providers.md @@ -0,0 +1,200 @@ + + +# Cloud Credential Providers + +Comet's native S3 readers can route every credential request to a vendor-supplied Java provider +loaded via `java.util.ServiceLoader`. This page is the integration contract for vendors writing +that bridge. + +If you are an operator deciding whether to enable this in your cluster, see the user guide page +on cloud credential providers instead. + +## Why this SPI exists + +Comet runs queries that bypass Spark's Hadoop S3A code path entirely. Native Parquet scans go +through the Rust `object_store` crate directly; native Iceberg scans go through `iceberg-rust` and +`opendal`. Neither path ever calls a Hadoop `Signer` or `AWSCredentialsProvider`. That means none +of the credential infrastructure Spark and Hadoop already configured for your cluster is reachable +from Comet's native code. + +For the simple case (static credentials, EC2 instance profiles, environment variables, the default +AWS credential chain) Comet reproduces that resolution natively in +`native/core/src/parquet/objectstore/s3.rs`. No SPI is needed; the existing default chain works. + +This SPI exists for the case the default chain _cannot_ express: a vendor-managed exchange that +takes a per-request path or token and returns short-lived STS credentials. + +The reasons that case can't be served by an existing API: + +- **`org.apache.spark.deploy.security.cloud.CloudCredentialsProvider`** yields a single auth proof + (typically a JWT) per service name. It has no path argument and no notion of returning AWS + credentials. It is the right place to obtain a JWT, but not to exchange one. +- **Hadoop S3A's custom signer mechanism** keeps path-aware logic inside the + `Signer.sign(request, credentials)` call. The standard `AWSCredentialsProvider.getCredentials()` + contract is parameterless, so vendors that need per-path STS lookup hide that lookup inside the + signer. The credential is never returned outside the AWS SDK's signing pipeline; even running + the signer on a synthesized request does not recover the underlying credential, because the + secret access key is an HMAC key, not a value present in the signed output. +- **Reflecting into vendor singletons** would require Comet to encode a vendor's class name, field + name, lazy-init lifecycle, and method signatures. Each vendor differs; each version may rename + things. That shifts the maintenance burden from the vendor to Comet and creates silent breakage + on upgrades. +- **A Comet-specific HTTP STS endpoint contract** would require vendors to expose and stabilize an + HTTP API just for Comet. Most vendors ship this logic as Java code, not as a public HTTP API, + and asking them to do otherwise is a larger change than a small Java class. + +Comet's SPI is a peer to two contracts vendors with full integration coverage already implement: + +| Path | Vendor implements | +| ----------------- | ----------------------------------------------------------- | +| Hadoop S3A | `org.apache.hadoop.fs.s3a.AwsSignerInitializer` plus signer | +| Iceberg-Java | `org.apache.iceberg.aws.AwsClientFactory` | +| Comet (this page) | `org.apache.comet.cloud.CometCloudCredentialProvider` | + +Adding a Comet implementation is the same shape as the first two with a smaller surface (one +method). + +## SPI contract + +Implement `org.apache.comet.cloud.CometCloudCredentialProvider`: + +```java +package org.apache.comet.cloud; + +public interface CometCloudCredentialProvider { + CometCredentials getCredentialsForPath(String bucket, String path) throws Exception; +} +``` + +`getCredentialsForPath` may be invoked concurrently from many native tokio worker threads. +Implementations must be thread-safe. + +The returned `CometCredentials` POJO carries: + +| Field | Type | Notes | +| ----------------------- | ------------------- | ------------------------------------------------------ | +| `accessKeyId` | `String` (non-null) | Required. | +| `secretAccessKey` | `String` (non-null) | Required. | +| `sessionToken` | `String` (nullable) | Pass `null` for non-STS credentials. | +| `region` | `String` (nullable) | `null` lets the native reader fall back to its config. | +| `expirationEpochMillis` | `long` | `0` means "unknown" (see expiration semantics below). | + +### Expiration semantics + +`expirationEpochMillis` is the absolute expiry of the returned credential, in milliseconds since +the Unix epoch. + +- For credentials with a known expiry, set this to the actual expiry. The Iceberg path uses it to + decide when `opendal` must re-call your provider for a fresh credential. +- `0` is treated as "unknown". The Iceberg path then defaults to a 5-minute refresh interval to + bound how long opendal can cache a possibly-stale credential. Spark jobs running for hours or + days with cached stale credentials would otherwise fail silently mid-task; the 5-minute floor + is a safety net, not a recommendation. **Provide a real expiry whenever you have one.** +- The `object_store` path (raw `s3://` Parquet scans) ignores expiration today and re-fetches per + request. + +### Error semantics + +- Throwing from `getCredentialsForPath` aborts the native S3 request and surfaces the exception + message (and chained causes) to the caller. +- Returning `null` is reserved for "this provider does not handle this path"; the native caller + treats it as an authorization failure rather than falling back to other providers. + +## Discovery + +Discovery is purely classpath-based. Register your implementation by adding a service file: + +``` +META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider +``` + +containing the fully-qualified name of your implementation class. + +Resolution rules at JVM startup: + +- **Zero impls registered** — the native readers fall through to the existing AWS credential chain + (the same behavior as a Comet without your jar on the classpath). +- **Exactly one impl registered** — cached and used for every credential request. +- **Multiple impls registered** — `CometCloudCredentialDispatcher` throws + `IllegalStateException` at class-load with the FQN of every impl found. The executor fails + loudly. This is intentional; pick one bridge jar. + +There is no Comet-specific config knob for selecting a provider. Discovery follows the same source +of truth Spark itself reads, so a query that falls back from Comet to Spark mid-execution sees +identical credentials. + +## Threading and lifecycle + +- The dispatcher caches the resolved provider in a `static final` field for the JVM lifetime. Your + provider is constructed once per executor and reused for every request. +- The dispatcher itself caches no credentials. Every native call dispatches through JNI on a tokio + worker thread, so any STS / token refresh logic must live inside your provider. +- Implementations should be cheap to construct (no long blocking work in the no-arg constructor) + and thread-safe. + +## Spark and AWS SDK version selection + +Vendors that need to pick between Spark 3 and Spark 4 implementations, or between AWS SDK v1 and +v2 backends, do so inside their own provider class. Comet is unaware of either. Common patterns +include reflectively probing for an SDK class on the classpath, or shipping two service files (one +per Spark profile) that each register a different provider class. + +## Worked example + +A minimal static-credential provider, suitable for tests and development: + +```java +package com.example.comet.test; + +import org.apache.comet.cloud.CometCloudCredentialProvider; +import org.apache.comet.cloud.CometCredentials; + +public final class StaticCometCredentialProvider implements CometCloudCredentialProvider { + private static final String ACCESS_KEY = System.getenv("EXAMPLE_ACCESS_KEY"); + private static final String SECRET_KEY = System.getenv("EXAMPLE_SECRET_KEY"); + private static final String REGION = System.getenv().getOrDefault("EXAMPLE_REGION", "us-east-1"); + + @Override + public CometCredentials getCredentialsForPath(String bucket, String path) { + return new CometCredentials(ACCESS_KEY, SECRET_KEY, null, REGION, 0L); + } +} +``` + +Register it via `META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider`: + +``` +com.example.comet.test.StaticCometCredentialProvider +``` + +A real provider would read a JWT from `SparkConf` (typically populated by a Spark +`CloudCredentialsProvider`), call its STS-vending service with the `(bucket, path)` tuple, and +return the resulting credentials with their actual expiry. + +## Where this lives in Comet + +| Component | Location | +| --------------------------------------------------------- | --------------------------------------------------------------------------------- | +| SPI interface | `common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java` | +| POJO | `common/src/main/java/org/apache/comet/cloud/CometCredentials.java` | +| Dispatcher (called from native via JNI) | `common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java` | +| Rust JNI bridge and `object_store::CredentialProvider` | `native/core/src/parquet/objectstore/comet_credential_bridge.rs` | +| `s3.rs` injection point (DataSourceExec / Parquet scans) | `native/core/src/parquet/objectstore/s3.rs` | +| Iceberg scan injection point (`iceberg-rust` + `opendal`) | `native/core/src/execution/operators/iceberg_scan.rs` | diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index 77c73d68da..e32a8c438d 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -46,6 +46,7 @@ ANSI Error Propagation Benchmarking Guide Adding a New Operator Adding a New Expression +Cloud Credential Providers Supported Spark Expressions Tracing Profiling diff --git a/docs/source/user-guide/latest/cloud-credential-providers.md b/docs/source/user-guide/latest/cloud-credential-providers.md new file mode 100644 index 0000000000..51f69017f1 --- /dev/null +++ b/docs/source/user-guide/latest/cloud-credential-providers.md @@ -0,0 +1,120 @@ + + +# Cloud Credential Providers + +Comet's native S3 readers normally fetch credentials from the standard AWS credential chain +(static keys, instance profiles, environment variables, etc.). Some clusters use a vendor-managed +mechanism instead, where credentials are issued per request based on a JWT or per S3 path. For +those clusters, Comet supports loading a vendor-supplied bridge JAR that routes every native +credential request through the vendor's Java code. + +## Do I need this? + +You don't, if any of the following describe your cluster: + +- You use static AWS credentials (`fs.s3a.access.key` / `fs.s3a.secret.key`). +- You use EC2 instance profiles, EKS pod identities, ECS task roles, or environment variables. +- Your S3 access works in Spark today via the default AWS credential chain. + +You probably do, if any of these are true: + +- You have a Hadoop S3A custom signer configured (`fs.s3a.custom.signers=...`). +- You have a Spark `CloudCredentialsProvider` configured + (`spark.security.credentials.providerList=...`) that issues a JWT for your cluster's STS + service. +- You have a custom Iceberg `client.factory` that injects a configured S3 client. +- Spark queries against your S3 paths work, but the same queries with Comet enabled fail. + +## How to tell + +If you suspect you need this, two symptoms confirm it: + +- **Comet S3 reads fail with 403 AccessDenied** while the same query without Comet succeeds. +- **Comet falls back to Spark execution** silently for scans against the affected paths. + +If both Spark and Comet succeed against your S3 paths, you do not need a bridge JAR. + +## How to enable it + +Add the vendor-supplied bridge JAR to your Spark executor classpath: + +```sh +spark-submit \ + --jars vendor-comet-bridge.jar \ + ... +``` + +Or via `spark.jars` in your `spark-defaults.conf`: + +``` +spark.jars=/path/to/vendor-comet-bridge.jar +``` + +That is the entire enablement step. There are no Spark or Comet config keys to set. Comet +discovers the bridge through `META-INF/services` on the classpath at executor startup. + +## Where to get the bridge JAR + +From the same vendor that supplies your Hadoop S3A signer or Iceberg client factory. OSS Comet +deliberately ships no vendor-specific bridges; the SPI is a contract, not a built-in. If your +authorization vendor does not yet provide a Comet bridge, refer them to the contributor guide page +on cloud credential providers. + +## Verification + +When the bridge is on the executor classpath and being used you should see, in the executor +logs: + +- At startup: + `Registered CometCloudCredentialProvider: ` +- The first time Comet performs an S3 access: + `Fetching credentials for bucket=<...> path=<...>` (at debug level) + +If neither log line appears and you expected the bridge to be in use, the JAR is missing from the +executor classpath or its `META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider` +entry is missing. + +When the bridge is _not_ registered (the default), you will see exactly one line at startup: + +``` +No CometCloudCredentialProvider registered; native S3 readers will use the default AWS +credential chain +``` + +This is the expected behavior for clusters that do not need the bridge. + +## Troubleshooting + +**`Multiple CometCloudCredentialProvider impls on classpath: [...]`** at executor startup. Two or +more bridge JARs were found. Remove all but one. Comet does not chain providers; it fails fast to +prevent silent ambiguity. + +**`No CometCloudCredentialProvider registered`** combined with `403 AccessDenied`. The bridge JAR +is not on the executor classpath. Re-check `--jars` / `spark.jars`. On YARN or Kubernetes, confirm +the JAR was actually shipped to the executor pods and not only available on the driver. + +**Slow first query, fast subsequent queries.** Expected. The vendor's STS-vending service is +likely cold-started; subsequent fetches reuse cached credentials inside the vendor's provider. + +**Credentials silently going stale during long-running jobs.** The bridge defaults to a 5-minute +maximum cache window in the Iceberg / opendal path when the vendor does not specify a credential +expiry. If you suspect the vendor is not setting an expiry, ask them to populate +`expirationEpochMillis` on every returned credential. The 5-minute floor is a safety net, not a +configurable knob. diff --git a/docs/source/user-guide/latest/index.rst b/docs/source/user-guide/latest/index.rst index 314a0a51bd..bbacdbe0b8 100644 --- a/docs/source/user-guide/latest/index.rst +++ b/docs/source/user-guide/latest/index.rst @@ -49,4 +49,5 @@ to read more. Tuning Guide Metrics Guide Iceberg Guide + Cloud Credential Providers Kubernetes Guide From 58b13641f9ba896d070d3e01492c7b61a10cc274 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 12 May 2026 20:43:10 -0400 Subject: [PATCH 05/24] tests --- .../cloud/CometCloudCredentialDispatcher.java | 3 +- dev/ci/check-suites.py | 1 + .../cloud-credential-providers.md | 44 ++++++ pom.xml | 19 ++- .../cloud/MinioCometCredentialProvider.java | 87 ++++++++++++ ...e.comet.cloud.CometCloudCredentialProvider | 1 + .../org/apache/comet/CometS3TestBase.scala | 8 ++ .../apache/comet/IcebergReadFromS3Suite.scala | 9 ++ .../CometCloudCredentialBridgeS3Suite.scala | 132 ++++++++++++++++++ 9 files changed, 301 insertions(+), 3 deletions(-) create mode 100644 spark/src/test/java/org/apache/comet/cloud/MinioCometCredentialProvider.java create mode 100644 spark/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider create mode 100644 spark/src/test/scala/org/apache/comet/cloud/CometCloudCredentialBridgeS3Suite.scala diff --git a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java index 3f9241c16a..30c6745e45 100644 --- a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java +++ b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java @@ -99,8 +99,7 @@ */ public final class CometCloudCredentialDispatcher { - private static final Logger LOG = - LoggerFactory.getLogger(CometCloudCredentialDispatcher.class); + private static final Logger LOG = LoggerFactory.getLogger(CometCloudCredentialDispatcher.class); /* * Process-lifetime singleton, justified per the contributor guide's "Global singletons" diff --git a/dev/ci/check-suites.py b/dev/ci/check-suites.py index 279c6a89c9..abc67280db 100644 --- a/dev/ci/check-suites.py +++ b/dev/ci/check-suites.py @@ -35,6 +35,7 @@ def file_to_class_name(path: Path) -> str | None: "org.apache.comet.parquet.ParquetReadSuite", # abstract "org.apache.comet.parquet.ParquetReadFromS3Suite", # manual test suite "org.apache.comet.IcebergReadFromS3Suite", # manual test suite + "org.apache.comet.cloud.CometCloudCredentialBridgeS3Suite", # manual test suite "org.apache.spark.sql.comet.CometPlanStabilitySuite", # abstract "org.apache.spark.sql.comet.ParquetDatetimeRebaseSuite", # abstract "org.apache.comet.exec.CometColumnarShuffleSuite" # abstract diff --git a/docs/source/contributor-guide/cloud-credential-providers.md b/docs/source/contributor-guide/cloud-credential-providers.md index 5ab7fc4676..04e6269c96 100644 --- a/docs/source/contributor-guide/cloud-credential-providers.md +++ b/docs/source/contributor-guide/cloud-credential-providers.md @@ -117,6 +117,24 @@ the Unix epoch. - Returning `null` is reserved for "this provider does not handle this path"; the native caller treats it as an authorization failure rather than falling back to other providers. +### Iceberg path: explicit S3 region required + +When the bridge is registered, Comet wires it into `iceberg-storage-opendal` as a custom +`AwsCredentialLoad`. opendal then requires explicit S3 region (and, for non-AWS endpoints, an +explicit endpoint) on the catalog properties — its built-in region auto-detection only runs +when no custom credential loader is configured. + +In practice this means deployments using the bridge for Iceberg must set, on the Spark catalog: + +``` +spark.sql.catalog..s3.region = us-east-1 (or your real region) +spark.sql.catalog..s3.endpoint = https://... (only for non-AWS) +spark.sql.catalog..s3.path-style-access = true (only for path-style endpoints) +``` + +If a deployment hits `region is missing. Please find it by S3::detect_region() or set them +in env`, this is the missing config. + ## Discovery Discovery is purely classpath-based. Register your implementation by adding a service file: @@ -156,6 +174,32 @@ v2 backends, do so inside their own provider class. Comet is unaware of either. include reflectively probing for an SDK class on the classpath, or shipping two service files (one per Spark profile) that each register a different provider class. +## Build setup + +Vendor implementations need the Comet SPI classes at compile time only. The standard pattern is +a `provided`-scope Maven dependency on `comet-common`: + +```xml + + org.apache.datafusion + comet-common-spark${spark.version.short}_${scala.binary.version} + ${comet.version} + provided + +``` + +`provided` scope means: + +- Compile-time only — your jar resolves the `CometCloudCredentialProvider` interface and + `CometCredentials` POJO for compilation. +- No runtime bundling — your jar does not ship Comet classes; no fat-jar bloat, no version + conflict on the executor classpath. +- The API classes are already present at runtime because Comet itself is loaded (that's the + whole reason the bridge exists). + +Same pattern vendors already use for implementing Hadoop S3A `AwsSignerInitializer` and Iceberg +`AwsClientFactory` — Comet is a third entry in the same list, not a different shape. + ## Worked example A minimal static-credential provider, suitable for tests and development: diff --git a/pom.xml b/pom.xml index 7419fecc92..1003fa51df 100644 --- a/pom.xml +++ b/pom.xml @@ -83,7 +83,7 @@ under the License. 4.13.6 2.0.17 33.2.1-jre - 1.21.0 + 1.21.4 2.31.51 ${project.basedir}/../native/target/debug darwin @@ -476,6 +476,23 @@ under the License. ${testcontainers.version} test + + + com.github.docker-java + docker-java-api + 3.7.1 + test + + + com.github.docker-java + docker-java-transport-zerodep + 3.7.1 + test + com.github.docker-java From 92b04167e88b602d6de774c2187f2d587a9595af Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 12 May 2026 21:14:21 -0400 Subject: [PATCH 07/24] cleanup --- .../cloud/CometCloudCredentialDispatcher.java | 16 +- .../src/execution/operators/iceberg_scan.rs | 6 +- .../objectstore/comet_credential_bridge.rs | 430 ++++++------------ native/core/src/parquet/objectstore/s3.rs | 3 - .../src/comet_cloud_credential_dispatcher.rs | 90 ++++ native/jni-bridge/src/lib.rs | 8 + .../org/apache/comet/CometS3TestBase.scala | 21 + .../apache/comet/IcebergReadFromS3Suite.scala | 17 +- .../CometCloudCredentialBridgeS3Suite.scala | 23 +- 9 files changed, 268 insertions(+), 346 deletions(-) create mode 100644 native/jni-bridge/src/comet_cloud_credential_dispatcher.rs diff --git a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java index 7e04e4b07a..b6885969fe 100644 --- a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java +++ b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java @@ -20,9 +20,9 @@ package org.apache.comet.cloud; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.ServiceLoader; +import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,7 +121,6 @@ public final class CometCloudCredentialDispatcher { private CometCloudCredentialDispatcher() {} - /** Returns true if a provider was discovered on the classpath. */ public static boolean isProviderRegistered() { return PROVIDER != null; } @@ -146,10 +145,9 @@ public static CometCredentials getCredentialsForPath(String bucket, String path) private static CometCloudCredentialProvider resolve() { List impls = new ArrayList<>(); - Iterator it = - ServiceLoader.load(CometCloudCredentialProvider.class).iterator(); - while (it.hasNext()) { - impls.add(it.next()); + for (CometCloudCredentialProvider impl : + ServiceLoader.load(CometCloudCredentialProvider.class)) { + impls.add(impl); } if (impls.isEmpty()) { LOG.info( @@ -158,10 +156,8 @@ private static CometCloudCredentialProvider resolve() { return null; } if (impls.size() > 1) { - List names = new ArrayList<>(impls.size()); - for (CometCloudCredentialProvider impl : impls) { - names.add(impl.getClass().getName()); - } + List names = + impls.stream().map(p -> p.getClass().getName()).collect(Collectors.toList()); LOG.error("Multiple CometCloudCredentialProvider impls on classpath: {}", names); throw new IllegalStateException( "Multiple CometCloudCredentialProvider impls on classpath: " + names); diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index eb19623ca2..77e8e64c3c 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -238,10 +238,8 @@ impl IcebergScanExec { } } -/// Build a `CustomAwsCredentialLoader` from the registered Comet cloud credential provider, if -/// any. Returns `None` when no Java provider is registered, when the metadata URL is malformed, -/// or when no bucket can be extracted - in those cases opendal falls back to its default -/// credential resolution. +/// Wires the registered Comet credential provider into opendal's S3 service for this scan, or +/// returns `None` so opendal falls back to its default credential chain. fn build_s3_credential_loader(metadata_location: &str) -> Option { if !comet_credential_bridge::is_provider_registered() { return None; diff --git a/native/core/src/parquet/objectstore/comet_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_credential_bridge.rs index 93f6e6e8c0..06fafe721c 100644 --- a/native/core/src/parquet/objectstore/comet_credential_bridge.rs +++ b/native/core/src/parquet/objectstore/comet_credential_bridge.rs @@ -18,15 +18,15 @@ //! JNI bridge to the Java `CometCloudCredentialDispatcher` SPI for per-request AWS credentials. //! //! See `common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java` for the -//! Java side and the architecture diagram. +//! Java side and the architecture diagram. JNI handles are cached on `JVMClasses` next to all +//! the other Comet JNI bridges; this file holds only the Rust trait impls that delegate through. use crate::execution::operators::ExecutionError; use crate::jvm_bridge::{check_exception, JVMClasses}; use async_trait::async_trait; use iceberg_storage_opendal::AwsCredential as IcebergAwsCredential; -use jni::objects::{JClass, JFieldID, JStaticMethodID, JString}; +use jni::objects::{JFieldID, JObject, JString, JValue}; use jni::signature::{Primitive, ReturnType}; -use jni::strings::JNIString; use log::{debug, warn}; use object_store::aws::AwsCredential; use object_store::CredentialProvider; @@ -36,226 +36,73 @@ use reqsign_core::{ Context, Error as ReqsignError, ErrorKind as ReqsignErrorKind, ProvideCredential as IcebergProvideCredential, }; +use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::time::Duration; -/// Default expiration to attach when the Java provider returns -/// `CometCredentials.expirationEpochMillis == 0` ("unknown"). Without a non-None expiry, opendal -/// would cache the credential for the entire executor lifetime - a silent footgun for Spark -/// jobs that run for hours. Five minutes is short enough to limit blast radius and long enough -/// to avoid per-request JNI overhead. Vendors that want a different cadence should set -/// `expirationEpochMillis` on every returned POJO. +/// Bound on opendal's credential cache when the Java provider returns `expirationEpochMillis = 0` +/// ("unknown"). Without this, opendal would hold the credential for the entire executor lifetime, +/// a silent footgun for Spark jobs that run for hours. Five minutes trades a small JNI-call +/// cadence for a tight staleness bound. Vendors that know the real expiry should set it. const DEFAULT_EXPIRY_WHEN_UNKNOWN: Duration = Duration::from_secs(300); -const DISPATCHER_CLASS: &str = "org/apache/comet/cloud/CometCloudCredentialDispatcher"; -const CREDENTIALS_CLASS: &str = "org/apache/comet/cloud/CometCredentials"; +/// Cached "is a Java provider registered?" answer. Resolution is one JNI round-trip and the +/// result never changes within a JVM lifetime, so memoize. +static PROVIDER_REGISTERED: OnceCell = OnceCell::new(); -/// Process-lifetime cache of the JNI handles needed to call into the dispatcher. -/// -/// ## Why static / process lifetime? -/// -/// `JStaticMethodID` and `JFieldID` are tied to their owning `JClass`; the class itself must -/// remain reachable for the IDs to stay valid. Resolving the class and IDs requires a JNI -/// round-trip, so caching them once per executor avoids per-request overhead. -/// -/// ## Bounded -/// -/// One entry, populated exactly once. `OnceCell` enforces single initialization. -/// -/// ## Credential refresh -/// -/// This cache holds *no* credentials, only the JNI handles needed to invoke the Java provider. -/// Every `get_credential` call dispatches through JNI; the Java provider owns all token / STS -/// refresh logic. There is no Rust-side credential staleness window. -static BRIDGE_HANDLE: OnceCell = OnceCell::new(); - -enum BridgeHandleState { - /// Java dispatcher reported a registered provider; cached handles are usable. - Registered(BridgeHandle), - /// Java dispatcher reported no provider, or initialization could not reach the JVM. - /// Native callers should fall back to the default AWS credential chain. - NotRegistered, -} - -struct BridgeHandle { - /// Used by `get_credential` to invoke the static dispatcher method. - dispatcher_class: JClass<'static>, - /// Kept alive so that the cached field IDs remain valid. - _credentials_class: JClass<'static>, - method_get_credentials: JStaticMethodID, - field_access_key_id: JFieldID, - field_secret_access_key: JFieldID, - field_session_token: JFieldID, - field_expiration_epoch_millis: JFieldID, -} - -// SAFETY: The cached `JClass`, `JStaticMethodID`, and `JFieldID` are all global identifiers -// that may be used from any thread once acquired. JVMClasses applies the same reasoning to its -// own cached classes/methods. -unsafe impl Send for BridgeHandle {} -unsafe impl Sync for BridgeHandle {} - -/// Attempt to initialize the JNI handles. Returns `Some(handle)` if the dispatcher reports a -/// registered provider, `None` otherwise. Cached after the first call. -fn get_handle() -> Option<&'static BridgeHandle> { - let state = BRIDGE_HANDLE.get_or_init(init_handle); - match state { - BridgeHandleState::Registered(h) => Some(h), - BridgeHandleState::NotRegistered => None, - } -} - -fn init_handle() -> BridgeHandleState { - let result: Result = JVMClasses::with_env(|env| { - // SAFETY: Match the transmute trick in `JVMClasses::init` so that classes acquired here - // can be cached for process lifetime. - let env_static = - unsafe { std::mem::transmute::<&mut jni::Env, &'static mut jni::Env>(env) }; - - let dispatcher_class = env_static - .find_class(JNIString::new(DISPATCHER_CLASS)) - .map_err(|e| { - ExecutionError::GeneralError(format!("Failed to find {DISPATCHER_CLASS}: {e}")) - })?; - - let is_registered_method = env_static - .get_static_method_id( - JNIString::new(DISPATCHER_CLASS), - jni::jni_str!("isProviderRegistered"), - jni::jni_sig!("()Z"), - ) - .map_err(|e| { - ExecutionError::GeneralError(format!("Failed to resolve isProviderRegistered: {e}")) - })?; - - let registered = unsafe { - env_static.call_static_method_unchecked( - &dispatcher_class, - is_registered_method, - ReturnType::Primitive(Primitive::Boolean), - &[], - ) - } - .map_err(|e| { - ExecutionError::GeneralError(format!("isProviderRegistered call failed: {e}")) - })? - .z() - .map_err(|e| { - ExecutionError::GeneralError(format!( - "isProviderRegistered did not return boolean: {e}" - )) - })?; - - if !registered { - debug!("CometCloudCredentialDispatcher reports no registered provider"); - return Ok(BridgeHandleState::NotRegistered); - } - - let method_get_credentials = env_static - .get_static_method_id( - JNIString::new(DISPATCHER_CLASS), - jni::jni_str!("getCredentialsForPath"), - jni::jni_sig!( - "(Ljava/lang/String;Ljava/lang/String;)Lorg/apache/comet/cloud/CometCredentials;" - ), - ) - .map_err(|e| { - ExecutionError::GeneralError(format!( - "Failed to resolve getCredentialsForPath: {e}" - )) - })?; - - let credentials_class = env_static - .find_class(JNIString::new(CREDENTIALS_CLASS)) - .map_err(|e| { - ExecutionError::GeneralError(format!("Failed to find {CREDENTIALS_CLASS}: {e}")) - })?; - - let field_access_key_id = env_static - .get_field_id( - &credentials_class, - jni::jni_str!("accessKeyId"), - jni::jni_sig!("Ljava/lang/String;"), - ) - .map_err(|e| { - ExecutionError::GeneralError(format!("Failed to resolve accessKeyId field: {e}")) - })?; - let field_secret_access_key = env_static - .get_field_id( - &credentials_class, - jni::jni_str!("secretAccessKey"), - jni::jni_sig!("Ljava/lang/String;"), - ) - .map_err(|e| { - ExecutionError::GeneralError(format!( - "Failed to resolve secretAccessKey field: {e}" - )) - })?; - let field_session_token = env_static - .get_field_id( - &credentials_class, - jni::jni_str!("sessionToken"), - jni::jni_sig!("Ljava/lang/String;"), - ) - .map_err(|e| { - ExecutionError::GeneralError(format!("Failed to resolve sessionToken field: {e}")) - })?; - let field_expiration_epoch_millis = env_static - .get_field_id( - &credentials_class, - jni::jni_str!("expirationEpochMillis"), - jni::jni_sig!("J"), - ) +/// True iff a `CometCloudCredentialProvider` was discovered on the JVM classpath. Used by +/// `s3.rs::create_store` and `iceberg_scan.rs` to decide whether to wire a [`CometCredentialBridge`] +/// in front of the default credential paths. +pub fn is_provider_registered() -> bool { + *PROVIDER_REGISTERED.get_or_init(|| { + JVMClasses::with_env(|env| -> Result { + let dispatcher = &JVMClasses::get().comet_cloud_credential_dispatcher; + let result = unsafe { + env.call_static_method_unchecked( + &dispatcher.class, + dispatcher.method_is_provider_registered, + dispatcher.method_is_provider_registered_ret, + &[], + ) + } .map_err(|e| { - ExecutionError::GeneralError(format!( - "Failed to resolve expirationEpochMillis field: {e}" - )) + ExecutionError::GeneralError(format!("isProviderRegistered call failed: {e}")) })?; - - Ok(BridgeHandleState::Registered(BridgeHandle { - dispatcher_class, - _credentials_class: credentials_class, - method_get_credentials, - field_access_key_id, - field_secret_access_key, - field_session_token, - field_expiration_epoch_millis, - })) - }); - - match result { - Ok(state) => { - if matches!(state, BridgeHandleState::Registered(_)) { - debug!("CometCredentialBridge initialized; will route credentials through JNI"); + if let Some(exception) = check_exception(env) + .map_err(|e| ExecutionError::GeneralError(format!("Exception check failed: {e}")))? + { + return Err(ExecutionError::GeneralError(format!( + "Java exception in isProviderRegistered: {exception}" + ))); } - state - } - Err(e) => { - // Initialization is best-effort. If the JVM isn't reachable or the dispatcher - // class isn't on the classpath, fall back silently to the default chain. - debug!("CometCredentialBridge unavailable, falling back to default chain: {e}"); - BridgeHandleState::NotRegistered - } - } -} - -/// Returns true if a `CometCloudCredentialProvider` is registered on the JVM classpath. -/// Used by `s3.rs::create_store` to decide whether to construct a [`CometCredentialBridge`]. -pub fn is_provider_registered() -> bool { - get_handle().is_some() + result.z().map_err(|e| { + ExecutionError::GeneralError(format!("isProviderRegistered did not return Z: {e}")) + }) + }) + .unwrap_or_else(|e| { + debug!( + "CometCloudCredentialDispatcher.isProviderRegistered failed; \ + native S3 readers will use the default AWS credential chain: {e}" + ); + false + }) + }) } /// Per-request credential provider that delegates to the Java SPI via JNI. /// -/// One instance is constructed per S3 store for the `object_store` path (per-URL in -/// `s3.rs::create_store`) or per FileIO for the iceberg-rust path. The Java provider receives -/// `(bucket, path)` on every credential fetch and is free to return different credentials per -/// path; on the iceberg-rust path the path is the table's metadata location, so credentials are -/// effectively per-table. +/// One instance is constructed per S3 store (per-URL in `create_store`) or per FileIO (the +/// metadata location, in `iceberg_scan.rs`). The `(bucket, path)` tuple is forwarded verbatim +/// on every credential fetch; the Java provider is free to return different credentials for +/// different paths. #[derive(Debug)] pub struct CometCredentialBridge { bucket: String, path: String, + /// Latched once the bridge observes a credential without an expiry, so the warning that + /// goes with [`DEFAULT_EXPIRY_WHEN_UNKNOWN`] only fires once per bridge instance instead of + /// per request. + warned_missing_expiry: AtomicBool, } impl CometCredentialBridge { @@ -263,53 +110,45 @@ impl CometCredentialBridge { Self { bucket: bucket.into(), path: path.into(), + warned_missing_expiry: AtomicBool::new(false), } } - /// Invoke the Java dispatcher and extract the three string fields off the returned POJO. - /// Shared between the `object_store::CredentialProvider` and `reqsign_core::ProvideCredential` - /// impls. + /// Single JNI round-trip to the dispatcher; both async trait impls share this. fn fetch_raw(&self) -> Result { - let handle = get_handle().ok_or_else(|| { - ExecutionError::GeneralError( - "CometCredentialBridge invoked but no Java provider is registered".to_string(), - ) - })?; - JVMClasses::with_env(|env| -> Result { - let bucket_jstr = env.new_string(&self.bucket).map_err(|e| { - ExecutionError::GeneralError(format!("Failed to create bucket JString: {e}")) - })?; - let path_jstr = env.new_string(&self.path).map_err(|e| { - ExecutionError::GeneralError(format!("Failed to create path JString: {e}")) - })?; + let dispatcher = &JVMClasses::get().comet_cloud_credential_dispatcher; + + let bucket_jstr = env + .new_string(&self.bucket) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(bucket): {e}")))?; + let path_jstr = env + .new_string(&self.path) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(path): {e}")))?; let result = unsafe { env.call_static_method_unchecked( - &handle.dispatcher_class, - handle.method_get_credentials, - ReturnType::Object, + &dispatcher.class, + dispatcher.method_get_credentials_for_path, + dispatcher.method_get_credentials_for_path_ret, &[ - jni::objects::JValue::from(&bucket_jstr).as_jni(), - jni::objects::JValue::from(&path_jstr).as_jni(), + JValue::from(&bucket_jstr).as_jni(), + JValue::from(&path_jstr).as_jni(), ], ) }; - if let Some(exception) = check_exception(env).map_err(|e| { - ExecutionError::GeneralError(format!("Failed to check Java exception: {e}")) - })? { + if let Some(exception) = check_exception(env) + .map_err(|e| ExecutionError::GeneralError(format!("Exception check failed: {e}")))? + { return Err(ExecutionError::GeneralError(format!( - "Java exception in CometCloudCredentialDispatcher.getCredentialsForPath: \ - {exception}" + "Java exception in getCredentialsForPath: {exception}" ))); } let creds_obj = result .map_err(|e| { - ExecutionError::GeneralError(format!( - "getCredentialsForPath JNI call failed: {e}" - )) + ExecutionError::GeneralError(format!("getCredentialsForPath JNI call: {e}")) })? .l() .map_err(|e| { @@ -320,44 +159,42 @@ impl CometCredentialBridge { if creds_obj.is_null() { return Err(ExecutionError::GeneralError( - "getCredentialsForPath returned null".to_string(), + "getCredentialsForPath returned null (contract violation)".to_string(), )); } - let access_key_id = - read_string_field(env, &creds_obj, handle.field_access_key_id, "accessKeyId")? - .ok_or_else(|| { - ExecutionError::GeneralError("accessKeyId was null".to_string()) - })?; - let secret_access_key = read_string_field( - env, - &creds_obj, - handle.field_secret_access_key, - "secretAccessKey", - )? - .ok_or_else(|| ExecutionError::GeneralError("secretAccessKey was null".to_string()))?; - let session_token = - read_string_field(env, &creds_obj, handle.field_session_token, "sessionToken")?; - let expiration_epoch_millis = unsafe { - env.get_field_unchecked( - &creds_obj, - handle.field_expiration_epoch_millis, - ReturnType::Primitive(Primitive::Long), - ) - } - .map_err(|e| { - ExecutionError::GeneralError(format!("Failed to read expirationEpochMillis: {e}")) - })? - .j() - .map_err(|e| { - ExecutionError::GeneralError(format!("expirationEpochMillis was not a long: {e}")) - })?; - Ok(RawCredentials { - access_key_id, - secret_access_key, - session_token, - expiration_epoch_millis, + access_key_id: read_required_string( + env, + &creds_obj, + dispatcher.field_access_key_id, + "accessKeyId", + )?, + secret_access_key: read_required_string( + env, + &creds_obj, + dispatcher.field_secret_access_key, + "secretAccessKey", + )?, + session_token: read_optional_string( + env, + &creds_obj, + dispatcher.field_session_token, + )?, + expiration_epoch_millis: unsafe { + env.get_field_unchecked( + &creds_obj, + dispatcher.field_expiration_epoch_millis, + ReturnType::Primitive(Primitive::Long), + ) + } + .map_err(|e| { + ExecutionError::GeneralError(format!("read expirationEpochMillis: {e}")) + })? + .j() + .map_err(|e| { + ExecutionError::GeneralError(format!("expirationEpochMillis not a long: {e}")) + })?, }) }) } @@ -367,8 +204,8 @@ struct RawCredentials { access_key_id: String, secret_access_key: String, session_token: Option, - /// Provider-supplied absolute expiry. `0` means "unknown"; callers translate that into a - /// short fallback so opendal cannot cache the credential for the entire executor lifetime. + /// Provider-supplied absolute expiry. `0` means the provider didn't say; callers translate + /// that into a short fallback so opendal can't cache a stale credential indefinitely. expiration_epoch_millis: i64, } @@ -413,13 +250,15 @@ impl IcebergProvideCredential for CometCredentialBridge { })?, ) } else { - // Provider did not set an expiration. Opendal would otherwise cache this credential - // for the entire executor lifetime; force a refresh after DEFAULT_EXPIRY_WHEN_UNKNOWN. - warn!( - "CometCloudCredentialProvider returned credentials with no expiration; \ - defaulting to {}s expiry to bound opendal caching", - DEFAULT_EXPIRY_WHEN_UNKNOWN.as_secs() - ); + if !self.warned_missing_expiry.swap(true, Ordering::Relaxed) { + warn!( + "CometCloudCredentialProvider returned credentials without expiration for \ + bucket={} path={}; defaulting to {}s expiry to bound opendal caching", + self.bucket, + self.path, + DEFAULT_EXPIRY_WHEN_UNKNOWN.as_secs() + ); + } Some(Timestamp::now() + DEFAULT_EXPIRY_WHEN_UNKNOWN) }; @@ -432,29 +271,30 @@ impl IcebergProvideCredential for CometCredentialBridge { } } -/// Read a Java `String` instance field via cached `JFieldID`. Returns `Ok(None)` if the field -/// value is null (allowed for nullable fields like `sessionToken`). -fn read_string_field( +fn read_required_string( + env: &mut jni::Env, + instance: &JObject, + field: JFieldID, + name: &str, +) -> Result { + read_optional_string(env, instance, field)? + .ok_or_else(|| ExecutionError::GeneralError(format!("{name} was null"))) +} + +fn read_optional_string( env: &mut jni::Env, - instance: &jni::objects::JObject, - field_id: JFieldID, - field_name: &str, + instance: &JObject, + field: JFieldID, ) -> Result, ExecutionError> { - let value = unsafe { env.get_field_unchecked(instance, field_id, ReturnType::Object) } - .map_err(|e| { - ExecutionError::GeneralError(format!("Failed to read field {field_name}: {e}")) - })? + let value = unsafe { env.get_field_unchecked(instance, field, ReturnType::Object) } + .map_err(|e| ExecutionError::GeneralError(format!("get_field_unchecked: {e}")))? .l() - .map_err(|e| { - ExecutionError::GeneralError(format!("Field {field_name} was not an Object: {e}")) - })?; - + .map_err(|e| ExecutionError::GeneralError(format!("field was not an Object: {e}")))?; if value.is_null() { return Ok(None); } let jstr = unsafe { JString::from_raw(env, value.into_raw()) }; - let s = jstr.try_to_string(env).map_err(|e| { - ExecutionError::GeneralError(format!("Failed to read field {field_name} as String: {e}")) - })?; - Ok(Some(s)) + jstr.try_to_string(env) + .map(Some) + .map_err(|e| ExecutionError::GeneralError(format!("try_to_string: {e}"))) } diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index 9fd05390d6..da72c8d206 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -82,9 +82,6 @@ pub fn create_store( let credential_provider = get_runtime().block_on(build_credential_provider(configs, bucket, min_ttl))?; builder = if comet_credential_bridge::is_provider_registered() { - // A vendor `CometCloudCredentialProvider` is registered on the JVM classpath. Route - // every credential fetch through it; the Java provider owns any STS / token refresh - // and may return different credentials per S3 path. debug!("Using CometCredentialBridge for bucket {bucket}"); let bridge = comet_credential_bridge::CometCredentialBridge::new(bucket, url.path()); builder.with_credentials(Arc::new(bridge)) diff --git a/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs b/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs new file mode 100644 index 0000000000..629777b316 --- /dev/null +++ b/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs @@ -0,0 +1,90 @@ +// 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. + +use jni::{ + errors::Result as JniResult, + objects::{JClass, JFieldID, JStaticMethodID}, + signature::{Primitive, ReturnType}, + strings::JNIString, + Env, +}; + +/// JNI handles for the JVM `org.apache.comet.cloud.CometCloudCredentialDispatcher` SPI plus the +/// `CometCredentials` POJO whose fields the native bridge reads back. Cached at JVM-init time +/// so the underlying `JClass` references stay alive for the executor lifetime - acquiring them +/// inside a per-call local frame would let them be freed on frame pop. +pub struct CometCloudCredentialDispatcher<'a> { + pub class: JClass<'a>, + pub credentials_class: JClass<'a>, + pub method_is_provider_registered: JStaticMethodID, + pub method_is_provider_registered_ret: ReturnType, + pub method_get_credentials_for_path: JStaticMethodID, + pub method_get_credentials_for_path_ret: ReturnType, + pub field_access_key_id: JFieldID, + pub field_secret_access_key: JFieldID, + pub field_session_token: JFieldID, + pub field_expiration_epoch_millis: JFieldID, +} + +impl<'a> CometCloudCredentialDispatcher<'a> { + pub const JVM_CLASS: &'static str = "org/apache/comet/cloud/CometCloudCredentialDispatcher"; + pub const CREDENTIALS_CLASS: &'static str = "org/apache/comet/cloud/CometCredentials"; + + pub fn new(env: &mut Env<'a>) -> JniResult> { + let class = env.find_class(JNIString::new(Self::JVM_CLASS))?; + let credentials_class = env.find_class(JNIString::new(Self::CREDENTIALS_CLASS))?; + + Ok(CometCloudCredentialDispatcher { + method_is_provider_registered: env.get_static_method_id( + JNIString::new(Self::JVM_CLASS), + jni::jni_str!("isProviderRegistered"), + jni::jni_sig!("()Z"), + )?, + method_is_provider_registered_ret: ReturnType::Primitive(Primitive::Boolean), + method_get_credentials_for_path: env.get_static_method_id( + JNIString::new(Self::JVM_CLASS), + jni::jni_str!("getCredentialsForPath"), + jni::jni_sig!( + "(Ljava/lang/String;Ljava/lang/String;)Lorg/apache/comet/cloud/CometCredentials;" + ), + )?, + method_get_credentials_for_path_ret: ReturnType::Object, + field_access_key_id: env.get_field_id( + &credentials_class, + jni::jni_str!("accessKeyId"), + jni::jni_sig!("Ljava/lang/String;"), + )?, + field_secret_access_key: env.get_field_id( + &credentials_class, + jni::jni_str!("secretAccessKey"), + jni::jni_sig!("Ljava/lang/String;"), + )?, + field_session_token: env.get_field_id( + &credentials_class, + jni::jni_str!("sessionToken"), + jni::jni_sig!("Ljava/lang/String;"), + )?, + field_expiration_epoch_millis: env.get_field_id( + &credentials_class, + jni::jni_str!("expirationEpochMillis"), + jni::jni_sig!("J"), + )?, + class, + credentials_class, + }) + } +} diff --git a/native/jni-bridge/src/lib.rs b/native/jni-bridge/src/lib.rs index d72323c961..e9bc71ba91 100644 --- a/native/jni-bridge/src/lib.rs +++ b/native/jni-bridge/src/lib.rs @@ -190,12 +190,14 @@ impl<'a> TryFrom> for BinaryWrapper<'a> { mod comet_exec; pub use comet_exec::*; mod batch_iterator; +mod comet_cloud_credential_dispatcher; mod comet_metric_node; mod comet_task_memory_manager; mod comet_udf_bridge; mod shuffle_block_iterator; use batch_iterator::CometBatchIterator; +pub use comet_cloud_credential_dispatcher::CometCloudCredentialDispatcher; pub use comet_metric_node::*; pub use comet_task_memory_manager::*; use comet_udf_bridge::CometUdfBridge; @@ -233,6 +235,10 @@ pub struct JVMClasses<'a> { /// The CometUdfBridge class used to dispatch JVM scalar UDFs. /// `None` if the class is not on the classpath. pub comet_udf_bridge: Option>, + /// JNI handles for the CometCloudCredentialDispatcher SPI and the CometCredentials POJO. + /// Always present (the classes ship in `comet-common`); whether a vendor provider is actually + /// registered is a separate runtime check. + pub comet_cloud_credential_dispatcher: CometCloudCredentialDispatcher<'a>, } unsafe impl Send for JVMClasses<'_> {} @@ -310,6 +316,8 @@ impl JVMClasses<'_> { } bridge }, + comet_cloud_credential_dispatcher: CometCloudCredentialDispatcher::new(env) + .unwrap(), } }); } diff --git a/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala b/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala index 37bb9c91fb..644ab402d8 100644 --- a/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala +++ b/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala @@ -74,6 +74,27 @@ trait CometS3TestBase extends CometTestBase { conf.set("spark.hadoop.fs.s3a.path.style.access", "true") } + protected def icebergAvailable: Boolean = { + try { + Class.forName("org.apache.iceberg.catalog.Catalog") + true + } catch { + case _: ClassNotFoundException => false + } + } + + /** + * Apply the S3 properties Comet's native Iceberg reader requires on the given catalog. + * iceberg-rust + opendal stops auto-detecting region when a custom credential loader is wired + * in (which is always, when the bridge SPI is registered), so the region/endpoint/ path-style + * triple has to be set explicitly. See the contributor guide on CometCloudCredentialProvider. + */ + protected def applyS3CatalogProps(conf: SparkConf, catalogName: String): Unit = { + conf.set(s"spark.sql.catalog.$catalogName.s3.endpoint", minioContainer.getS3URL) + conf.set(s"spark.sql.catalog.$catalogName.s3.region", "us-east-1") + conf.set(s"spark.sql.catalog.$catalogName.s3.path-style-access", "true") + } + protected def createBucketIfNotExists(bucketName: String): Unit = { val credentials = AwsBasicCredentials.create(userName, password) val s3Client = S3Client diff --git a/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala b/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala index 1d8c8aab8f..4b15f1c978 100644 --- a/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala +++ b/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala @@ -29,28 +29,13 @@ class IcebergReadFromS3Suite extends CometS3TestBase with RESTCatalogHelper { override protected val testBucketName = "test-iceberg-bucket" - private def icebergAvailable: Boolean = { - try { - Class.forName("org.apache.iceberg.catalog.Catalog") - true - } catch { - case _: ClassNotFoundException => false - } - } - override protected def sparkConf: SparkConf = { val conf = super.sparkConf conf.set("spark.sql.catalog.s3_catalog", "org.apache.iceberg.spark.SparkCatalog") conf.set("spark.sql.catalog.s3_catalog.type", "hadoop") conf.set("spark.sql.catalog.s3_catalog.warehouse", s"s3a://$testBucketName/warehouse") - // Required by Comet's native Iceberg reader (iceberg-rust + opendal). When a custom - // credential loader is wired in (CometCloudCredentialBridge), opendal stops auto-detecting - // region and requires explicit S3 config. See the contributor guide page on - // CometCloudCredentialProvider. - conf.set("spark.sql.catalog.s3_catalog.s3.endpoint", minioContainer.getS3URL) - conf.set("spark.sql.catalog.s3_catalog.s3.region", "us-east-1") - conf.set("spark.sql.catalog.s3_catalog.s3.path-style-access", "true") + applyS3CatalogProps(conf, "s3_catalog") conf.set(CometConf.COMET_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") diff --git a/spark/src/test/scala/org/apache/comet/cloud/CometCloudCredentialBridgeS3Suite.scala b/spark/src/test/scala/org/apache/comet/cloud/CometCloudCredentialBridgeS3Suite.scala index 3f35b60857..1685084004 100644 --- a/spark/src/test/scala/org/apache/comet/cloud/CometCloudCredentialBridgeS3Suite.scala +++ b/spark/src/test/scala/org/apache/comet/cloud/CometCloudCredentialBridgeS3Suite.scala @@ -43,27 +43,12 @@ class CometCloudCredentialBridgeS3Suite extends CometS3TestBase with AdaptiveSpa override protected val testBucketName = "bridge-test-bucket" - private def icebergAvailable: Boolean = { - try { - Class.forName("org.apache.iceberg.catalog.Catalog") - true - } catch { - case _: ClassNotFoundException => false - } - } - override protected def sparkConf: SparkConf = { val conf = super.sparkConf conf.set("spark.sql.catalog.s3_catalog", "org.apache.iceberg.spark.SparkCatalog") conf.set("spark.sql.catalog.s3_catalog.type", "hadoop") conf.set("spark.sql.catalog.s3_catalog.warehouse", s"s3a://$testBucketName/warehouse") - // Comet's native Iceberg reader uses iceberg-rust + opendal which requires explicit S3 config - // when a custom credential loader is wired in (opendal skips its default region-detection - // path in that case). The Hadoop catalog above doesn't propagate these, so we set them - // directly on the catalog config. - conf.set("spark.sql.catalog.s3_catalog.s3.endpoint", minioContainer.getS3URL) - conf.set("spark.sql.catalog.s3_catalog.s3.region", "us-east-1") - conf.set("spark.sql.catalog.s3_catalog.s3.path-style-access", "true") + applyS3CatalogProps(conf, "s3_catalog") conf.set(CometConf.COMET_ICEBERG_NATIVE_ENABLED.key, "true") conf } @@ -88,12 +73,14 @@ class CometCloudCredentialBridgeS3Suite extends CometS3TestBase with AdaptiveSpa test("Parquet read on S3 routes credentials through CometCloudCredentialProvider") { val testFilePath = s"s3a://$testBucketName/data/bridge-parquet.parquet" - spark.range(0, 1000).write.format("parquet").mode(SaveMode.Overwrite).save(testFilePath) + val rowCount = 1000L + spark.range(0, rowCount).write.format("parquet").mode(SaveMode.Overwrite).save(testFilePath) + val expectedSum = (0L until rowCount).sum MinioCometCredentialProvider.resetCounters() val df = spark.read.format("parquet").load(testFilePath).agg(sum(col("id"))) assertHasCometParquetScan(df.queryExecution.executedPlan) - assert(df.first().getLong(0) == 499500) + assert(df.first().getLong(0) == expectedSum) assert( MinioCometCredentialProvider.callCount() > 0, From 4d03053e1358842f9ab0fc35aeba8a9cb458f141 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 12 May 2026 21:48:52 -0400 Subject: [PATCH 08/24] fix native test failure in CI --- .../src/parquet/objectstore/comet_credential_bridge.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/native/core/src/parquet/objectstore/comet_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_credential_bridge.rs index 06fafe721c..8efb604dde 100644 --- a/native/core/src/parquet/objectstore/comet_credential_bridge.rs +++ b/native/core/src/parquet/objectstore/comet_credential_bridge.rs @@ -23,6 +23,7 @@ use crate::execution::operators::ExecutionError; use crate::jvm_bridge::{check_exception, JVMClasses}; +use crate::JAVA_VM; use async_trait::async_trait; use iceberg_storage_opendal::AwsCredential as IcebergAwsCredential; use jni::objects::{JFieldID, JObject, JString, JValue}; @@ -55,6 +56,12 @@ static PROVIDER_REGISTERED: OnceCell = OnceCell::new(); /// in front of the default credential paths. pub fn is_provider_registered() -> bool { *PROVIDER_REGISTERED.get_or_init(|| { + // Unit tests construct stores without a JVM; treat that as "no provider registered" so we + // don't trip `with_env`'s debug_assert. In production the JVM is always initialized before + // any object_store is built. + if JAVA_VM.get().is_none() { + return false; + } JVMClasses::with_env(|env| -> Result { let dispatcher = &JVMClasses::get().comet_cloud_credential_dispatcher; let result = unsafe { From a8cbe8efe9ce86cb50b15b32ae3b6ed4c226515d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 12 May 2026 21:55:33 -0400 Subject: [PATCH 09/24] update contributor guide about multiple providers --- .../cloud-credential-providers.md | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/docs/source/contributor-guide/cloud-credential-providers.md b/docs/source/contributor-guide/cloud-credential-providers.md index 04e6269c96..3bf7066c4a 100644 --- a/docs/source/contributor-guide/cloud-credential-providers.md +++ b/docs/source/contributor-guide/cloud-credential-providers.md @@ -158,6 +158,34 @@ There is no Comet-specific config knob for selecting a provider. Discovery follo of truth Spark itself reads, so a query that falls back from Comet to Spark mid-execution sees identical credentials. +### Future: multi-provider chains + +The "exactly one" rule is a design choice, not a fundamental constraint. Hadoop S3A's +`fs.s3a.aws.credentials.provider` chain (try each provider in order; first non-null wins) is a +familiar pattern, and a vendor with both a per-path STS provider and a catchall bucket provider +might reasonably want to compose them. + +If demand emerges, the dispatcher would change shape roughly as follows: + +- Replace the multi-impl `IllegalStateException` with "store the list." +- On `getCredentialsForPath`, iterate the list; first non-null return wins. Returning `null` + becomes a "I don't handle this path; try the next provider" signal rather than an authorization + failure. +- Order would be controlled by an explicit Spark config (e.g. a comma-separated list of class + names) because `ServiceLoader` iteration order is unspecified across JDKs. + +Until then, a vendor that needs chaining should implement the chain logic _inside_ a single +provider (e.g. try STS, fall back to vended creds, fall back to instance metadata). That keeps +ordering decisions in vendor code and avoids committing Comet to a chaining contract before there +is real usage to inform it. + +Note that this only applies to Comet SPI implementations. Existing JVM-side +`AwsCredentialsProvider` chains in `fs.s3a.aws.credentials.provider` are not reachable from +Comet's native readers regardless: the providers there implement AWS SDK interfaces, not the Comet +SPI, and calling them would force Comet to depend on the AWS SDK (and pick v1 vs v2). Vendors who +want their existing JVM chain to apply to Comet's native scans need a thin Comet SPI that +delegates to it. + ## Threading and lifecycle - The dispatcher caches the resolved provider in a `static final` field for the JVM lifetime. Your From 9d07ff00470c300a8b38d5a80b8ed092ea4c870b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 13 May 2026 10:40:29 -0400 Subject: [PATCH 10/24] add access mode to the SPI since a provider might have different credentials for read vs. write --- .../apache/comet/cloud/CometAccessMode.java | 36 +++++ .../cloud/CometCloudCredentialDispatcher.java | 17 ++- .../cloud/CometCloudCredentialProvider.java | 14 +- .../CometCloudCredentialDispatcherTest.java | 27 +++- .../TestCometCloudCredentialProvider.java | 5 +- .../cloud-credential-providers.md | 143 +++++++++++++----- .../src/execution/operators/iceberg_scan.rs | 8 +- .../objectstore/comet_credential_bridge.rs | 38 ++++- native/core/src/parquet/objectstore/s3.rs | 6 +- .../src/comet_cloud_credential_dispatcher.rs | 2 +- .../cloud/MinioCometCredentialProvider.java | 2 +- 11 files changed, 238 insertions(+), 60 deletions(-) create mode 100644 common/src/main/java/org/apache/comet/cloud/CometAccessMode.java diff --git a/common/src/main/java/org/apache/comet/cloud/CometAccessMode.java b/common/src/main/java/org/apache/comet/cloud/CometAccessMode.java new file mode 100644 index 0000000000..41575968fa --- /dev/null +++ b/common/src/main/java/org/apache/comet/cloud/CometAccessMode.java @@ -0,0 +1,36 @@ +/* + * 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.comet.cloud; + +/** + * Access intent for a credential request issued by Comet's native code, passed to {@link + * CometCloudCredentialProvider#getCredentialsForPath}. + * + *

Granularity is intentionally binary. Vendors that issue WRITE-scoped credentials are expected + * to include READ permissions when their workflows require it (multipart-completion HEAD, Iceberg + * manifest reads on the write path, etc.) — the SPI does not promise that a WRITE credential is + * also read-capable; the vendor's IAM policy does. + */ +public enum CometAccessMode { + /** GET / HEAD / LIST and equivalent. All Comet native scan paths request this today. */ + READ, + /** PUT / POST / DELETE / multipart and equivalent. Reserved for future native write paths. */ + WRITE +} diff --git a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java index b6885969fe..ebabd2ae04 100644 --- a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java +++ b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java @@ -53,7 +53,7 @@ * │ └────────────────────────────────────┘ │ | │ ╔════════════════════════════════════╗ │ * │ │ │ | │ ║ JNI CALL: ║ │ * │ ▼ │ | │ ║ getCredentialsForPath( ║ │ - * │ ┌────────────────────────────────────┐ │ | │ ║ bucket, path) ║ │ + * │ ┌────────────────────────────────────┐ │ | │ ║ bucket, path, mode) ║ │ * │ │ return CometCredentials POJO │──┼───────┼─────┼─►║ ║ │ * │ │ (access key, secret, token, │ │ | │ ╚════════════════════════════════════╝ │ * │ │ region, expiration) │ │ | │ │ │ @@ -73,7 +73,7 @@ * * Runtime Phase (per S3 request): * 4. object_store / opendal calls its async credential trait on CometCredentialBridge. - * 5. Bridge enters JNI, invokes dispatcher.getCredentialsForPath(bucket, path). + * 5. Bridge enters JNI, invokes dispatcher.getCredentialsForPath(bucket, path, mode). * 6. Provider returns a CometCredentials POJO; vendor may call its own STS / authorization service. * 7. Rust reads fields via JNI accessors, returns AwsCredential for request signing. */ @@ -128,19 +128,26 @@ public static boolean isProviderRegistered() { /** * Invoked by native code via JNI. Delegates to the registered provider. * + *

{@code mode} is passed as a {@code String} (the {@link CometAccessMode} name) rather than + * the enum itself to keep the JNI signature trivial — Rust passes a Java string, this method + * parses to the enum and forwards. + * * @throws IllegalStateException if no provider is registered (callers should check {@link * #isProviderRegistered()} first) + * @throws IllegalArgumentException if {@code mode} is not a recognized {@link CometAccessMode} + * name */ - public static CometCredentials getCredentialsForPath(String bucket, String path) + public static CometCredentials getCredentialsForPath(String bucket, String path, String mode) throws Exception { if (PROVIDER == null) { throw new IllegalStateException( "No CometCloudCredentialProvider registered; check META-INF/services on the classpath"); } + CometAccessMode accessMode = CometAccessMode.valueOf(mode); if (LOG.isDebugEnabled()) { - LOG.debug("Fetching credentials for bucket={} path={}", bucket, path); + LOG.debug("Fetching credentials for bucket={} path={} mode={}", bucket, path, accessMode); } - return PROVIDER.getCredentialsForPath(bucket, path); + return PROVIDER.getCredentialsForPath(bucket, path, accessMode); } private static CometCloudCredentialProvider resolve() { diff --git a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java index c66c6de9dd..62b0b5ce88 100644 --- a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java +++ b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java @@ -33,6 +33,11 @@ * *

Implementations must be thread-safe; {@link #getCredentialsForPath} may be invoked * concurrently from many native tokio tasks. + * + *

Contract: returns credentials or throws. There is no "fall through to the default chain" + * return value; if a provider is registered, it is responsible for every credential fetch on the + * paths it sees. See the contributor guide section on cloud credential providers for the rationale + * and patterns for vendors that need to defer to a default credential chain on a subset of paths. */ public interface CometCloudCredentialProvider { @@ -40,9 +45,12 @@ public interface CometCloudCredentialProvider { * Returns credentials usable to sign an S3 request for the given path. * * @param bucket the S3 bucket name (no scheme, no path) - * @param path the object key being accessed (no leading slash) - * @return credentials, or {@code null} if this provider does not handle the given path + * @param path the object key or prefix being accessed; the URL path of the store, leading slash + * included + * @param mode the access intent for this credential request + * @return non-null credentials * @throws Exception any failure surfaces to the native caller and aborts the request */ - CometCredentials getCredentialsForPath(String bucket, String path) throws Exception; + CometCredentials getCredentialsForPath(String bucket, String path, CometAccessMode mode) + throws Exception; } diff --git a/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java b/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java index e55dc6b527..7e8f5ee963 100644 --- a/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java +++ b/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java @@ -44,7 +44,7 @@ public void providerIsRegisteredFromTestClasspath() { @Test public void getCredentialsRoundTripsThroughProvider() throws Exception { CometCredentials creds = - CometCloudCredentialDispatcher.getCredentialsForPath("my-bucket", "path/to/object"); + CometCloudCredentialDispatcher.getCredentialsForPath("my-bucket", "path/to/object", "READ"); assertNotNull(creds); assertEquals("AKIATEST", creds.getAccessKeyId()); @@ -56,6 +56,20 @@ public void getCredentialsRoundTripsThroughProvider() throws Exception { assertEquals(1, TestCometCloudCredentialProvider.callCount.get()); assertEquals("my-bucket", TestCometCloudCredentialProvider.lastBucket); assertEquals("path/to/object", TestCometCloudCredentialProvider.lastPath); + assertEquals(CometAccessMode.READ, TestCometCloudCredentialProvider.lastMode); + } + + @Test + public void writeModeIsForwarded() throws Exception { + CometCloudCredentialDispatcher.getCredentialsForPath("b", "k", "WRITE"); + assertEquals(CometAccessMode.WRITE, TestCometCloudCredentialProvider.lastMode); + } + + @Test + public void unknownModeRejected() { + assertThrows( + IllegalArgumentException.class, + () -> CometCloudCredentialDispatcher.getCredentialsForPath("b", "k", "BOGUS")); } @Test @@ -65,7 +79,8 @@ public void providerExceptionsPropagate() { Exception thrown = assertThrows( - Exception.class, () -> CometCloudCredentialDispatcher.getCredentialsForPath("b", "k")); + Exception.class, + () -> CometCloudCredentialDispatcher.getCredentialsForPath("b", "k", "READ")); assertSame(boom, thrown); } @@ -74,7 +89,7 @@ public void nullSessionTokenAndRegionAreAllowed() throws Exception { TestCometCloudCredentialProvider.nextResult = new CometCredentials("AKIA", "sec", null, null, 0L); - CometCredentials creds = CometCloudCredentialDispatcher.getCredentialsForPath("b", "k"); + CometCredentials creds = CometCloudCredentialDispatcher.getCredentialsForPath("b", "k", "READ"); assertNull(creds.getSessionToken()); assertNull(creds.getRegion()); @@ -82,9 +97,9 @@ public void nullSessionTokenAndRegionAreAllowed() throws Exception { @Test public void providerReceivesEachCallSeparately() throws Exception { - CometCloudCredentialDispatcher.getCredentialsForPath("b1", "k1"); - CometCloudCredentialDispatcher.getCredentialsForPath("b2", "k2"); - CometCloudCredentialDispatcher.getCredentialsForPath("b3", "k3"); + CometCloudCredentialDispatcher.getCredentialsForPath("b1", "k1", "READ"); + CometCloudCredentialDispatcher.getCredentialsForPath("b2", "k2", "READ"); + CometCloudCredentialDispatcher.getCredentialsForPath("b3", "k3", "READ"); assertEquals(3, TestCometCloudCredentialProvider.callCount.get()); assertEquals("b3", TestCometCloudCredentialProvider.lastBucket); diff --git a/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java b/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java index 59776efad3..0774c849ca 100644 --- a/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java +++ b/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java @@ -31,6 +31,7 @@ public class TestCometCloudCredentialProvider implements CometCloudCredentialPro static final AtomicInteger callCount = new AtomicInteger(0); static volatile String lastBucket; static volatile String lastPath; + static volatile CometAccessMode lastMode; static volatile RuntimeException throwOnNext; static volatile CometCredentials nextResult = new CometCredentials("AKIATEST", "secret", "session-tok", "us-east-1", 0L); @@ -39,15 +40,17 @@ static void reset() { callCount.set(0); lastBucket = null; lastPath = null; + lastMode = null; throwOnNext = null; nextResult = new CometCredentials("AKIATEST", "secret", "session-tok", "us-east-1", 0L); } @Override - public CometCredentials getCredentialsForPath(String bucket, String path) { + public CometCredentials getCredentialsForPath(String bucket, String path, CometAccessMode mode) { callCount.incrementAndGet(); lastBucket = bucket; lastPath = path; + lastMode = mode; RuntimeException toThrow = throwOnNext; if (toThrow != null) { throwOnNext = null; diff --git a/docs/source/contributor-guide/cloud-credential-providers.md b/docs/source/contributor-guide/cloud-credential-providers.md index 3bf7066c4a..592e1651dd 100644 --- a/docs/source/contributor-guide/cloud-credential-providers.md +++ b/docs/source/contributor-guide/cloud-credential-providers.md @@ -79,13 +79,26 @@ Implement `org.apache.comet.cloud.CometCloudCredentialProvider`: package org.apache.comet.cloud; public interface CometCloudCredentialProvider { - CometCredentials getCredentialsForPath(String bucket, String path) throws Exception; + CometCredentials getCredentialsForPath( + String bucket, String path, CometAccessMode mode) throws Exception; } ``` `getCredentialsForPath` may be invoked concurrently from many native tokio worker threads. Implementations must be thread-safe. +The `mode` is the access intent for this credential request: + +| Value | Used for | +| ----------------- | ----------------------------------------------------------------------- | +| `READ` | All native scan paths (raw Parquet, Iceberg). Comet today only sends READ. | +| `WRITE` | Reserved for future native write paths (Iceberg writes, native INSERT). | + +The SPI does not promise that a `WRITE` credential is also read-capable; vendors that need +read-during-write workflows (multipart-completion HEAD, Iceberg manifest reads on the write path, +etc.) include the necessary read permissions in the IAM policy attached to their `WRITE` +credentials. + The returned `CometCredentials` POJO carries: | Field | Type | Notes | @@ -96,6 +109,66 @@ The returned `CometCredentials` POJO carries: | `region` | `String` (nullable) | `null` lets the native reader fall back to its config. | | `expirationEpochMillis` | `long` | `0` means "unknown" (see expiration semantics below). | +### Why this SPI returns or throws (no fallthrough return value) + +The SPI follows the same shape as every other AWS-credential SPI in the JVM ecosystem: + +| SPI | Method | Behavior | +| ------------------------------------------- | -------------------------------------------- | ------------------------- | +| AWS SDK v1 `AWSCredentialsProvider` | `getCredentials()` | returns or throws | +| AWS SDK v2 `AwsCredentialsProvider` | `resolveCredentials()` | returns or throws | +| Hadoop S3A `AWSCredentialsProvider` | `getCredentials()` | returns or throws | +| Iceberg `VendedCredentialsProvider` | `resolveCredentials()` | returns or throws | +| Iceberg `AwsClientFactory` | `s3()` | returns a configured client | +| **Comet `CometCloudCredentialProvider`** | `getCredentialsForPath(bucket, path, mode)` | **returns or throws** | + +Chaining/fallback in the AWS world is a separate concern, composed *outside* the provider — +e.g. AWS SDK v2's `AwsCredentialsProviderChain.builder().credentialsProviders(...)`, or the Hadoop +S3A `fs.s3a.aws.credentials.provider` comma-separated list resolved by +`AWSCredentialsProviderChain`. Each *individual* provider is atomic ("give me a credential or +fail"); a chain class composes them. + +Comet's SPI keeps to this convention. We considered an alternative `Optional` +return where empty would mean "fall through to Comet's native default AWS chain" — convenient +for path-scoped vendors but inconsistent with everything else at this layer. Composition is +cheap to do vendor-side and keeps the SPI surface narrow. + +#### Pattern: vendor that handles only a subset of paths + +If your provider is authoritative only for a subset of buckets/prefixes (e.g. you have indexed +policies for some paths and want others to use the host's default AWS credentials), construct +the default chain in your provider and return its credentials directly: + +```java +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; + +public final class PathScopedProvider implements CometCloudCredentialProvider { + + private final DefaultCredentialsProvider defaultChain = DefaultCredentialsProvider.create(); + + @Override + public CometCredentials getCredentialsForPath( + String bucket, String path, CometAccessMode mode) throws Exception { + if (handlesPath(bucket, path)) { + return mintFromMyVendorService(bucket, path, mode); + } + AwsCredentials c = defaultChain.resolveCredentials(); + String token = (c instanceof AwsSessionCredentials) + ? ((AwsSessionCredentials) c).sessionToken() + : null; + return new CometCredentials(c.accessKeyId(), c.secretAccessKey(), token, null, 0L); + } +} +``` + +This mirrors how Iceberg's `VendedCredentialsProvider` is implemented (it composes +`HTTPClient` + `CachedSupplier` internally; the SPI itself just returns or throws). Pick AWS SDK +v1 vs v2 to match whatever your existing signer / `AwsClientFactory` integration uses — the +choice is invisible to Comet because only `CometCredentials` (a plain POJO with strings + a long) +crosses the JNI boundary. + ### Expiration semantics `expirationEpochMillis` is the absolute expiry of the returned credential, in milliseconds since @@ -113,16 +186,34 @@ the Unix epoch. ### Error semantics - Throwing from `getCredentialsForPath` aborts the native S3 request and surfaces the exception - message (and chained causes) to the caller. -- Returning `null` is reserved for "this provider does not handle this path"; the native caller - treats it as an authorization failure rather than falling back to other providers. + message (and chained causes) to the caller. (See the iceberg-path note below for one current + exception to that propagation.) +- Returning `null` is a contract violation; the native bridge surfaces it as a request failure. + Implementations that want to defer to a default credential chain on a subset of paths should + resolve the default chain themselves and return its credentials — see the worked pattern under + *Why this SPI returns or throws* above. + +#### Iceberg path: error message fidelity + +When the bridge is wired into `iceberg-rust` (Iceberg native scans), iceberg-rust currently wraps +our credential loader in its own `ProvideCredentialChain`. That outer chain swallows errors into +"no credential" before the request reaches opendal (see `reqsign-core::ProvideCredentialChain`), +so a thrown exception surfaces as an opaque opendal/anonymous-request failure rather than your +exception message. The credential is still not issued and the request still fails — only the +message is degraded. + +This is tied to opendal's chain semantics. It would resolve if iceberg-rust either (a) stops +wrapping custom loaders in its own outer chain, or (b) moves Iceberg's S3 storage backend to +`object_store` (whose `CredentialProvider` has no chain-swallow behavior — auth failures +propagate cleanly as they do on Comet's raw Parquet path today). No Comet SPI change is needed +in either case. ### Iceberg path: explicit S3 region required -When the bridge is registered, Comet wires it into `iceberg-storage-opendal` as a custom -`AwsCredentialLoad`. opendal then requires explicit S3 region (and, for non-AWS endpoints, an -explicit endpoint) on the catalog properties — its built-in region auto-detection only runs -when no custom credential loader is configured. +When the bridge is registered, Comet wires it into `iceberg-storage-opendal` as a +`CustomAwsCredentialLoader`. opendal then requires explicit S3 region (and, for non-AWS +endpoints, an explicit endpoint) on the catalog properties — its built-in region auto-detection +only runs when no custom credential loader is configured. In practice this means deployments using the bridge for Iceberg must set, on the Spark catalog: @@ -158,33 +249,11 @@ There is no Comet-specific config knob for selecting a provider. Discovery follo of truth Spark itself reads, so a query that falls back from Comet to Spark mid-execution sees identical credentials. -### Future: multi-provider chains - -The "exactly one" rule is a design choice, not a fundamental constraint. Hadoop S3A's -`fs.s3a.aws.credentials.provider` chain (try each provider in order; first non-null wins) is a -familiar pattern, and a vendor with both a per-path STS provider and a catchall bucket provider -might reasonably want to compose them. - -If demand emerges, the dispatcher would change shape roughly as follows: - -- Replace the multi-impl `IllegalStateException` with "store the list." -- On `getCredentialsForPath`, iterate the list; first non-null return wins. Returning `null` - becomes a "I don't handle this path; try the next provider" signal rather than an authorization - failure. -- Order would be controlled by an explicit Spark config (e.g. a comma-separated list of class - names) because `ServiceLoader` iteration order is unspecified across JDKs. - -Until then, a vendor that needs chaining should implement the chain logic _inside_ a single -provider (e.g. try STS, fall back to vended creds, fall back to instance metadata). That keeps -ordering decisions in vendor code and avoids committing Comet to a chaining contract before there -is real usage to inform it. - -Note that this only applies to Comet SPI implementations. Existing JVM-side -`AwsCredentialsProvider` chains in `fs.s3a.aws.credentials.provider` are not reachable from -Comet's native readers regardless: the providers there implement AWS SDK interfaces, not the Comet -SPI, and calling them would force Comet to depend on the AWS SDK (and pick v1 vs v2). Vendors who -want their existing JVM chain to apply to Comet's native scans need a thin Comet SPI that -delegates to it. +A vendor that wants to compose multiple credential sources (per-path STS for some prefixes, a +catchall provider for others) does so inside their single provider implementation — same as how +AWS SDK v1/v2 providers compose into `AwsCredentialsProviderChain` at the call site rather than +exposing chain semantics through the individual provider contract. See *Why this SPI returns or +throws* above for a worked example. ## Threading and lifecycle @@ -235,6 +304,7 @@ A minimal static-credential provider, suitable for tests and development: ```java package com.example.comet.test; +import org.apache.comet.cloud.CometAccessMode; import org.apache.comet.cloud.CometCloudCredentialProvider; import org.apache.comet.cloud.CometCredentials; @@ -244,7 +314,8 @@ public final class StaticCometCredentialProvider implements CometCloudCredential private static final String REGION = System.getenv().getOrDefault("EXAMPLE_REGION", "us-east-1"); @Override - public CometCredentials getCredentialsForPath(String bucket, String path) { + public CometCredentials getCredentialsForPath( + String bucket, String path, CometAccessMode mode) { return new CometCredentials(ACCESS_KEY, SECRET_KEY, null, REGION, 0L); } } diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 77e8e64c3c..c9d16d0b89 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -40,6 +40,7 @@ use datafusion::physical_plan::{ use futures::{Stream, StreamExt, TryStreamExt}; use iceberg::arrow::ScanMetrics; use iceberg::io::{FileIO, FileIOBuilder, StorageFactory}; +use iceberg_storage_opendal::CustomAwsCredentialLoader; use iceberg_storage_opendal::OpenDalStorageFactory; use crate::execution::operators::ExecutionError; @@ -49,7 +50,6 @@ use crate::parquet::schema_adapter::SparkPhysicalExprAdapterFactory; use datafusion_comet_spark_expr::EvalMode; use datafusion_physical_expr_adapter::{PhysicalExprAdapter, PhysicalExprAdapterFactory}; use iceberg::scan::FileScanTask; -use iceberg_storage_opendal::CustomAwsCredentialLoader; /// Iceberg table scan operator that uses iceberg-rust to read Iceberg tables. /// @@ -246,7 +246,11 @@ fn build_s3_credential_loader(metadata_location: &str) -> Option = OnceCell::new(); +/// Access intent passed to the Java SPI. Mirrors `CometAccessMode` on the Java side; the JVM-side +/// `valueOf` parses these names so they are part of the cross-language contract. +#[derive(Debug, Clone, Copy)] +pub enum AccessMode { + Read, + /// Reserved for future native write paths (Iceberg writes, native INSERT). No call site + /// constructs this yet; allow it to keep the enum complete and to lock the JVM-side + /// `CometAccessMode.valueOf("WRITE")` contract. + #[allow(dead_code)] + Write, +} + +impl AccessMode { + fn as_jvm_str(self) -> &'static str { + match self { + AccessMode::Read => "READ", + AccessMode::Write => "WRITE", + } + } +} + /// True iff a `CometCloudCredentialProvider` was discovered on the JVM classpath. Used by /// `s3.rs::create_store` and `iceberg_scan.rs` to decide whether to wire a [`CometCredentialBridge`] /// in front of the default credential paths. @@ -99,13 +124,13 @@ pub fn is_provider_registered() -> bool { /// Per-request credential provider that delegates to the Java SPI via JNI. /// /// One instance is constructed per S3 store (per-URL in `create_store`) or per FileIO (the -/// metadata location, in `iceberg_scan.rs`). The `(bucket, path)` tuple is forwarded verbatim -/// on every credential fetch; the Java provider is free to return different credentials for -/// different paths. +/// metadata location, in `iceberg_scan.rs`). The `(bucket, path, mode)` tuple is forwarded +/// verbatim on every credential fetch. #[derive(Debug)] pub struct CometCredentialBridge { bucket: String, path: String, + mode: AccessMode, /// Latched once the bridge observes a credential without an expiry, so the warning that /// goes with [`DEFAULT_EXPIRY_WHEN_UNKNOWN`] only fires once per bridge instance instead of /// per request. @@ -113,10 +138,11 @@ pub struct CometCredentialBridge { } impl CometCredentialBridge { - pub fn new(bucket: impl Into, path: impl Into) -> Self { + pub fn new(bucket: impl Into, path: impl Into, mode: AccessMode) -> Self { Self { bucket: bucket.into(), path: path.into(), + mode, warned_missing_expiry: AtomicBool::new(false), } } @@ -132,6 +158,9 @@ impl CometCredentialBridge { let path_jstr = env .new_string(&self.path) .map_err(|e| ExecutionError::GeneralError(format!("new_string(path): {e}")))?; + let mode_jstr = env + .new_string(self.mode.as_jvm_str()) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(mode): {e}")))?; let result = unsafe { env.call_static_method_unchecked( @@ -141,6 +170,7 @@ impl CometCredentialBridge { &[ JValue::from(&bucket_jstr).as_jni(), JValue::from(&path_jstr).as_jni(), + JValue::from(&mode_jstr).as_jni(), ], ) }; diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index da72c8d206..23b3456a51 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -83,7 +83,11 @@ pub fn create_store( get_runtime().block_on(build_credential_provider(configs, bucket, min_ttl))?; builder = if comet_credential_bridge::is_provider_registered() { debug!("Using CometCredentialBridge for bucket {bucket}"); - let bridge = comet_credential_bridge::CometCredentialBridge::new(bucket, url.path()); + let bridge = comet_credential_bridge::CometCredentialBridge::new( + bucket, + url.path(), + comet_credential_bridge::AccessMode::Read, + ); builder.with_credentials(Arc::new(bridge)) } else { match credential_provider { diff --git a/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs b/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs index 629777b316..f37e28fb95 100644 --- a/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs +++ b/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs @@ -59,7 +59,7 @@ impl<'a> CometCloudCredentialDispatcher<'a> { JNIString::new(Self::JVM_CLASS), jni::jni_str!("getCredentialsForPath"), jni::jni_sig!( - "(Ljava/lang/String;Ljava/lang/String;)Lorg/apache/comet/cloud/CometCredentials;" + "(Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;)Lorg/apache/comet/cloud/CometCredentials;" ), )?, method_get_credentials_for_path_ret: ReturnType::Object, diff --git a/spark/src/test/java/org/apache/comet/cloud/MinioCometCredentialProvider.java b/spark/src/test/java/org/apache/comet/cloud/MinioCometCredentialProvider.java index 92f785f18b..04348df949 100644 --- a/spark/src/test/java/org/apache/comet/cloud/MinioCometCredentialProvider.java +++ b/spark/src/test/java/org/apache/comet/cloud/MinioCometCredentialProvider.java @@ -63,7 +63,7 @@ public static void resetCounters() { } @Override - public CometCredentials getCredentialsForPath(String bucket, String path) { + public CometCredentials getCredentialsForPath(String bucket, String path, CometAccessMode mode) { CALL_COUNT.incrementAndGet(); LAST_BUCKET.set(bucket); LAST_PATH.set(path); From 9b1e6222ac42f056bfcb3799d14fb019c1856aaa Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 13 May 2026 11:00:39 -0400 Subject: [PATCH 11/24] clean up iceberg path discrepancy --- native/core/src/execution/operators/iceberg_scan.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index c9d16d0b89..6de8a1e261 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -246,9 +246,11 @@ fn build_s3_credential_loader(metadata_location: &str) -> Option Date: Wed, 13 May 2026 11:17:32 -0400 Subject: [PATCH 12/24] run prettier on the docs --- .../cloud-credential-providers.md | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/docs/source/contributor-guide/cloud-credential-providers.md b/docs/source/contributor-guide/cloud-credential-providers.md index 592e1651dd..7cdf21df99 100644 --- a/docs/source/contributor-guide/cloud-credential-providers.md +++ b/docs/source/contributor-guide/cloud-credential-providers.md @@ -89,10 +89,10 @@ Implementations must be thread-safe. The `mode` is the access intent for this credential request: -| Value | Used for | -| ----------------- | ----------------------------------------------------------------------- | -| `READ` | All native scan paths (raw Parquet, Iceberg). Comet today only sends READ. | -| `WRITE` | Reserved for future native write paths (Iceberg writes, native INSERT). | +| Value | Used for | +| ------- | -------------------------------------------------------------------------- | +| `READ` | All native scan paths (raw Parquet, Iceberg). Comet today only sends READ. | +| `WRITE` | Reserved for future native write paths (Iceberg writes, native INSERT). | The SPI does not promise that a `WRITE` credential is also read-capable; vendors that need read-during-write workflows (multipart-completion HEAD, Iceberg manifest reads on the write path, @@ -113,19 +113,19 @@ The returned `CometCredentials` POJO carries: The SPI follows the same shape as every other AWS-credential SPI in the JVM ecosystem: -| SPI | Method | Behavior | -| ------------------------------------------- | -------------------------------------------- | ------------------------- | -| AWS SDK v1 `AWSCredentialsProvider` | `getCredentials()` | returns or throws | -| AWS SDK v2 `AwsCredentialsProvider` | `resolveCredentials()` | returns or throws | -| Hadoop S3A `AWSCredentialsProvider` | `getCredentials()` | returns or throws | -| Iceberg `VendedCredentialsProvider` | `resolveCredentials()` | returns or throws | -| Iceberg `AwsClientFactory` | `s3()` | returns a configured client | -| **Comet `CometCloudCredentialProvider`** | `getCredentialsForPath(bucket, path, mode)` | **returns or throws** | +| SPI | Method | Behavior | +| ---------------------------------------- | ------------------------------------------- | --------------------------- | +| AWS SDK v1 `AWSCredentialsProvider` | `getCredentials()` | returns or throws | +| AWS SDK v2 `AwsCredentialsProvider` | `resolveCredentials()` | returns or throws | +| Hadoop S3A `AWSCredentialsProvider` | `getCredentials()` | returns or throws | +| Iceberg `VendedCredentialsProvider` | `resolveCredentials()` | returns or throws | +| Iceberg `AwsClientFactory` | `s3()` | returns a configured client | +| **Comet `CometCloudCredentialProvider`** | `getCredentialsForPath(bucket, path, mode)` | **returns or throws** | -Chaining/fallback in the AWS world is a separate concern, composed *outside* the provider — +Chaining/fallback in the AWS world is a separate concern, composed _outside_ the provider — e.g. AWS SDK v2's `AwsCredentialsProviderChain.builder().credentialsProviders(...)`, or the Hadoop S3A `fs.s3a.aws.credentials.provider` comma-separated list resolved by -`AWSCredentialsProviderChain`. Each *individual* provider is atomic ("give me a credential or +`AWSCredentialsProviderChain`. Each _individual_ provider is atomic ("give me a credential or fail"); a chain class composes them. Comet's SPI keeps to this convention. We considered an alternative `Optional` @@ -191,7 +191,7 @@ the Unix epoch. - Returning `null` is a contract violation; the native bridge surfaces it as a request failure. Implementations that want to defer to a default credential chain on a subset of paths should resolve the default chain themselves and return its credentials — see the worked pattern under - *Why this SPI returns or throws* above. + _Why this SPI returns or throws_ above. #### Iceberg path: error message fidelity @@ -252,8 +252,8 @@ identical credentials. A vendor that wants to compose multiple credential sources (per-path STS for some prefixes, a catchall provider for others) does so inside their single provider implementation — same as how AWS SDK v1/v2 providers compose into `AwsCredentialsProviderChain` at the call site rather than -exposing chain semantics through the individual provider contract. See *Why this SPI returns or -throws* above for a worked example. +exposing chain semantics through the individual provider contract. See _Why this SPI returns or +throws_ above for a worked example. ## Threading and lifecycle From d9596dede9fe71091e017f804afd738111f8d096 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 14 May 2026 12:12:58 -0400 Subject: [PATCH 13/24] cleanup to get ready for review --- .../cloud/CometCloudCredentialDispatcher.java | 176 --------- .../cloud/CometCloudCredentialProvider.java | 56 --- .../CometS3AccessMode.java} | 18 +- .../cloud/s3/CometS3CredentialDispatcher.java | 90 +++++ .../cloud/s3/CometS3CredentialProvider.java | 67 ++++ .../CometS3Credentials.java} | 30 +- .../CometS3CredentialDispatcherTest.java} | 54 +-- .../TestCometS3CredentialProvider.java} | 20 +- ...e.comet.cloud.CometCloudCredentialProvider | 1 - ...e.comet.cloud.s3.CometS3CredentialProvider | 1 + dev/ci/check-suites.py | 2 +- .../cloud-credential-providers.md | 343 ------------------ docs/source/contributor-guide/index.md | 1 - .../latest/cloud-credential-providers.md | 190 +++++++--- .../src/execution/operators/iceberg_scan.rs | 18 +- .../objectstore/comet_credential_bridge.rs | 337 ----------------- .../objectstore/comet_s3_credential_bridge.rs | 290 +++++++++++++++ native/core/src/parquet/objectstore/mod.rs | 2 +- native/core/src/parquet/objectstore/s3.rs | 16 +- ...r.rs => comet_s3_credential_dispatcher.rs} | 22 +- native/jni-bridge/src/lib.rs | 11 +- .../MinioCometS3CredentialProvider.java} | 23 +- ...e.comet.cloud.CometCloudCredentialProvider | 1 - ...e.comet.cloud.s3.CometS3CredentialProvider | 1 + .../org/apache/comet/CometS3TestBase.scala | 15 +- .../CometS3CredentialBridgeSuite.scala} | 34 +- 26 files changed, 694 insertions(+), 1125 deletions(-) delete mode 100644 common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java delete mode 100644 common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java rename common/src/main/java/org/apache/comet/cloud/{CometAccessMode.java => s3/CometS3AccessMode.java} (52%) create mode 100644 common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java create mode 100644 common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java rename common/src/main/java/org/apache/comet/cloud/{CometCredentials.java => s3/CometS3Credentials.java} (64%) rename common/src/test/java/org/apache/comet/cloud/{CometCloudCredentialDispatcherTest.java => s3/CometS3CredentialDispatcherTest.java} (54%) rename common/src/test/java/org/apache/comet/cloud/{TestCometCloudCredentialProvider.java => s3/TestCometS3CredentialProvider.java} (65%) delete mode 100644 common/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider create mode 100644 common/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider delete mode 100644 docs/source/contributor-guide/cloud-credential-providers.md delete mode 100644 native/core/src/parquet/objectstore/comet_credential_bridge.rs create mode 100644 native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs rename native/jni-bridge/src/{comet_cloud_credential_dispatcher.rs => comet_s3_credential_dispatcher.rs} (80%) rename spark/src/test/java/org/apache/comet/cloud/{MinioCometCredentialProvider.java => s3/MinioCometS3CredentialProvider.java} (66%) delete mode 100644 spark/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider create mode 100644 spark/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider rename spark/src/test/scala/org/apache/comet/cloud/{CometCloudCredentialBridgeS3Suite.scala => s3/CometS3CredentialBridgeSuite.scala} (69%) diff --git a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java deleted file mode 100644 index ebabd2ae04..0000000000 --- a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java +++ /dev/null @@ -1,176 +0,0 @@ -/* - * 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.comet.cloud; - -import java.util.ArrayList; -import java.util.List; -import java.util.ServiceLoader; -import java.util.stream.Collectors; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -// spotless:off -/* - * Architecture Overview: - * - * JVM Side | Native Side - * ┌──────────────────────────────────────────┐ | ┌──────────────────────────────────────────┐ - * │ CometCloudCredentialDispatcher │ | │ S3 Object Reading │ - * │ │ | │ │ - * │ ┌────────────────────────────────────┐ │ | │ ┌────────────────────────────────────┐ │ - * │ │ ServiceLoader discovery: │ │ | │ │ DataFusion iceberg-rust │ │ - * │ │ META-INF/services/ │ │ | │ │ object_store opendal │ │ - * │ │ o.a.c.cloud.CometCloudCred... │ │ | │ └────────────────────────────────────┘ │ - * │ └────────────────────────────────────┘ │ | │ │ │ │ - * │ │ │ | │ ▼ ▼ │ - * │ ▼ │ | │ ┌────────────────────────────────────┐ │ - * │ ┌────────────────────────────────────┐ │ | │ │ CometCredentialBridge (Rust) │ │ - * │ │ CometCloudCredentialProvider │ │ | │ │ impl object_store:: │ │ - * │ │ (single instance, cached) │ │ | │ │ CredentialProvider │ │ - * │ └────────────────────────────────────┘ │ | │ │ impl reqsign_core:: │ │ - * │ │ │ | │ │ ProvideCredential │ │ - * │ ▼ │ | │ └────────────────────────────────────┘ │ - * │ ┌────────────────────────────────────┐ │ | │ │ │ - * │ │ .getCredentialsForPath(...) │◄─┼───────┼─────┼──╗ ▼ │ - * │ └────────────────────────────────────┘ │ | │ ╔════════════════════════════════════╗ │ - * │ │ │ | │ ║ JNI CALL: ║ │ - * │ ▼ │ | │ ║ getCredentialsForPath( ║ │ - * │ ┌────────────────────────────────────┐ │ | │ ║ bucket, path, mode) ║ │ - * │ │ return CometCredentials POJO │──┼───────┼─────┼─►║ ║ │ - * │ │ (access key, secret, token, │ │ | │ ╚════════════════════════════════════╝ │ - * │ │ region, expiration) │ │ | │ │ │ - * │ └────────────────────────────────────┘ │ | │ ▼ │ - * │ │ | │ ┌────────────────────────────────────┐ │ - * │ │ | │ │ AwsCredential │ │ - * │ │ | │ │ used to sign S3 requests │ │ - * │ │ | │ └────────────────────────────────────┘ │ - * └──────────────────────────────────────────┘ | └──────────────────────────────────────────┘ - * | - * JNI Boundary - * - * Setup Phase (one-time per executor): - * 1. Vendor JAR ships an impl of CometCloudCredentialProvider via META-INF/services. - * 2. CometCloudCredentialDispatcher resolves it via ServiceLoader on first class-load. - * 3. Native side caches dispatcher class + static method ID in OnceCell. - * - * Runtime Phase (per S3 request): - * 4. object_store / opendal calls its async credential trait on CometCredentialBridge. - * 5. Bridge enters JNI, invokes dispatcher.getCredentialsForPath(bucket, path, mode). - * 6. Provider returns a CometCredentials POJO; vendor may call its own STS / authorization service. - * 7. Rust reads fields via JNI accessors, returns AwsCredential for request signing. - */ -// spotless:on - -/** - * Static entry point invoked from Comet's native code (via JNI) to fetch AWS credentials for an S3 - * request. - * - *

Resolution rules at first class-load: - * - *

- * - *

Discovery is via classpath only; there is no Comet-specific config knob for selecting a - * provider. This keeps the credentials Comet uses identical to whatever the same JVM would use if a - * query fell back to Spark execution mid-flight. - */ -public final class CometCloudCredentialDispatcher { - - private static final Logger LOG = LoggerFactory.getLogger(CometCloudCredentialDispatcher.class); - - /* - * Process-lifetime singleton, justified per the contributor guide's "Global singletons" - * section. - * - * Why static is the right lifetime: ServiceLoader discovers the impl from the executor - * classpath, which is fixed once Spark has launched the JVM. The same instance must serve - * every credential request from native code so that a query falling back from Comet to - * Spark mid-execution sees identical credentials. - * - * Bounded: a single reference, not a cache. - * - * Credential refresh: this dispatcher does NOT cache credentials. Each call to - * getCredentialsForPath delegates straight to the provider, which is responsible for any - * STS / token refresh logic. Stale-credential failure modes therefore live in the provider - * impl, not here. - */ - private static final CometCloudCredentialProvider PROVIDER = resolve(); - - private CometCloudCredentialDispatcher() {} - - public static boolean isProviderRegistered() { - return PROVIDER != null; - } - - /** - * Invoked by native code via JNI. Delegates to the registered provider. - * - *

{@code mode} is passed as a {@code String} (the {@link CometAccessMode} name) rather than - * the enum itself to keep the JNI signature trivial — Rust passes a Java string, this method - * parses to the enum and forwards. - * - * @throws IllegalStateException if no provider is registered (callers should check {@link - * #isProviderRegistered()} first) - * @throws IllegalArgumentException if {@code mode} is not a recognized {@link CometAccessMode} - * name - */ - public static CometCredentials getCredentialsForPath(String bucket, String path, String mode) - throws Exception { - if (PROVIDER == null) { - throw new IllegalStateException( - "No CometCloudCredentialProvider registered; check META-INF/services on the classpath"); - } - CometAccessMode accessMode = CometAccessMode.valueOf(mode); - if (LOG.isDebugEnabled()) { - LOG.debug("Fetching credentials for bucket={} path={} mode={}", bucket, path, accessMode); - } - return PROVIDER.getCredentialsForPath(bucket, path, accessMode); - } - - private static CometCloudCredentialProvider resolve() { - List impls = new ArrayList<>(); - for (CometCloudCredentialProvider impl : - ServiceLoader.load(CometCloudCredentialProvider.class)) { - impls.add(impl); - } - if (impls.isEmpty()) { - LOG.info( - "No CometCloudCredentialProvider registered; native S3 readers will use the default " - + "AWS credential chain"); - return null; - } - if (impls.size() > 1) { - List names = - impls.stream().map(p -> p.getClass().getName()).collect(Collectors.toList()); - LOG.error("Multiple CometCloudCredentialProvider impls on classpath: {}", names); - throw new IllegalStateException( - "Multiple CometCloudCredentialProvider impls on classpath: " + names); - } - CometCloudCredentialProvider provider = impls.get(0); - LOG.info("Registered CometCloudCredentialProvider: {}", provider.getClass().getName()); - return provider; - } -} diff --git a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java b/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java deleted file mode 100644 index 62b0b5ce88..0000000000 --- a/common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.comet.cloud; - -/** - * SPI for supplying AWS credentials to Comet's native S3 readers. - * - *

Comet's native scan paths ({@code object_store} for raw Parquet, {@code opendal} via {@code - * iceberg-rust} for Iceberg) bypass Spark's Hadoop S3A code path. The standard {@code - * AWSCredentialsProvider.getCredentials()} contract has no path argument, so vendors that issue - * per-path STS credentials cannot expose them through that interface. This SPI fills the gap. - * - *

Vendors register an implementation via {@code - * META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider}. Comet discovers it at - * executor startup and routes every per-request credential fetch through it. - * - *

Implementations must be thread-safe; {@link #getCredentialsForPath} may be invoked - * concurrently from many native tokio tasks. - * - *

Contract: returns credentials or throws. There is no "fall through to the default chain" - * return value; if a provider is registered, it is responsible for every credential fetch on the - * paths it sees. See the contributor guide section on cloud credential providers for the rationale - * and patterns for vendors that need to defer to a default credential chain on a subset of paths. - */ -public interface CometCloudCredentialProvider { - - /** - * Returns credentials usable to sign an S3 request for the given path. - * - * @param bucket the S3 bucket name (no scheme, no path) - * @param path the object key or prefix being accessed; the URL path of the store, leading slash - * included - * @param mode the access intent for this credential request - * @return non-null credentials - * @throws Exception any failure surfaces to the native caller and aborts the request - */ - CometCredentials getCredentialsForPath(String bucket, String path, CometAccessMode mode) - throws Exception; -} diff --git a/common/src/main/java/org/apache/comet/cloud/CometAccessMode.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java similarity index 52% rename from common/src/main/java/org/apache/comet/cloud/CometAccessMode.java rename to common/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java index 41575968fa..f6ef294141 100644 --- a/common/src/main/java/org/apache/comet/cloud/CometAccessMode.java +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java @@ -17,20 +17,12 @@ * under the License. */ -package org.apache.comet.cloud; +package org.apache.comet.cloud.s3; -/** - * Access intent for a credential request issued by Comet's native code, passed to {@link - * CometCloudCredentialProvider#getCredentialsForPath}. - * - *

Granularity is intentionally binary. Vendors that issue WRITE-scoped credentials are expected - * to include READ permissions when their workflows require it (multipart-completion HEAD, Iceberg - * manifest reads on the write path, etc.) — the SPI does not promise that a WRITE credential is - * also read-capable; the vendor's IAM policy does. - */ -public enum CometAccessMode { - /** GET / HEAD / LIST and equivalent. All Comet native scan paths request this today. */ +/** Access intent passed to {@link CometS3CredentialProvider#getCredentialsForPath}. */ +public enum CometS3AccessMode { + /** GET / HEAD / LIST. All Comet native scan paths request this today. */ READ, - /** PUT / POST / DELETE / multipart and equivalent. Reserved for future native write paths. */ + /** PUT / POST / DELETE / multipart. Reserved for future native write paths. */ WRITE } diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java new file mode 100644 index 0000000000..e54e9fa026 --- /dev/null +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java @@ -0,0 +1,90 @@ +/* + * 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.comet.cloud.s3; + +import java.util.ArrayList; +import java.util.List; +import java.util.ServiceLoader; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * JNI entry point invoked from native code to resolve {@link CometS3CredentialProvider}. + * + *

The provider is resolved once via {@link ServiceLoader} and cached in a {@code static final} + * field. A query falling back from Comet to Spark mid-execution therefore sees identical + * credentials, since both paths resolve from the same executor classpath. + * + *

Multiple registered impls fail fast at class-load; chaining is a vendor-side concern. + */ +public final class CometS3CredentialDispatcher { + + private static final Logger LOG = LoggerFactory.getLogger(CometS3CredentialDispatcher.class); + + private static final CometS3CredentialProvider PROVIDER = resolve(); + private static final CometS3AccessMode[] MODES = CometS3AccessMode.values(); + + private CometS3CredentialDispatcher() {} + + public static boolean isProviderRegistered() { + return PROVIDER != null; + } + + /** Invoked by native code. {@code mode} is the {@link CometS3AccessMode} ordinal. */ + public static CometS3Credentials getCredentialsForPath(String bucket, String path, int mode) + throws Exception { + if (PROVIDER == null) { + throw new IllegalStateException( + "No CometS3CredentialProvider registered; check META-INF/services on the classpath"); + } + if (mode < 0 || mode >= MODES.length) { + throw new IllegalArgumentException("Invalid CometS3AccessMode ordinal: " + mode); + } + CometS3AccessMode accessMode = MODES[mode]; + if (LOG.isDebugEnabled()) { + LOG.debug("Fetching credentials for bucket={} path={} mode={}", bucket, path, accessMode); + } + return PROVIDER.getCredentialsForPath(bucket, path, accessMode); + } + + private static CometS3CredentialProvider resolve() { + List impls = new ArrayList<>(); + for (CometS3CredentialProvider impl : ServiceLoader.load(CometS3CredentialProvider.class)) { + impls.add(impl); + } + if (impls.isEmpty()) { + LOG.info( + "No CometS3CredentialProvider registered; native S3 readers will use the default " + + "AWS credential chain"); + return null; + } + if (impls.size() > 1) { + List names = + impls.stream().map(p -> p.getClass().getName()).collect(Collectors.toList()); + throw new IllegalStateException( + "Multiple CometS3CredentialProvider impls on classpath: " + names); + } + CometS3CredentialProvider provider = impls.get(0); + LOG.info("Registered CometS3CredentialProvider: {}", provider.getClass().getName()); + return provider; + } +} diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java new file mode 100644 index 0000000000..677645a3e7 --- /dev/null +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java @@ -0,0 +1,67 @@ +/* + * 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.comet.cloud.s3; + +/** + * SPI for supplying AWS credentials to Comet's native S3 readers, which bypass Spark's Hadoop S3A + * code path and cannot reach signer-based or path-aware credential mechanisms through the standard + * parameterless {@code AWSCredentialsProvider.getCredentials()} contract. + * + *

Peer to {@code org.apache.hadoop.fs.s3a.AwsSignerInitializer} (Hadoop S3A) and {@code + * org.apache.iceberg.aws.AwsClientFactory} (Iceberg-Java): the same shape vendors already implement + * for those two, with a smaller surface (one method). + * + *

Why a new SPI?

+ * + * No existing contract carries per-path AWS credentials from vendor code to Comet's native readers: + * + * + * + *

Vendors register an implementation via {@code + * META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider}. {@link + * #getCredentialsForPath} may be invoked concurrently from many native tokio tasks, so + * implementations must be thread-safe. + * + *

Returns credentials or throws; there is no fall-through return value. A provider that is only + * authoritative for some paths should resolve the default AWS chain itself for the rest. See the + * user guide on cloud credential providers. + */ +public interface CometS3CredentialProvider { + + /** + * @param bucket S3 bucket name (no scheme, no path) + * @param path object key or prefix, leading slash included (matches the URL path component) + * @param mode access intent for this request + * @return non-null credentials; {@code null} is a contract violation + */ + CometS3Credentials getCredentialsForPath(String bucket, String path, CometS3AccessMode mode) + throws Exception; +} diff --git a/common/src/main/java/org/apache/comet/cloud/CometCredentials.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java similarity index 64% rename from common/src/main/java/org/apache/comet/cloud/CometCredentials.java rename to common/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java index 1df349fa8a..6c443bde52 100644 --- a/common/src/main/java/org/apache/comet/cloud/CometCredentials.java +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java @@ -17,38 +17,30 @@ * under the License. */ -package org.apache.comet.cloud; +package org.apache.comet.cloud.s3; import java.util.Objects; /** - * Credentials returned by a {@link CometCloudCredentialProvider}, consumed by Comet's native code - * via JNI field accessors. + * Credentials returned by a {@link CometS3CredentialProvider}. Fields are read back over JNI by + * name, so the field names are part of the cross-language contract. * - *

{@code sessionToken} is null for non-STS credentials; {@code region} is null when the provider - * has no opinion (the native side falls back to its configured region). {@code - * expirationEpochMillis} is {@code 0} when the provider does not track expiration; in that case - * Comet will not pre-emptively refresh and relies on the provider to return fresh credentials on - * each call. + *

{@code sessionToken} is null for non-STS credentials. {@code expirationEpochMillis} of {@code + * 0} means "unknown"; the Iceberg path then caps opendal's cache at a short fallback to avoid + * serving stale credentials for the executor lifetime. */ -public final class CometCredentials { +public final class CometS3Credentials { private final String accessKeyId; private final String secretAccessKey; private final String sessionToken; - private final String region; private final long expirationEpochMillis; - public CometCredentials( - String accessKeyId, - String secretAccessKey, - String sessionToken, - String region, - long expirationEpochMillis) { + public CometS3Credentials( + String accessKeyId, String secretAccessKey, String sessionToken, long expirationEpochMillis) { this.accessKeyId = Objects.requireNonNull(accessKeyId, "accessKeyId"); this.secretAccessKey = Objects.requireNonNull(secretAccessKey, "secretAccessKey"); this.sessionToken = sessionToken; - this.region = region; this.expirationEpochMillis = expirationEpochMillis; } @@ -64,10 +56,6 @@ public String getSessionToken() { return sessionToken; } - public String getRegion() { - return region; - } - public long getExpirationEpochMillis() { return expirationEpochMillis; } diff --git a/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java b/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java similarity index 54% rename from common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java rename to common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java index 7e8f5ee963..bc360e401d 100644 --- a/common/src/test/java/org/apache/comet/cloud/CometCloudCredentialDispatcherTest.java +++ b/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.cloud; +package org.apache.comet.cloud.s3; import org.junit.Before; import org.junit.Test; @@ -29,80 +29,80 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; -public class CometCloudCredentialDispatcherTest { +public class CometS3CredentialDispatcherTest { + + private static final int READ = CometS3AccessMode.READ.ordinal(); + private static final int WRITE = CometS3AccessMode.WRITE.ordinal(); @Before public void resetTestProvider() { - TestCometCloudCredentialProvider.reset(); + TestCometS3CredentialProvider.reset(); } @Test public void providerIsRegisteredFromTestClasspath() { - assertTrue(CometCloudCredentialDispatcher.isProviderRegistered()); + assertTrue(CometS3CredentialDispatcher.isProviderRegistered()); } @Test public void getCredentialsRoundTripsThroughProvider() throws Exception { - CometCredentials creds = - CometCloudCredentialDispatcher.getCredentialsForPath("my-bucket", "path/to/object", "READ"); + CometS3Credentials creds = + CometS3CredentialDispatcher.getCredentialsForPath("my-bucket", "path/to/object", READ); assertNotNull(creds); assertEquals("AKIATEST", creds.getAccessKeyId()); assertEquals("secret", creds.getSecretAccessKey()); assertEquals("session-tok", creds.getSessionToken()); - assertEquals("us-east-1", creds.getRegion()); assertEquals(0L, creds.getExpirationEpochMillis()); - assertEquals(1, TestCometCloudCredentialProvider.callCount.get()); - assertEquals("my-bucket", TestCometCloudCredentialProvider.lastBucket); - assertEquals("path/to/object", TestCometCloudCredentialProvider.lastPath); - assertEquals(CometAccessMode.READ, TestCometCloudCredentialProvider.lastMode); + assertEquals(1, TestCometS3CredentialProvider.callCount.get()); + assertEquals("my-bucket", TestCometS3CredentialProvider.lastBucket); + assertEquals("path/to/object", TestCometS3CredentialProvider.lastPath); + assertEquals(CometS3AccessMode.READ, TestCometS3CredentialProvider.lastMode); } @Test public void writeModeIsForwarded() throws Exception { - CometCloudCredentialDispatcher.getCredentialsForPath("b", "k", "WRITE"); - assertEquals(CometAccessMode.WRITE, TestCometCloudCredentialProvider.lastMode); + CometS3CredentialDispatcher.getCredentialsForPath("b", "k", WRITE); + assertEquals(CometS3AccessMode.WRITE, TestCometS3CredentialProvider.lastMode); } @Test public void unknownModeRejected() { assertThrows( IllegalArgumentException.class, - () -> CometCloudCredentialDispatcher.getCredentialsForPath("b", "k", "BOGUS")); + () -> CometS3CredentialDispatcher.getCredentialsForPath("b", "k", 99)); } @Test public void providerExceptionsPropagate() { IllegalStateException boom = new IllegalStateException("simulated STS failure"); - TestCometCloudCredentialProvider.throwOnNext = boom; + TestCometS3CredentialProvider.throwOnNext = boom; Exception thrown = assertThrows( Exception.class, - () -> CometCloudCredentialDispatcher.getCredentialsForPath("b", "k", "READ")); + () -> CometS3CredentialDispatcher.getCredentialsForPath("b", "k", READ)); assertSame(boom, thrown); } @Test - public void nullSessionTokenAndRegionAreAllowed() throws Exception { - TestCometCloudCredentialProvider.nextResult = - new CometCredentials("AKIA", "sec", null, null, 0L); + public void nullSessionTokenAllowed() throws Exception { + TestCometS3CredentialProvider.nextResult = new CometS3Credentials("AKIA", "sec", null, 0L); - CometCredentials creds = CometCloudCredentialDispatcher.getCredentialsForPath("b", "k", "READ"); + CometS3Credentials creds = CometS3CredentialDispatcher.getCredentialsForPath("b", "k", READ); assertNull(creds.getSessionToken()); - assertNull(creds.getRegion()); } @Test public void providerReceivesEachCallSeparately() throws Exception { - CometCloudCredentialDispatcher.getCredentialsForPath("b1", "k1", "READ"); - CometCloudCredentialDispatcher.getCredentialsForPath("b2", "k2", "READ"); - CometCloudCredentialDispatcher.getCredentialsForPath("b3", "k3", "READ"); + CometS3CredentialDispatcher.getCredentialsForPath("b1", "k1", READ); + CometS3CredentialDispatcher.getCredentialsForPath("b2", "k2", READ); + CometS3CredentialDispatcher.getCredentialsForPath("b3", "k3", READ); - assertEquals(3, TestCometCloudCredentialProvider.callCount.get()); - assertEquals("b3", TestCometCloudCredentialProvider.lastBucket); - assertEquals("k3", TestCometCloudCredentialProvider.lastPath); + assertEquals(3, TestCometS3CredentialProvider.callCount.get()); + assertEquals("b3", TestCometS3CredentialProvider.lastBucket); + assertEquals("k3", TestCometS3CredentialProvider.lastPath); } } diff --git a/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java b/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java similarity index 65% rename from common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java rename to common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java index 0774c849ca..f0429ebf1c 100644 --- a/common/src/test/java/org/apache/comet/cloud/TestCometCloudCredentialProvider.java +++ b/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java @@ -17,24 +17,23 @@ * under the License. */ -package org.apache.comet.cloud; +package org.apache.comet.cloud.s3; import java.util.concurrent.atomic.AtomicInteger; /** - * Test-only provider registered via {@code META-INF/services} in the test classpath. State is - * static because {@link CometCloudCredentialDispatcher} caches a single instance in a {@code static - * final} field for the JVM lifetime; tests reset state via {@link #reset()}. + * Test-only provider registered via {@code META-INF/services}. State is static because the + * dispatcher caches a single provider instance for the JVM lifetime. */ -public class TestCometCloudCredentialProvider implements CometCloudCredentialProvider { +public class TestCometS3CredentialProvider implements CometS3CredentialProvider { static final AtomicInteger callCount = new AtomicInteger(0); static volatile String lastBucket; static volatile String lastPath; - static volatile CometAccessMode lastMode; + static volatile CometS3AccessMode lastMode; static volatile RuntimeException throwOnNext; - static volatile CometCredentials nextResult = - new CometCredentials("AKIATEST", "secret", "session-tok", "us-east-1", 0L); + static volatile CometS3Credentials nextResult = + new CometS3Credentials("AKIATEST", "secret", "session-tok", 0L); static void reset() { callCount.set(0); @@ -42,11 +41,12 @@ static void reset() { lastPath = null; lastMode = null; throwOnNext = null; - nextResult = new CometCredentials("AKIATEST", "secret", "session-tok", "us-east-1", 0L); + nextResult = new CometS3Credentials("AKIATEST", "secret", "session-tok", 0L); } @Override - public CometCredentials getCredentialsForPath(String bucket, String path, CometAccessMode mode) { + public CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) { callCount.incrementAndGet(); lastBucket = bucket; lastPath = path; diff --git a/common/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider b/common/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider deleted file mode 100644 index fb1c912058..0000000000 --- a/common/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider +++ /dev/null @@ -1 +0,0 @@ -org.apache.comet.cloud.TestCometCloudCredentialProvider diff --git a/common/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider b/common/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider new file mode 100644 index 0000000000..cb6a1717e7 --- /dev/null +++ b/common/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider @@ -0,0 +1 @@ +org.apache.comet.cloud.s3.TestCometS3CredentialProvider diff --git a/dev/ci/check-suites.py b/dev/ci/check-suites.py index abc67280db..b7369d1707 100644 --- a/dev/ci/check-suites.py +++ b/dev/ci/check-suites.py @@ -35,7 +35,7 @@ def file_to_class_name(path: Path) -> str | None: "org.apache.comet.parquet.ParquetReadSuite", # abstract "org.apache.comet.parquet.ParquetReadFromS3Suite", # manual test suite "org.apache.comet.IcebergReadFromS3Suite", # manual test suite - "org.apache.comet.cloud.CometCloudCredentialBridgeS3Suite", # manual test suite + "org.apache.comet.cloud.s3.CometS3CredentialBridgeSuite", # manual test suite "org.apache.spark.sql.comet.CometPlanStabilitySuite", # abstract "org.apache.spark.sql.comet.ParquetDatetimeRebaseSuite", # abstract "org.apache.comet.exec.CometColumnarShuffleSuite" # abstract diff --git a/docs/source/contributor-guide/cloud-credential-providers.md b/docs/source/contributor-guide/cloud-credential-providers.md deleted file mode 100644 index 7cdf21df99..0000000000 --- a/docs/source/contributor-guide/cloud-credential-providers.md +++ /dev/null @@ -1,343 +0,0 @@ - - -# Cloud Credential Providers - -Comet's native S3 readers can route every credential request to a vendor-supplied Java provider -loaded via `java.util.ServiceLoader`. This page is the integration contract for vendors writing -that bridge. - -If you are an operator deciding whether to enable this in your cluster, see the user guide page -on cloud credential providers instead. - -## Why this SPI exists - -Comet runs queries that bypass Spark's Hadoop S3A code path entirely. Native Parquet scans go -through the Rust `object_store` crate directly; native Iceberg scans go through `iceberg-rust` and -`opendal`. Neither path ever calls a Hadoop `Signer` or `AWSCredentialsProvider`. That means none -of the credential infrastructure Spark and Hadoop already configured for your cluster is reachable -from Comet's native code. - -For the simple case (static credentials, EC2 instance profiles, environment variables, the default -AWS credential chain) Comet reproduces that resolution natively in -`native/core/src/parquet/objectstore/s3.rs`. No SPI is needed; the existing default chain works. - -This SPI exists for the case the default chain _cannot_ express: a vendor-managed exchange that -takes a per-request path or token and returns short-lived STS credentials. - -The reasons that case can't be served by an existing API: - -- **`org.apache.spark.deploy.security.cloud.CloudCredentialsProvider`** yields a single auth proof - (typically a JWT) per service name. It has no path argument and no notion of returning AWS - credentials. It is the right place to obtain a JWT, but not to exchange one. -- **Hadoop S3A's custom signer mechanism** keeps path-aware logic inside the - `Signer.sign(request, credentials)` call. The standard `AWSCredentialsProvider.getCredentials()` - contract is parameterless, so vendors that need per-path STS lookup hide that lookup inside the - signer. The credential is never returned outside the AWS SDK's signing pipeline; even running - the signer on a synthesized request does not recover the underlying credential, because the - secret access key is an HMAC key, not a value present in the signed output. -- **Reflecting into vendor singletons** would require Comet to encode a vendor's class name, field - name, lazy-init lifecycle, and method signatures. Each vendor differs; each version may rename - things. That shifts the maintenance burden from the vendor to Comet and creates silent breakage - on upgrades. -- **A Comet-specific HTTP STS endpoint contract** would require vendors to expose and stabilize an - HTTP API just for Comet. Most vendors ship this logic as Java code, not as a public HTTP API, - and asking them to do otherwise is a larger change than a small Java class. - -Comet's SPI is a peer to two contracts vendors with full integration coverage already implement: - -| Path | Vendor implements | -| ----------------- | ----------------------------------------------------------- | -| Hadoop S3A | `org.apache.hadoop.fs.s3a.AwsSignerInitializer` plus signer | -| Iceberg-Java | `org.apache.iceberg.aws.AwsClientFactory` | -| Comet (this page) | `org.apache.comet.cloud.CometCloudCredentialProvider` | - -Adding a Comet implementation is the same shape as the first two with a smaller surface (one -method). - -## SPI contract - -Implement `org.apache.comet.cloud.CometCloudCredentialProvider`: - -```java -package org.apache.comet.cloud; - -public interface CometCloudCredentialProvider { - CometCredentials getCredentialsForPath( - String bucket, String path, CometAccessMode mode) throws Exception; -} -``` - -`getCredentialsForPath` may be invoked concurrently from many native tokio worker threads. -Implementations must be thread-safe. - -The `mode` is the access intent for this credential request: - -| Value | Used for | -| ------- | -------------------------------------------------------------------------- | -| `READ` | All native scan paths (raw Parquet, Iceberg). Comet today only sends READ. | -| `WRITE` | Reserved for future native write paths (Iceberg writes, native INSERT). | - -The SPI does not promise that a `WRITE` credential is also read-capable; vendors that need -read-during-write workflows (multipart-completion HEAD, Iceberg manifest reads on the write path, -etc.) include the necessary read permissions in the IAM policy attached to their `WRITE` -credentials. - -The returned `CometCredentials` POJO carries: - -| Field | Type | Notes | -| ----------------------- | ------------------- | ------------------------------------------------------ | -| `accessKeyId` | `String` (non-null) | Required. | -| `secretAccessKey` | `String` (non-null) | Required. | -| `sessionToken` | `String` (nullable) | Pass `null` for non-STS credentials. | -| `region` | `String` (nullable) | `null` lets the native reader fall back to its config. | -| `expirationEpochMillis` | `long` | `0` means "unknown" (see expiration semantics below). | - -### Why this SPI returns or throws (no fallthrough return value) - -The SPI follows the same shape as every other AWS-credential SPI in the JVM ecosystem: - -| SPI | Method | Behavior | -| ---------------------------------------- | ------------------------------------------- | --------------------------- | -| AWS SDK v1 `AWSCredentialsProvider` | `getCredentials()` | returns or throws | -| AWS SDK v2 `AwsCredentialsProvider` | `resolveCredentials()` | returns or throws | -| Hadoop S3A `AWSCredentialsProvider` | `getCredentials()` | returns or throws | -| Iceberg `VendedCredentialsProvider` | `resolveCredentials()` | returns or throws | -| Iceberg `AwsClientFactory` | `s3()` | returns a configured client | -| **Comet `CometCloudCredentialProvider`** | `getCredentialsForPath(bucket, path, mode)` | **returns or throws** | - -Chaining/fallback in the AWS world is a separate concern, composed _outside_ the provider — -e.g. AWS SDK v2's `AwsCredentialsProviderChain.builder().credentialsProviders(...)`, or the Hadoop -S3A `fs.s3a.aws.credentials.provider` comma-separated list resolved by -`AWSCredentialsProviderChain`. Each _individual_ provider is atomic ("give me a credential or -fail"); a chain class composes them. - -Comet's SPI keeps to this convention. We considered an alternative `Optional` -return where empty would mean "fall through to Comet's native default AWS chain" — convenient -for path-scoped vendors but inconsistent with everything else at this layer. Composition is -cheap to do vendor-side and keeps the SPI surface narrow. - -#### Pattern: vendor that handles only a subset of paths - -If your provider is authoritative only for a subset of buckets/prefixes (e.g. you have indexed -policies for some paths and want others to use the host's default AWS credentials), construct -the default chain in your provider and return its credentials directly: - -```java -import software.amazon.awssdk.auth.credentials.AwsCredentials; -import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; - -public final class PathScopedProvider implements CometCloudCredentialProvider { - - private final DefaultCredentialsProvider defaultChain = DefaultCredentialsProvider.create(); - - @Override - public CometCredentials getCredentialsForPath( - String bucket, String path, CometAccessMode mode) throws Exception { - if (handlesPath(bucket, path)) { - return mintFromMyVendorService(bucket, path, mode); - } - AwsCredentials c = defaultChain.resolveCredentials(); - String token = (c instanceof AwsSessionCredentials) - ? ((AwsSessionCredentials) c).sessionToken() - : null; - return new CometCredentials(c.accessKeyId(), c.secretAccessKey(), token, null, 0L); - } -} -``` - -This mirrors how Iceberg's `VendedCredentialsProvider` is implemented (it composes -`HTTPClient` + `CachedSupplier` internally; the SPI itself just returns or throws). Pick AWS SDK -v1 vs v2 to match whatever your existing signer / `AwsClientFactory` integration uses — the -choice is invisible to Comet because only `CometCredentials` (a plain POJO with strings + a long) -crosses the JNI boundary. - -### Expiration semantics - -`expirationEpochMillis` is the absolute expiry of the returned credential, in milliseconds since -the Unix epoch. - -- For credentials with a known expiry, set this to the actual expiry. The Iceberg path uses it to - decide when `opendal` must re-call your provider for a fresh credential. -- `0` is treated as "unknown". The Iceberg path then defaults to a 5-minute refresh interval to - bound how long opendal can cache a possibly-stale credential. Spark jobs running for hours or - days with cached stale credentials would otherwise fail silently mid-task; the 5-minute floor - is a safety net, not a recommendation. **Provide a real expiry whenever you have one.** -- The `object_store` path (raw `s3://` Parquet scans) ignores expiration today and re-fetches per - request. - -### Error semantics - -- Throwing from `getCredentialsForPath` aborts the native S3 request and surfaces the exception - message (and chained causes) to the caller. (See the iceberg-path note below for one current - exception to that propagation.) -- Returning `null` is a contract violation; the native bridge surfaces it as a request failure. - Implementations that want to defer to a default credential chain on a subset of paths should - resolve the default chain themselves and return its credentials — see the worked pattern under - _Why this SPI returns or throws_ above. - -#### Iceberg path: error message fidelity - -When the bridge is wired into `iceberg-rust` (Iceberg native scans), iceberg-rust currently wraps -our credential loader in its own `ProvideCredentialChain`. That outer chain swallows errors into -"no credential" before the request reaches opendal (see `reqsign-core::ProvideCredentialChain`), -so a thrown exception surfaces as an opaque opendal/anonymous-request failure rather than your -exception message. The credential is still not issued and the request still fails — only the -message is degraded. - -This is tied to opendal's chain semantics. It would resolve if iceberg-rust either (a) stops -wrapping custom loaders in its own outer chain, or (b) moves Iceberg's S3 storage backend to -`object_store` (whose `CredentialProvider` has no chain-swallow behavior — auth failures -propagate cleanly as they do on Comet's raw Parquet path today). No Comet SPI change is needed -in either case. - -### Iceberg path: explicit S3 region required - -When the bridge is registered, Comet wires it into `iceberg-storage-opendal` as a -`CustomAwsCredentialLoader`. opendal then requires explicit S3 region (and, for non-AWS -endpoints, an explicit endpoint) on the catalog properties — its built-in region auto-detection -only runs when no custom credential loader is configured. - -In practice this means deployments using the bridge for Iceberg must set, on the Spark catalog: - -``` -spark.sql.catalog..s3.region = us-east-1 (or your real region) -spark.sql.catalog..s3.endpoint = https://... (only for non-AWS) -spark.sql.catalog..s3.path-style-access = true (only for path-style endpoints) -``` - -If a deployment hits `region is missing. Please find it by S3::detect_region() or set them -in env`, this is the missing config. - -## Discovery - -Discovery is purely classpath-based. Register your implementation by adding a service file: - -``` -META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider -``` - -containing the fully-qualified name of your implementation class. - -Resolution rules at JVM startup: - -- **Zero impls registered** — the native readers fall through to the existing AWS credential chain - (the same behavior as a Comet without your jar on the classpath). -- **Exactly one impl registered** — cached and used for every credential request. -- **Multiple impls registered** — `CometCloudCredentialDispatcher` throws - `IllegalStateException` at class-load with the FQN of every impl found. The executor fails - loudly. This is intentional; pick one bridge jar. - -There is no Comet-specific config knob for selecting a provider. Discovery follows the same source -of truth Spark itself reads, so a query that falls back from Comet to Spark mid-execution sees -identical credentials. - -A vendor that wants to compose multiple credential sources (per-path STS for some prefixes, a -catchall provider for others) does so inside their single provider implementation — same as how -AWS SDK v1/v2 providers compose into `AwsCredentialsProviderChain` at the call site rather than -exposing chain semantics through the individual provider contract. See _Why this SPI returns or -throws_ above for a worked example. - -## Threading and lifecycle - -- The dispatcher caches the resolved provider in a `static final` field for the JVM lifetime. Your - provider is constructed once per executor and reused for every request. -- The dispatcher itself caches no credentials. Every native call dispatches through JNI on a tokio - worker thread, so any STS / token refresh logic must live inside your provider. -- Implementations should be cheap to construct (no long blocking work in the no-arg constructor) - and thread-safe. - -## Spark and AWS SDK version selection - -Vendors that need to pick between Spark 3 and Spark 4 implementations, or between AWS SDK v1 and -v2 backends, do so inside their own provider class. Comet is unaware of either. Common patterns -include reflectively probing for an SDK class on the classpath, or shipping two service files (one -per Spark profile) that each register a different provider class. - -## Build setup - -Vendor implementations need the Comet SPI classes at compile time only. The standard pattern is -a `provided`-scope Maven dependency on `comet-common`: - -```xml - - org.apache.datafusion - comet-common-spark${spark.version.short}_${scala.binary.version} - ${comet.version} - provided - -``` - -`provided` scope means: - -- Compile-time only — your jar resolves the `CometCloudCredentialProvider` interface and - `CometCredentials` POJO for compilation. -- No runtime bundling — your jar does not ship Comet classes; no fat-jar bloat, no version - conflict on the executor classpath. -- The API classes are already present at runtime because Comet itself is loaded (that's the - whole reason the bridge exists). - -Same pattern vendors already use for implementing Hadoop S3A `AwsSignerInitializer` and Iceberg -`AwsClientFactory` — Comet is a third entry in the same list, not a different shape. - -## Worked example - -A minimal static-credential provider, suitable for tests and development: - -```java -package com.example.comet.test; - -import org.apache.comet.cloud.CometAccessMode; -import org.apache.comet.cloud.CometCloudCredentialProvider; -import org.apache.comet.cloud.CometCredentials; - -public final class StaticCometCredentialProvider implements CometCloudCredentialProvider { - private static final String ACCESS_KEY = System.getenv("EXAMPLE_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("EXAMPLE_SECRET_KEY"); - private static final String REGION = System.getenv().getOrDefault("EXAMPLE_REGION", "us-east-1"); - - @Override - public CometCredentials getCredentialsForPath( - String bucket, String path, CometAccessMode mode) { - return new CometCredentials(ACCESS_KEY, SECRET_KEY, null, REGION, 0L); - } -} -``` - -Register it via `META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider`: - -``` -com.example.comet.test.StaticCometCredentialProvider -``` - -A real provider would read a JWT from `SparkConf` (typically populated by a Spark -`CloudCredentialsProvider`), call its STS-vending service with the `(bucket, path)` tuple, and -return the resulting credentials with their actual expiry. - -## Where this lives in Comet - -| Component | Location | -| --------------------------------------------------------- | --------------------------------------------------------------------------------- | -| SPI interface | `common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java` | -| POJO | `common/src/main/java/org/apache/comet/cloud/CometCredentials.java` | -| Dispatcher (called from native via JNI) | `common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java` | -| Rust JNI bridge and `object_store::CredentialProvider` | `native/core/src/parquet/objectstore/comet_credential_bridge.rs` | -| `s3.rs` injection point (DataSourceExec / Parquet scans) | `native/core/src/parquet/objectstore/s3.rs` | -| Iceberg scan injection point (`iceberg-rust` + `opendal`) | `native/core/src/execution/operators/iceberg_scan.rs` | diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index 85eb2378c7..f3bbfba044 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -47,7 +47,6 @@ Benchmarking Guide Adding a New Operator Adding a New Expression Adding a New Spark Version -Cloud Credential Providers Supported Spark Expressions Supported Spark Configurations Tracing diff --git a/docs/source/user-guide/latest/cloud-credential-providers.md b/docs/source/user-guide/latest/cloud-credential-providers.md index 51f69017f1..7a39c84cfb 100644 --- a/docs/source/user-guide/latest/cloud-credential-providers.md +++ b/docs/source/user-guide/latest/cloud-credential-providers.md @@ -36,85 +36,169 @@ You don't, if any of the following describe your cluster: You probably do, if any of these are true: - You have a Hadoop S3A custom signer configured (`fs.s3a.custom.signers=...`). -- You have a Spark `CloudCredentialsProvider` configured - (`spark.security.credentials.providerList=...`) that issues a JWT for your cluster's STS - service. +- You have a Spark `CloudCredentialsProvider` that issues a JWT for a vendor STS service. - You have a custom Iceberg `client.factory` that injects a configured S3 client. -- Spark queries against your S3 paths work, but the same queries with Comet enabled fail. +- Spark queries against your S3 paths work, but the same queries with Comet enabled fail with 403. -## How to tell - -If you suspect you need this, two symptoms confirm it: - -- **Comet S3 reads fail with 403 AccessDenied** while the same query without Comet succeeds. -- **Comet falls back to Spark execution** silently for scans against the affected paths. - -If both Spark and Comet succeed against your S3 paths, you do not need a bridge JAR. - -## How to enable it +## Enabling a bridge JAR Add the vendor-supplied bridge JAR to your Spark executor classpath: ```sh -spark-submit \ - --jars vendor-comet-bridge.jar \ - ... +spark-submit --jars vendor-comet-bridge.jar ... ``` -Or via `spark.jars` in your `spark-defaults.conf`: +Or via `spark.jars`: ``` spark.jars=/path/to/vendor-comet-bridge.jar ``` -That is the entire enablement step. There are no Spark or Comet config keys to set. Comet -discovers the bridge through `META-INF/services` on the classpath at executor startup. +Comet discovers the bridge through `META-INF/services` at executor startup. There are no Comet +config keys to set. -## Where to get the bridge JAR - -From the same vendor that supplies your Hadoop S3A signer or Iceberg client factory. OSS Comet -deliberately ships no vendor-specific bridges; the SPI is a contract, not a built-in. If your -authorization vendor does not yet provide a Comet bridge, refer them to the contributor guide page -on cloud credential providers. +OSS Comet ships no vendor-specific bridges. Get one from the same vendor that supplies your +Hadoop S3A signer or Iceberg client factory. If they do not yet provide one, send them to the +"Writing a bridge" section below. ## Verification -When the bridge is on the executor classpath and being used you should see, in the executor -logs: +With the bridge on the classpath, executor logs show: -- At startup: - `Registered CometCloudCredentialProvider: ` -- The first time Comet performs an S3 access: - `Fetching credentials for bucket=<...> path=<...>` (at debug level) +- At startup: `Registered CometS3CredentialProvider: ` +- On first S3 access (debug level): `Fetching credentials for bucket=... path=... mode=...` -If neither log line appears and you expected the bridge to be in use, the JAR is missing from the -executor classpath or its `META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider` -entry is missing. +Without a bridge registered you get exactly one line at startup: -When the bridge is _not_ registered (the default), you will see exactly one line at startup: +``` +No CometS3CredentialProvider registered; native S3 readers will use the default AWS credential chain +``` + +## Troubleshooting + +**`Multiple CometS3CredentialProvider impls on classpath: [...]`** at startup. Remove all but one +bridge JAR. Comet does not chain providers; it fails fast to prevent silent ambiguity. + +**`No CometS3CredentialProvider registered`** combined with `403 AccessDenied`. The bridge JAR is +not on the executor classpath. Re-check `--jars` / `spark.jars`. On YARN or Kubernetes, confirm +the JAR actually reached the executor and not only the driver. + +**Credentials silently going stale during long-running jobs.** The bridge caps opendal's +credential cache at 5 minutes when the vendor does not populate `expirationEpochMillis`. Ask the +vendor to return a real expiry; the 5-minute floor is a safety net, not a knob. + +## Iceberg: explicit S3 region required + +With the bridge registered, Comet wires a custom credential loader into `iceberg-storage-opendal`. +opendal's built-in S3 region auto-detection only runs when no custom loader is configured, so on +the bridge path the region (and endpoint for non-AWS) must be set explicitly on the Spark catalog: ``` -No CometCloudCredentialProvider registered; native S3 readers will use the default AWS -credential chain +spark.sql.catalog..s3.region = us-east-1 +spark.sql.catalog..s3.endpoint = https://... (non-AWS only) +spark.sql.catalog..s3.path-style-access = true (path-style endpoints only) ``` -This is the expected behavior for clusters that do not need the bridge. +If you hit `region is missing. Please find it by S3::detect_region() or set them in env`, this +is the missing config. -## Troubleshooting +## Writing a bridge -**`Multiple CometCloudCredentialProvider impls on classpath: [...]`** at executor startup. Two or -more bridge JARs were found. Remove all but one. Comet does not chain providers; it fails fast to -prevent silent ambiguity. +Comet's native scan paths (`object_store` for raw Parquet, `opendal` via `iceberg-rust` for +Iceberg) bypass Hadoop S3A entirely. The standard `AWSCredentialsProvider.getCredentials()` has no +path argument, so vendors that issue per-path STS credentials cannot expose them through it. The +`CometS3CredentialProvider` SPI fills that gap. -**`No CometCloudCredentialProvider registered`** combined with `403 AccessDenied`. The bridge JAR -is not on the executor classpath. Re-check `--jars` / `spark.jars`. On YARN or Kubernetes, confirm -the JAR was actually shipped to the executor pods and not only available on the driver. +Implement `org.apache.comet.cloud.s3.CometS3CredentialProvider`: + +```java +package org.apache.comet.cloud.s3; + +public interface CometS3CredentialProvider { + CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) throws Exception; +} +``` + +Register via `META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider` with the +fully-qualified class name. `getCredentialsForPath` may be invoked concurrently from many native +tokio worker threads; the implementation must be thread-safe. + +### Returned fields + +| Field | Notes | +| ----------------------- | ------------------------------------------------- | +| `accessKeyId` | Required. | +| `secretAccessKey` | Required. | +| `sessionToken` | `null` for non-STS credentials. | +| `expirationEpochMillis` | Absolute expiry. `0` means "unknown" (see below). | + +Provide a real `expirationEpochMillis` whenever you have one. The Iceberg path uses it to decide +when `opendal` must re-call the provider for a fresh credential. `0` is treated as unknown and the +Iceberg path defaults to a 5-minute refresh to bound staleness. + +### Returns or throws + +The SPI follows the same shape as the other JVM AWS-credential SPIs (AWS SDK v1/v2, +Hadoop S3A, Iceberg `VendedCredentialsProvider`): return credentials or throw. There is no +"fall-through" return value. Chaining is a vendor-side concern. + +If your provider is authoritative only for some paths, resolve the default AWS chain yourself for +the rest: + +```java +private final DefaultCredentialsProvider defaultChain = DefaultCredentialsProvider.create(); + +@Override +public CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) throws Exception { + if (handlesPath(bucket, path)) { + return mintFromMyVendorService(bucket, path, mode); + } + AwsCredentials c = defaultChain.resolveCredentials(); + String token = (c instanceof AwsSessionCredentials) + ? ((AwsSessionCredentials) c).sessionToken() + : null; + return new CometS3Credentials(c.accessKeyId(), c.secretAccessKey(), token, 0L); +} +``` + +### Access mode + +| Value | Used for | +| ------- | -------------------------------------------------------------------------- | +| `READ` | All native scan paths (raw Parquet, Iceberg). Comet today only sends READ. | +| `WRITE` | Reserved for future native write paths. | + +A `WRITE` credential is not implicitly read-capable. Vendors that need read-during-write +workflows include the required read permissions in the IAM policy attached to their `WRITE` +credentials. + +### Discovery rules + +- Zero impls registered: native readers use the default AWS credential chain. +- One impl registered: cached and used for every request. +- Multiple impls registered: `CometS3CredentialDispatcher` throws `IllegalStateException` at + class-load. Pick one bridge JAR. + +### Build setup + +Vendor implementations need the Comet SPI classes at compile time only. Use `provided`-scope: + +```xml + + org.apache.datafusion + comet-common-spark${spark.version.short}_${scala.binary.version} + ${comet.version} + provided + +``` -**Slow first query, fast subsequent queries.** Expected. The vendor's STS-vending service is -likely cold-started; subsequent fetches reuse cached credentials inside the vendor's provider. +### Iceberg path: error message fidelity -**Credentials silently going stale during long-running jobs.** The bridge defaults to a 5-minute -maximum cache window in the Iceberg / opendal path when the vendor does not specify a credential -expiry. If you suspect the vendor is not setting an expiry, ask them to populate -`expirationEpochMillis` on every returned credential. The 5-minute floor is a safety net, not a -configurable knob. +When the bridge is wired into `iceberg-rust`, the outer `reqsign-core::ProvideCredentialChain` +currently swallows thrown exceptions into "no credential" before the request reaches opendal. The +credential is still not issued and the request still fails, only the message is degraded to an +opaque anonymous-request failure. No Comet change fixes this; it is resolved upstream if +`iceberg-rust` stops wrapping custom loaders in its outer chain or moves its S3 backend to +`object_store`. diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 6de8a1e261..217a28a699 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -44,7 +44,9 @@ use iceberg_storage_opendal::CustomAwsCredentialLoader; use iceberg_storage_opendal::OpenDalStorageFactory; use crate::execution::operators::ExecutionError; -use crate::parquet::objectstore::comet_credential_bridge::{self, CometCredentialBridge}; +use crate::parquet::objectstore::comet_s3_credential_bridge::{ + AccessMode, CometS3CredentialBridge, +}; use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkPhysicalExprAdapterFactory; use datafusion_comet_spark_expr::EvalMode; @@ -241,18 +243,8 @@ impl IcebergScanExec { /// Wires the registered Comet credential provider into opendal's S3 service for this scan, or /// returns `None` so opendal falls back to its default credential chain. fn build_s3_credential_loader(metadata_location: &str) -> Option { - if !comet_credential_bridge::is_provider_registered() { - return None; - } let url = url::Url::parse(metadata_location).ok()?; - let bucket = url.host_str()?.to_string(); - // Pass the URL path component (matches `s3.rs::create_store`); SPI vendors expect the - // (bucket, path) pair to compose into a single URI without re-parsing a scheme out. - let bridge = CometCredentialBridge::new( - bucket, - url.path().to_string(), - comet_credential_bridge::AccessMode::Read, - ); + let bridge = CometS3CredentialBridge::for_url(&url, AccessMode::Read)?; Some(CustomAwsCredentialLoader::new(bridge)) } @@ -420,7 +412,7 @@ fn adapt_batch_with_expressions( return Ok(batch); } - // Zero-column projection (e.g. SELECT count(*)) — preserve row count + // Zero-column projection (e.g. SELECT count(*)), preserve row count if projection_exprs.is_empty() { return Ok(RecordBatch::try_new_with_options( Arc::clone(target_schema), diff --git a/native/core/src/parquet/objectstore/comet_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_credential_bridge.rs deleted file mode 100644 index c34d92770b..0000000000 --- a/native/core/src/parquet/objectstore/comet_credential_bridge.rs +++ /dev/null @@ -1,337 +0,0 @@ -// 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. - -//! JNI bridge to the Java `CometCloudCredentialDispatcher` SPI for per-request AWS credentials. -//! -//! See `common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java` for the -//! Java side and the architecture diagram. JNI handles are cached on `JVMClasses` next to all -//! the other Comet JNI bridges; this file holds only the Rust trait impls that delegate through. -//! -//! Contract: the SPI returns credentials or throws. There is no "fall through" return value. See -//! `docs/source/contributor-guide/cloud-credential-providers.md` for the rationale and patterns -//! for vendors that need to defer to a default credential chain on a subset of paths. - -use crate::execution::operators::ExecutionError; -use crate::jvm_bridge::{check_exception, JVMClasses}; -use crate::JAVA_VM; -use async_trait::async_trait; -use iceberg_storage_opendal::AwsCredential as IcebergAwsCredential; -use jni::objects::{JFieldID, JObject, JString, JValue}; -use jni::signature::{Primitive, ReturnType}; -use log::{debug, warn}; -use object_store::aws::AwsCredential; -use object_store::CredentialProvider; -use once_cell::sync::OnceCell; -use reqsign_core::time::Timestamp; -use reqsign_core::{ - Context, Error as ReqsignError, ErrorKind as ReqsignErrorKind, - ProvideCredential as IcebergProvideCredential, -}; -use std::sync::atomic::{AtomicBool, Ordering}; -use std::sync::Arc; -use std::time::Duration; - -/// Bound on opendal's credential cache when the Java provider returns `expirationEpochMillis = 0` -/// ("unknown"). Without this, opendal would hold the credential for the entire executor lifetime, -/// a silent footgun for Spark jobs that run for hours. Five minutes trades a small JNI-call -/// cadence for a tight staleness bound. Vendors that know the real expiry should set it. -const DEFAULT_EXPIRY_WHEN_UNKNOWN: Duration = Duration::from_secs(300); - -/// Cached "is a Java provider registered?" answer. Resolution is one JNI round-trip and the -/// result never changes within a JVM lifetime, so memoize. -static PROVIDER_REGISTERED: OnceCell = OnceCell::new(); - -/// Access intent passed to the Java SPI. Mirrors `CometAccessMode` on the Java side; the JVM-side -/// `valueOf` parses these names so they are part of the cross-language contract. -#[derive(Debug, Clone, Copy)] -pub enum AccessMode { - Read, - /// Reserved for future native write paths (Iceberg writes, native INSERT). No call site - /// constructs this yet; allow it to keep the enum complete and to lock the JVM-side - /// `CometAccessMode.valueOf("WRITE")` contract. - #[allow(dead_code)] - Write, -} - -impl AccessMode { - fn as_jvm_str(self) -> &'static str { - match self { - AccessMode::Read => "READ", - AccessMode::Write => "WRITE", - } - } -} - -/// True iff a `CometCloudCredentialProvider` was discovered on the JVM classpath. Used by -/// `s3.rs::create_store` and `iceberg_scan.rs` to decide whether to wire a [`CometCredentialBridge`] -/// in front of the default credential paths. -pub fn is_provider_registered() -> bool { - *PROVIDER_REGISTERED.get_or_init(|| { - // Unit tests construct stores without a JVM; treat that as "no provider registered" so we - // don't trip `with_env`'s debug_assert. In production the JVM is always initialized before - // any object_store is built. - if JAVA_VM.get().is_none() { - return false; - } - JVMClasses::with_env(|env| -> Result { - let dispatcher = &JVMClasses::get().comet_cloud_credential_dispatcher; - let result = unsafe { - env.call_static_method_unchecked( - &dispatcher.class, - dispatcher.method_is_provider_registered, - dispatcher.method_is_provider_registered_ret, - &[], - ) - } - .map_err(|e| { - ExecutionError::GeneralError(format!("isProviderRegistered call failed: {e}")) - })?; - if let Some(exception) = check_exception(env) - .map_err(|e| ExecutionError::GeneralError(format!("Exception check failed: {e}")))? - { - return Err(ExecutionError::GeneralError(format!( - "Java exception in isProviderRegistered: {exception}" - ))); - } - result.z().map_err(|e| { - ExecutionError::GeneralError(format!("isProviderRegistered did not return Z: {e}")) - }) - }) - .unwrap_or_else(|e| { - debug!( - "CometCloudCredentialDispatcher.isProviderRegistered failed; \ - native S3 readers will use the default AWS credential chain: {e}" - ); - false - }) - }) -} - -/// Per-request credential provider that delegates to the Java SPI via JNI. -/// -/// One instance is constructed per S3 store (per-URL in `create_store`) or per FileIO (the -/// metadata location, in `iceberg_scan.rs`). The `(bucket, path, mode)` tuple is forwarded -/// verbatim on every credential fetch. -#[derive(Debug)] -pub struct CometCredentialBridge { - bucket: String, - path: String, - mode: AccessMode, - /// Latched once the bridge observes a credential without an expiry, so the warning that - /// goes with [`DEFAULT_EXPIRY_WHEN_UNKNOWN`] only fires once per bridge instance instead of - /// per request. - warned_missing_expiry: AtomicBool, -} - -impl CometCredentialBridge { - pub fn new(bucket: impl Into, path: impl Into, mode: AccessMode) -> Self { - Self { - bucket: bucket.into(), - path: path.into(), - mode, - warned_missing_expiry: AtomicBool::new(false), - } - } - - /// Single JNI round-trip to the dispatcher; both async trait impls share this. - fn fetch_raw(&self) -> Result { - JVMClasses::with_env(|env| -> Result { - let dispatcher = &JVMClasses::get().comet_cloud_credential_dispatcher; - - let bucket_jstr = env - .new_string(&self.bucket) - .map_err(|e| ExecutionError::GeneralError(format!("new_string(bucket): {e}")))?; - let path_jstr = env - .new_string(&self.path) - .map_err(|e| ExecutionError::GeneralError(format!("new_string(path): {e}")))?; - let mode_jstr = env - .new_string(self.mode.as_jvm_str()) - .map_err(|e| ExecutionError::GeneralError(format!("new_string(mode): {e}")))?; - - let result = unsafe { - env.call_static_method_unchecked( - &dispatcher.class, - dispatcher.method_get_credentials_for_path, - dispatcher.method_get_credentials_for_path_ret, - &[ - JValue::from(&bucket_jstr).as_jni(), - JValue::from(&path_jstr).as_jni(), - JValue::from(&mode_jstr).as_jni(), - ], - ) - }; - - if let Some(exception) = check_exception(env) - .map_err(|e| ExecutionError::GeneralError(format!("Exception check failed: {e}")))? - { - return Err(ExecutionError::GeneralError(format!( - "Java exception in getCredentialsForPath: {exception}" - ))); - } - - let creds_obj = result - .map_err(|e| { - ExecutionError::GeneralError(format!("getCredentialsForPath JNI call: {e}")) - })? - .l() - .map_err(|e| { - ExecutionError::GeneralError(format!( - "getCredentialsForPath did not return an object: {e}" - )) - })?; - - if creds_obj.is_null() { - return Err(ExecutionError::GeneralError( - "getCredentialsForPath returned null (contract violation)".to_string(), - )); - } - - Ok(RawCredentials { - access_key_id: read_required_string( - env, - &creds_obj, - dispatcher.field_access_key_id, - "accessKeyId", - )?, - secret_access_key: read_required_string( - env, - &creds_obj, - dispatcher.field_secret_access_key, - "secretAccessKey", - )?, - session_token: read_optional_string( - env, - &creds_obj, - dispatcher.field_session_token, - )?, - expiration_epoch_millis: unsafe { - env.get_field_unchecked( - &creds_obj, - dispatcher.field_expiration_epoch_millis, - ReturnType::Primitive(Primitive::Long), - ) - } - .map_err(|e| { - ExecutionError::GeneralError(format!("read expirationEpochMillis: {e}")) - })? - .j() - .map_err(|e| { - ExecutionError::GeneralError(format!("expirationEpochMillis not a long: {e}")) - })?, - }) - }) - } -} - -struct RawCredentials { - access_key_id: String, - secret_access_key: String, - session_token: Option, - /// Provider-supplied absolute expiry. `0` means the provider didn't say; callers translate - /// that into a short fallback so opendal can't cache a stale credential indefinitely. - expiration_epoch_millis: i64, -} - -#[async_trait] -impl CredentialProvider for CometCredentialBridge { - type Credential = AwsCredential; - - async fn get_credential(&self) -> object_store::Result> { - let raw = self.fetch_raw().map_err(|e| object_store::Error::Generic { - store: "S3", - source: e.to_string().into(), - })?; - Ok(Arc::new(AwsCredential { - key_id: raw.access_key_id, - secret_key: raw.secret_access_key, - token: raw.session_token, - })) - } -} - -impl IcebergProvideCredential for CometCredentialBridge { - type Credential = IcebergAwsCredential; - - async fn provide_credential( - &self, - _ctx: &Context, - ) -> reqsign_core::Result> { - let raw = self - .fetch_raw() - .map_err(|e| ReqsignError::new(ReqsignErrorKind::CredentialInvalid, e.to_string()))?; - - let expires_in = if raw.expiration_epoch_millis > 0 { - Some( - Timestamp::from_millisecond(raw.expiration_epoch_millis).map_err(|e| { - ReqsignError::new( - ReqsignErrorKind::CredentialInvalid, - format!( - "Invalid expirationEpochMillis {}: {e}", - raw.expiration_epoch_millis - ), - ) - })?, - ) - } else { - if !self.warned_missing_expiry.swap(true, Ordering::Relaxed) { - warn!( - "CometCloudCredentialProvider returned credentials without expiration for \ - bucket={} path={}; defaulting to {}s expiry to bound opendal caching", - self.bucket, - self.path, - DEFAULT_EXPIRY_WHEN_UNKNOWN.as_secs() - ); - } - Some(Timestamp::now() + DEFAULT_EXPIRY_WHEN_UNKNOWN) - }; - - Ok(Some(IcebergAwsCredential { - access_key_id: raw.access_key_id, - secret_access_key: raw.secret_access_key, - session_token: raw.session_token, - expires_in, - })) - } -} - -fn read_required_string( - env: &mut jni::Env, - instance: &JObject, - field: JFieldID, - name: &str, -) -> Result { - read_optional_string(env, instance, field)? - .ok_or_else(|| ExecutionError::GeneralError(format!("{name} was null"))) -} - -fn read_optional_string( - env: &mut jni::Env, - instance: &JObject, - field: JFieldID, -) -> Result, ExecutionError> { - let value = unsafe { env.get_field_unchecked(instance, field, ReturnType::Object) } - .map_err(|e| ExecutionError::GeneralError(format!("get_field_unchecked: {e}")))? - .l() - .map_err(|e| ExecutionError::GeneralError(format!("field was not an Object: {e}")))?; - if value.is_null() { - return Ok(None); - } - let jstr = unsafe { JString::from_raw(env, value.into_raw()) }; - jstr.try_to_string(env) - .map(Some) - .map_err(|e| ExecutionError::GeneralError(format!("try_to_string: {e}"))) -} diff --git a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs new file mode 100644 index 0000000000..f9f7dcdf1b --- /dev/null +++ b/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs @@ -0,0 +1,290 @@ +// 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. + +//! JNI bridge to the `CometS3CredentialDispatcher` SPI, exposed as +//! `object_store::CredentialProvider` and `reqsign_core::ProvideCredential` for the raw Parquet +//! and Iceberg scan paths respectively. +//! +//! ```text +//! JVM Native (Rust) +//! --- ------------- +//! +//! ServiceLoader s3.rs (object_store) +//! (one-time, at class-load) iceberg_scan.rs (opendal) +//! | | +//! v v +//! CometS3CredentialDispatcher CometS3CredentialBridge +//! (static singleton) impl object_store::CredentialProvider +//! | ^ impl reqsign_core::ProvideCredential +//! | | | +//! | +<---- JNI call -----------------+ +//! | getCredentialsForPath(bucket, path, mode ordinal) +//! v +//! vendor CometS3CredentialProvider +//! | +//! v +//! CometS3Credentials POJO +//! | +//! +------- JNI field reads ---------------->+ +//! | +//! v +//! AwsCredential / IcebergAwsCredential +//! (used to sign S3 requests) +//! ``` + +use crate::execution::operators::ExecutionError; +use crate::jvm_bridge::{jni_static_call, JVMClasses}; +use crate::JAVA_VM; +use async_trait::async_trait; +use iceberg_storage_opendal::AwsCredential as IcebergAwsCredential; +use jni::objects::{JFieldID, JObject, JString}; +use jni::signature::{Primitive, ReturnType}; +use jni::sys::{jboolean, jint}; +use log::{debug, warn}; +use object_store::aws::AwsCredential; +use object_store::CredentialProvider; +use once_cell::sync::OnceCell; +use reqsign_core::time::Timestamp; +use reqsign_core::{ + Context, Error as ReqsignError, ErrorKind as ReqsignErrorKind, + ProvideCredential as IcebergProvideCredential, +}; +use std::sync::Arc; +use std::time::Duration; +use url::Url; + +/// Cap on opendal's credential cache when the provider does not report an expiry. Prevents the +/// executor from holding a stale credential for the entire job lifetime. +const DEFAULT_EXPIRY_WHEN_UNKNOWN: Duration = Duration::from_secs(300); + +static PROVIDER_REGISTERED: OnceCell = OnceCell::new(); +/// Once-per-process latch for the "missing expiry" warning; bridges are per-scan so a per-bridge +/// latch would log once per scan on the same misbehaving provider. +static WARNED_MISSING_EXPIRY: OnceCell<()> = OnceCell::new(); + +/// Access intent forwarded to the Java SPI. Ordinal must match the JVM `CometS3AccessMode` enum. +#[derive(Debug, Clone, Copy)] +pub enum AccessMode { + Read = 0, + /// No native write path yet; kept so the SPI contract is complete. + #[allow(dead_code)] + Write = 1, +} + +/// True iff a `CometS3CredentialProvider` was discovered on the JVM classpath. +pub fn is_provider_registered() -> bool { + *PROVIDER_REGISTERED.get_or_init(|| { + // Unit tests construct stores without a JVM. Production init always precedes any store + // construction, so the None branch only fires in tests. + if JAVA_VM.get().is_none() { + return false; + } + JVMClasses::with_env(|env| -> Result { + let registered: jboolean = unsafe { + jni_static_call!(env, + comet_s3_credential_dispatcher.is_provider_registered() -> jboolean + )? + }; + Ok(registered) + }) + .unwrap_or_else(|e| { + debug!( + "CometS3CredentialDispatcher.isProviderRegistered failed; native S3 readers \ + will use the default AWS credential chain: {e}" + ); + false + }) + }) +} + +/// Per-request credential provider that delegates to the Java SPI via JNI. Constructed once per +/// S3 store or FileIO and forwards the same `(bucket, path, mode)` tuple on every fetch. +#[derive(Debug)] +pub struct CometS3CredentialBridge { + bucket: String, + path: String, + mode: AccessMode, +} + +impl CometS3CredentialBridge { + pub fn new(bucket: impl Into, path: impl Into, mode: AccessMode) -> Self { + Self { + bucket: bucket.into(), + path: path.into(), + mode, + } + } + + /// Shared constructor for the s3.rs and iceberg_scan.rs call sites. Returns `None` when no + /// provider is registered so callers can fall through to their default credential path. + pub fn for_url(url: &Url, mode: AccessMode) -> Option { + if !is_provider_registered() { + return None; + } + let bucket = url.host_str()?.to_string(); + debug!("Routing S3 credentials for bucket {bucket} through CometS3CredentialProvider"); + Some(Self::new(bucket, url.path(), mode)) + } + + fn fetch_raw(&self) -> Result { + JVMClasses::with_env(|env| -> Result { + let bucket = env + .new_string(&self.bucket) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(bucket): {e}")))?; + let path = env + .new_string(&self.path) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(path): {e}")))?; + let mode = self.mode as jint; + + let creds_obj: JObject = unsafe { + jni_static_call!(env, + comet_s3_credential_dispatcher.get_credentials_for_path( + &bucket, &path, mode + ) -> JObject + )? + }; + if creds_obj.is_null() { + return Err(ExecutionError::GeneralError( + "getCredentialsForPath returned null (contract violation)".to_string(), + )); + } + + let d = &JVMClasses::get().comet_s3_credential_dispatcher; + Ok(RawCredentials { + access_key_id: read_required_string( + env, + &creds_obj, + d.field_access_key_id, + "accessKeyId", + )?, + secret_access_key: read_required_string( + env, + &creds_obj, + d.field_secret_access_key, + "secretAccessKey", + )?, + session_token: read_optional_string(env, &creds_obj, d.field_session_token)?, + expiration_epoch_millis: unsafe { + env.get_field_unchecked( + &creds_obj, + d.field_expiration_epoch_millis, + ReturnType::Primitive(Primitive::Long), + ) + } + .and_then(|v| v.j()) + .map_err(|e| { + ExecutionError::GeneralError(format!("read expirationEpochMillis: {e}")) + })?, + }) + }) + } +} + +struct RawCredentials { + access_key_id: String, + secret_access_key: String, + session_token: Option, + /// Absolute expiry. `0` means the provider did not report one. + expiration_epoch_millis: i64, +} + +#[async_trait] +impl CredentialProvider for CometS3CredentialBridge { + type Credential = AwsCredential; + + async fn get_credential(&self) -> object_store::Result> { + let raw = self.fetch_raw().map_err(|e| object_store::Error::Generic { + store: "S3", + source: e.to_string().into(), + })?; + Ok(Arc::new(AwsCredential { + key_id: raw.access_key_id, + secret_key: raw.secret_access_key, + token: raw.session_token, + })) + } +} + +impl IcebergProvideCredential for CometS3CredentialBridge { + type Credential = IcebergAwsCredential; + + async fn provide_credential( + &self, + _ctx: &Context, + ) -> reqsign_core::Result> { + let raw = self + .fetch_raw() + .map_err(|e| ReqsignError::new(ReqsignErrorKind::CredentialInvalid, e.to_string()))?; + + let expires_in = if raw.expiration_epoch_millis > 0 { + Some( + Timestamp::from_millisecond(raw.expiration_epoch_millis).map_err(|e| { + ReqsignError::new( + ReqsignErrorKind::CredentialInvalid, + format!( + "Invalid expirationEpochMillis {}: {e}", + raw.expiration_epoch_millis + ), + ) + })?, + ) + } else { + if WARNED_MISSING_EXPIRY.set(()).is_ok() { + warn!( + "CometS3CredentialProvider returned credentials without expiration; \ + defaulting to {}s expiry to bound opendal caching", + DEFAULT_EXPIRY_WHEN_UNKNOWN.as_secs() + ); + } + Some(Timestamp::now() + DEFAULT_EXPIRY_WHEN_UNKNOWN) + }; + + Ok(Some(IcebergAwsCredential { + access_key_id: raw.access_key_id, + secret_access_key: raw.secret_access_key, + session_token: raw.session_token, + expires_in, + })) + } +} + +fn read_required_string( + env: &mut jni::Env, + instance: &JObject, + field: JFieldID, + name: &str, +) -> Result { + read_optional_string(env, instance, field)? + .ok_or_else(|| ExecutionError::GeneralError(format!("{name} was null"))) +} + +fn read_optional_string( + env: &mut jni::Env, + instance: &JObject, + field: JFieldID, +) -> Result, ExecutionError> { + let value = unsafe { env.get_field_unchecked(instance, field, ReturnType::Object) } + .and_then(|v| v.l()) + .map_err(|e| ExecutionError::GeneralError(format!("get_field_unchecked: {e}")))?; + if value.is_null() { + return Ok(None); + } + let jstr = unsafe { JString::from_raw(env, value.into_raw()) }; + jstr.try_to_string(env) + .map(Some) + .map_err(|e| ExecutionError::GeneralError(format!("try_to_string: {e}"))) +} diff --git a/native/core/src/parquet/objectstore/mod.rs b/native/core/src/parquet/objectstore/mod.rs index 5f7bc3b133..a12835d5cf 100644 --- a/native/core/src/parquet/objectstore/mod.rs +++ b/native/core/src/parquet/objectstore/mod.rs @@ -15,5 +15,5 @@ // specific language governing permissions and limitations // under the License. -pub mod comet_credential_bridge; +pub mod comet_s3_credential_bridge; pub mod s3; diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index 23b3456a51..3e90ad6391 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -21,7 +21,9 @@ use std::sync::OnceLock; use url::Url; use crate::execution::jni_api::get_runtime; -use crate::parquet::objectstore::comet_credential_bridge; +use crate::parquet::objectstore::comet_s3_credential_bridge::{ + AccessMode, CometS3CredentialBridge, +}; use async_trait::async_trait; use aws_config::{ ecs::EcsCredentialsProvider, environment::EnvironmentVariableCredentialsProvider, @@ -79,18 +81,10 @@ pub fn create_store( source: "Missing bucket name in S3 URL".into(), })?; - let credential_provider = - get_runtime().block_on(build_credential_provider(configs, bucket, min_ttl))?; - builder = if comet_credential_bridge::is_provider_registered() { - debug!("Using CometCredentialBridge for bucket {bucket}"); - let bridge = comet_credential_bridge::CometCredentialBridge::new( - bucket, - url.path(), - comet_credential_bridge::AccessMode::Read, - ); + builder = if let Some(bridge) = CometS3CredentialBridge::for_url(url, AccessMode::Read) { builder.with_credentials(Arc::new(bridge)) } else { - match credential_provider { + match get_runtime().block_on(build_credential_provider(configs, bucket, min_ttl))? { Some(provider) => builder.with_credentials(Arc::new(provider)), None => builder.with_skip_signature(true), } diff --git a/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs b/native/jni-bridge/src/comet_s3_credential_dispatcher.rs similarity index 80% rename from native/jni-bridge/src/comet_cloud_credential_dispatcher.rs rename to native/jni-bridge/src/comet_s3_credential_dispatcher.rs index f37e28fb95..7e3a606692 100644 --- a/native/jni-bridge/src/comet_cloud_credential_dispatcher.rs +++ b/native/jni-bridge/src/comet_s3_credential_dispatcher.rs @@ -23,12 +23,12 @@ use jni::{ Env, }; -/// JNI handles for the JVM `org.apache.comet.cloud.CometCloudCredentialDispatcher` SPI plus the -/// `CometCredentials` POJO whose fields the native bridge reads back. Cached at JVM-init time -/// so the underlying `JClass` references stay alive for the executor lifetime - acquiring them -/// inside a per-call local frame would let them be freed on frame pop. -pub struct CometCloudCredentialDispatcher<'a> { +/// JNI handles for the JVM `CometS3CredentialDispatcher` SPI plus the `CometS3Credentials` POJO +/// whose fields the native bridge reads back. +pub struct CometS3CredentialDispatcher<'a> { pub class: JClass<'a>, + /// Retained so the cached POJO `JFieldID`s stay alive for the executor lifetime. + #[allow(dead_code)] pub credentials_class: JClass<'a>, pub method_is_provider_registered: JStaticMethodID, pub method_is_provider_registered_ret: ReturnType, @@ -40,15 +40,15 @@ pub struct CometCloudCredentialDispatcher<'a> { pub field_expiration_epoch_millis: JFieldID, } -impl<'a> CometCloudCredentialDispatcher<'a> { - pub const JVM_CLASS: &'static str = "org/apache/comet/cloud/CometCloudCredentialDispatcher"; - pub const CREDENTIALS_CLASS: &'static str = "org/apache/comet/cloud/CometCredentials"; +impl<'a> CometS3CredentialDispatcher<'a> { + pub const JVM_CLASS: &'static str = "org/apache/comet/cloud/s3/CometS3CredentialDispatcher"; + pub const CREDENTIALS_CLASS: &'static str = "org/apache/comet/cloud/s3/CometS3Credentials"; - pub fn new(env: &mut Env<'a>) -> JniResult> { + pub fn new(env: &mut Env<'a>) -> JniResult> { let class = env.find_class(JNIString::new(Self::JVM_CLASS))?; let credentials_class = env.find_class(JNIString::new(Self::CREDENTIALS_CLASS))?; - Ok(CometCloudCredentialDispatcher { + Ok(CometS3CredentialDispatcher { method_is_provider_registered: env.get_static_method_id( JNIString::new(Self::JVM_CLASS), jni::jni_str!("isProviderRegistered"), @@ -59,7 +59,7 @@ impl<'a> CometCloudCredentialDispatcher<'a> { JNIString::new(Self::JVM_CLASS), jni::jni_str!("getCredentialsForPath"), jni::jni_sig!( - "(Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;)Lorg/apache/comet/cloud/CometCredentials;" + "(Ljava/lang/String;Ljava/lang/String;I)Lorg/apache/comet/cloud/s3/CometS3Credentials;" ), )?, method_get_credentials_for_path_ret: ReturnType::Object, diff --git a/native/jni-bridge/src/lib.rs b/native/jni-bridge/src/lib.rs index e9bc71ba91..5420cc8157 100644 --- a/native/jni-bridge/src/lib.rs +++ b/native/jni-bridge/src/lib.rs @@ -190,15 +190,15 @@ impl<'a> TryFrom> for BinaryWrapper<'a> { mod comet_exec; pub use comet_exec::*; mod batch_iterator; -mod comet_cloud_credential_dispatcher; mod comet_metric_node; +mod comet_s3_credential_dispatcher; mod comet_task_memory_manager; mod comet_udf_bridge; mod shuffle_block_iterator; use batch_iterator::CometBatchIterator; -pub use comet_cloud_credential_dispatcher::CometCloudCredentialDispatcher; pub use comet_metric_node::*; +pub use comet_s3_credential_dispatcher::CometS3CredentialDispatcher; pub use comet_task_memory_manager::*; use comet_udf_bridge::CometUdfBridge; use shuffle_block_iterator::CometShuffleBlockIterator; @@ -235,10 +235,10 @@ pub struct JVMClasses<'a> { /// The CometUdfBridge class used to dispatch JVM scalar UDFs. /// `None` if the class is not on the classpath. pub comet_udf_bridge: Option>, - /// JNI handles for the CometCloudCredentialDispatcher SPI and the CometCredentials POJO. + /// JNI handles for the CometS3CredentialDispatcher SPI and the CometS3Credentials POJO. /// Always present (the classes ship in `comet-common`); whether a vendor provider is actually /// registered is a separate runtime check. - pub comet_cloud_credential_dispatcher: CometCloudCredentialDispatcher<'a>, + pub comet_s3_credential_dispatcher: CometS3CredentialDispatcher<'a>, } unsafe impl Send for JVMClasses<'_> {} @@ -316,8 +316,7 @@ impl JVMClasses<'_> { } bridge }, - comet_cloud_credential_dispatcher: CometCloudCredentialDispatcher::new(env) - .unwrap(), + comet_s3_credential_dispatcher: CometS3CredentialDispatcher::new(env).unwrap(), } }); } diff --git a/spark/src/test/java/org/apache/comet/cloud/MinioCometCredentialProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java similarity index 66% rename from spark/src/test/java/org/apache/comet/cloud/MinioCometCredentialProvider.java rename to spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java index 04348df949..f255d9cb27 100644 --- a/spark/src/test/java/org/apache/comet/cloud/MinioCometCredentialProvider.java +++ b/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java @@ -17,28 +17,23 @@ * under the License. */ -package org.apache.comet.cloud; +package org.apache.comet.cloud.s3; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; /** - * Test-only {@link CometCloudCredentialProvider} that returns credentials previously installed by - * {@link #installCredentials} and counts every invocation. Registered via {@code - * META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider} on the test classpath. - * - *

State is held in static fields because {@link CometCloudCredentialDispatcher} caches a single - * provider instance for the JVM lifetime; tests interact with that instance via these statics - * rather than constructing their own. + * Test-only {@link CometS3CredentialProvider} registered via {@code META-INF/services}. Returns + * Minio's static credentials once {@link #installCredentials} has been called and counts every + * invocation so suites can assert the bridge was actually used. */ -public final class MinioCometCredentialProvider implements CometCloudCredentialProvider { +public final class MinioCometS3CredentialProvider implements CometS3CredentialProvider { private static final AtomicReference CREDS = new AtomicReference<>(); private static final AtomicInteger CALL_COUNT = new AtomicInteger(0); private static final AtomicReference LAST_BUCKET = new AtomicReference<>(); private static final AtomicReference LAST_PATH = new AtomicReference<>(); - /** Install the credentials this provider should return. Called from test {@code beforeAll}. */ public static void installCredentials(String accessKeyId, String secretAccessKey) { CREDS.set(new Credentials(accessKeyId, secretAccessKey)); } @@ -55,7 +50,6 @@ public static String lastPath() { return LAST_PATH.get(); } - /** Reset call tracking. Tests use this to take a clean snapshot before a query. */ public static void resetCounters() { CALL_COUNT.set(0); LAST_BUCKET.set(null); @@ -63,16 +57,17 @@ public static void resetCounters() { } @Override - public CometCredentials getCredentialsForPath(String bucket, String path, CometAccessMode mode) { + public CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) { CALL_COUNT.incrementAndGet(); LAST_BUCKET.set(bucket); LAST_PATH.set(path); Credentials c = CREDS.get(); if (c == null) { throw new IllegalStateException( - "MinioCometCredentialProvider used before installCredentials() was called"); + "MinioCometS3CredentialProvider used before installCredentials() was called"); } - return new CometCredentials(c.accessKeyId, c.secretAccessKey, null, null, 0L); + return new CometS3Credentials(c.accessKeyId, c.secretAccessKey, null, 0L); } private static final class Credentials { diff --git a/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider b/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider deleted file mode 100644 index 583dcfc8b8..0000000000 --- a/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.CometCloudCredentialProvider +++ /dev/null @@ -1 +0,0 @@ -org.apache.comet.cloud.MinioCometCredentialProvider diff --git a/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider b/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider new file mode 100644 index 0000000000..40757b041c --- /dev/null +++ b/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider @@ -0,0 +1 @@ +org.apache.comet.cloud.s3.MinioCometS3CredentialProvider diff --git a/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala b/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala index 644ab402d8..55647b253d 100644 --- a/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala +++ b/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala @@ -29,7 +29,7 @@ import org.testcontainers.utility.DockerImageName import org.apache.spark.SparkConf import org.apache.spark.sql.CometTestBase -import org.apache.comet.cloud.MinioCometCredentialProvider +import org.apache.comet.cloud.s3.MinioCometS3CredentialProvider import software.amazon.awssdk.auth.credentials.{AwsBasicCredentials, StaticCredentialsProvider} import software.amazon.awssdk.services.s3.S3Client @@ -50,11 +50,9 @@ trait CometS3TestBase extends CometTestBase { minioContainer.start() createBucketIfNotExists(testBucketName) - // The test SPI registered via META-INF/services routes Comet's native S3 credential - // requests through MinioCometCredentialProvider for every S3 suite in this JVM. Install - // Minio's static credentials here so all subclasses (not just CometCloudCredentialBridge - // S3Suite) work end-to-end. - MinioCometCredentialProvider.installCredentials(userName, password) + // Install Minio's credentials into the test SPI so every S3 suite sharing this JVM routes + // through the bridge cleanly. + MinioCometS3CredentialProvider.installCredentials(userName, password) super.beforeAll() } @@ -85,9 +83,8 @@ trait CometS3TestBase extends CometTestBase { /** * Apply the S3 properties Comet's native Iceberg reader requires on the given catalog. - * iceberg-rust + opendal stops auto-detecting region when a custom credential loader is wired - * in (which is always, when the bridge SPI is registered), so the region/endpoint/ path-style - * triple has to be set explicitly. See the contributor guide on CometCloudCredentialProvider. + * iceberg-rust / opendal disables region auto-detection when a custom credential loader is + * wired in, so the region/endpoint/path-style triple has to be set explicitly. */ protected def applyS3CatalogProps(conf: SparkConf, catalogName: String): Unit = { conf.set(s"spark.sql.catalog.$catalogName.s3.endpoint", minioContainer.getS3URL) diff --git a/spark/src/test/scala/org/apache/comet/cloud/CometCloudCredentialBridgeS3Suite.scala b/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala similarity index 69% rename from spark/src/test/scala/org/apache/comet/cloud/CometCloudCredentialBridgeS3Suite.scala rename to spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala index 1685084004..e13305982d 100644 --- a/spark/src/test/scala/org/apache/comet/cloud/CometCloudCredentialBridgeS3Suite.scala +++ b/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.cloud +package org.apache.comet.cloud.s3 import org.apache.spark.SparkConf import org.apache.spark.sql.SaveMode @@ -29,17 +29,11 @@ import org.apache.spark.sql.functions.{col, sum} import org.apache.comet.{CometConf, CometS3TestBase} /** - * End-to-end test that exercises [[CometCloudCredentialDispatcher]] and the Rust JNI bridge - * against a real S3 server (Minio). The test [[MinioCometCredentialProvider]] is registered via - * `META-INF/services` and returns the harness's Minio credentials; success here proves Comet's - * native scan paths actually invoked the SPI rather than falling back to the default AWS chain. - * - * Note: because [[CometCloudCredentialDispatcher.PROVIDER]] is a `static final` initialized once - * per JVM, registering this test SPI affects every test in the same JVM. Other Minio suites (e.g. - * ParquetReadFromS3Suite, IcebergReadFromS3Suite) continue to pass because the test provider - * returns the same Minio credentials they would have used through the default chain. + * End-to-end test that exercises [[CometS3CredentialDispatcher]] and the Rust JNI bridge against + * a real S3 server (Minio). Asserts the test SPI was actually invoked rather than the default AWS + * credential chain. */ -class CometCloudCredentialBridgeS3Suite extends CometS3TestBase with AdaptiveSparkPlanHelper { +class CometS3CredentialBridgeSuite extends CometS3TestBase with AdaptiveSparkPlanHelper { override protected val testBucketName = "bridge-test-bucket" @@ -55,7 +49,7 @@ class CometCloudCredentialBridgeS3Suite extends CometS3TestBase with AdaptiveSpa override def beforeAll(): Unit = { super.beforeAll() - MinioCometCredentialProvider.installCredentials(userName, password) + MinioCometS3CredentialProvider.installCredentials(userName, password) } private def assertHasCometParquetScan(plan: SparkPlan): Unit = { @@ -71,26 +65,26 @@ class CometCloudCredentialBridgeS3Suite extends CometS3TestBase with AdaptiveSpa assert(scans.nonEmpty, s"Expected at least one CometIcebergNativeScanExec in plan:\n$plan") } - test("Parquet read on S3 routes credentials through CometCloudCredentialProvider") { + test("Parquet read on S3 routes credentials through CometS3CredentialProvider") { val testFilePath = s"s3a://$testBucketName/data/bridge-parquet.parquet" val rowCount = 1000L spark.range(0, rowCount).write.format("parquet").mode(SaveMode.Overwrite).save(testFilePath) val expectedSum = (0L until rowCount).sum - MinioCometCredentialProvider.resetCounters() + MinioCometS3CredentialProvider.resetCounters() val df = spark.read.format("parquet").load(testFilePath).agg(sum(col("id"))) assertHasCometParquetScan(df.queryExecution.executedPlan) assert(df.first().getLong(0) == expectedSum) assert( - MinioCometCredentialProvider.callCount() > 0, + MinioCometS3CredentialProvider.callCount() > 0, "Bridge was not invoked during Comet Parquet read") assert( - MinioCometCredentialProvider.lastBucket() == testBucketName, - s"Bridge received unexpected bucket: ${MinioCometCredentialProvider.lastBucket()}") + MinioCometS3CredentialProvider.lastBucket() == testBucketName, + s"Bridge received unexpected bucket: ${MinioCometS3CredentialProvider.lastBucket()}") } - test("Iceberg read on S3 routes credentials through CometCloudCredentialProvider") { + test("Iceberg read on S3 routes credentials through CometS3CredentialProvider") { assume(icebergAvailable, "Iceberg not available in classpath") spark.sql(""" @@ -105,13 +99,13 @@ class CometCloudCredentialBridgeS3Suite extends CometS3TestBase with AdaptiveSpa VALUES (1, 'a'), (2, 'b'), (3, 'c') """) - MinioCometCredentialProvider.resetCounters() + MinioCometS3CredentialProvider.resetCounters() val df = spark.sql("SELECT * FROM s3_catalog.db.bridge_iceberg ORDER BY id") assertHasCometIcebergScan(df.queryExecution.executedPlan) assert(df.count() == 3) assert( - MinioCometCredentialProvider.callCount() > 0, + MinioCometS3CredentialProvider.callCount() > 0, "Bridge was not invoked during Comet Iceberg read") spark.sql("DROP TABLE s3_catalog.db.bridge_iceberg") From de5d5c2f68ca527e2062927827d46fac1a0ef872 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 15 May 2026 15:31:25 -0400 Subject: [PATCH 14/24] Replaced the ServiceLoader-based S3 credential SPI with config-driven activation (fs.s3a.comet.credential.provider.class for Parquet, s3.comet.credential.provider.class for Iceberg), so the bridge is opt-in per Spark config rather than implicit on classpath presence. --- .../cloud/s3/CometS3CredentialDispatcher.java | 93 +++++++------ .../cloud/s3/CometS3CredentialProvider.java | 9 +- .../s3/CometS3CredentialDispatcherTest.java | 67 +++++++-- .../comet/cloud/s3/NoNoArgCtorProvider.java | 33 +++++ .../cloud/s3/NotACredentialProvider.java | 25 ++++ .../s3/TestCometS3CredentialProvider.java | 4 +- ...e.comet.cloud.s3.CometS3CredentialProvider | 1 - .../latest/cloud-credential-providers.md | 131 +++++++++--------- .../src/execution/operators/iceberg_scan.rs | 28 +++- .../objectstore/comet_s3_credential_bridge.rs | 103 +++++++------- native/core/src/parquet/objectstore/s3.rs | 3 +- .../src/comet_s3_credential_dispatcher.rs | 12 +- ...e.comet.cloud.s3.CometS3CredentialProvider | 1 - .../org/apache/comet/CometS3TestBase.scala | 6 - .../s3/CometS3CredentialBridgeSuite.scala | 5 + 15 files changed, 321 insertions(+), 200 deletions(-) create mode 100644 common/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java create mode 100644 common/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java delete mode 100644 common/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider delete mode 100644 spark/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java index e54e9fa026..38457c927f 100644 --- a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java @@ -19,72 +19,87 @@ package org.apache.comet.cloud.s3; -import java.util.ArrayList; -import java.util.List; -import java.util.ServiceLoader; -import java.util.stream.Collectors; +import java.lang.reflect.InvocationTargetException; +import java.util.concurrent.ConcurrentHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * JNI entry point invoked from native code to resolve {@link CometS3CredentialProvider}. + * JNI entry point invoked from native code to resolve a {@link CometS3CredentialProvider}. * - *

The provider is resolved once via {@link ServiceLoader} and cached in a {@code static final} - * field. A query falling back from Comet to Spark mid-execution therefore sees identical - * credentials, since both paths resolve from the same executor classpath. - * - *

Multiple registered impls fail fast at class-load; chaining is a vendor-side concern. + *

The provider class is named in {@code fs.s3a.comet.credential.provider.class} (optionally + * per-bucket). Native code reads that key from the object-store options it already forwards over + * JNI and passes the FQCN here. We instantiate each named class once via reflection and cache it, + * so a single executor JVM can serve multiple providers (e.g. one for the default config and one + * for a specific bucket) without re-instantiation per request. */ public final class CometS3CredentialDispatcher { private static final Logger LOG = LoggerFactory.getLogger(CometS3CredentialDispatcher.class); - private static final CometS3CredentialProvider PROVIDER = resolve(); + private static final ConcurrentHashMap INSTANCES = + new ConcurrentHashMap<>(); private static final CometS3AccessMode[] MODES = CometS3AccessMode.values(); private CometS3CredentialDispatcher() {} - public static boolean isProviderRegistered() { - return PROVIDER != null; - } - - /** Invoked by native code. {@code mode} is the {@link CometS3AccessMode} ordinal. */ - public static CometS3Credentials getCredentialsForPath(String bucket, String path, int mode) - throws Exception { - if (PROVIDER == null) { - throw new IllegalStateException( - "No CometS3CredentialProvider registered; check META-INF/services on the classpath"); + /** + * Invoked by native code. {@code mode} is the {@link CometS3AccessMode} ordinal. + * + * @param providerClassName FQCN configured in {@code fs.s3a.comet.credential.provider.class} + */ + public static CometS3Credentials getCredentialsForPath( + String providerClassName, String bucket, String path, int mode) throws Exception { + if (providerClassName == null || providerClassName.isEmpty()) { + throw new IllegalArgumentException( + "providerClassName is empty; native side should not call without a configured class"); } if (mode < 0 || mode >= MODES.length) { throw new IllegalArgumentException("Invalid CometS3AccessMode ordinal: " + mode); } + CometS3CredentialProvider provider = resolve(providerClassName); CometS3AccessMode accessMode = MODES[mode]; if (LOG.isDebugEnabled()) { - LOG.debug("Fetching credentials for bucket={} path={} mode={}", bucket, path, accessMode); + LOG.debug( + "Fetching credentials via {} for bucket={} path={} mode={}", + providerClassName, + bucket, + path, + accessMode); } - return PROVIDER.getCredentialsForPath(bucket, path, accessMode); + return provider.getCredentialsForPath(bucket, path, accessMode); } - private static CometS3CredentialProvider resolve() { - List impls = new ArrayList<>(); - for (CometS3CredentialProvider impl : ServiceLoader.load(CometS3CredentialProvider.class)) { - impls.add(impl); + private static CometS3CredentialProvider resolve(String providerClassName) { + return INSTANCES.computeIfAbsent(providerClassName, CometS3CredentialDispatcher::instantiate); + } + + private static CometS3CredentialProvider instantiate(String providerClassName) { + Class clazz; + try { + clazz = Class.forName(providerClassName); + } catch (ClassNotFoundException e) { + throw new IllegalStateException( + "CometS3CredentialProvider class not found: " + + providerClassName + + ". Ensure the vendor JAR is on the executor classpath.", + e); } - if (impls.isEmpty()) { - LOG.info( - "No CometS3CredentialProvider registered; native S3 readers will use the default " - + "AWS credential chain"); - return null; + if (!CometS3CredentialProvider.class.isAssignableFrom(clazz)) { + throw new IllegalStateException( + providerClassName + " does not implement " + CometS3CredentialProvider.class.getName()); } - if (impls.size() > 1) { - List names = - impls.stream().map(p -> p.getClass().getName()).collect(Collectors.toList()); + try { + Object instance = clazz.getDeclaredConstructor().newInstance(); + LOG.info("Instantiated CometS3CredentialProvider {}", providerClassName); + return (CometS3CredentialProvider) instance; + } catch (NoSuchMethodException e) { + throw new IllegalStateException( + providerClassName + " must declare a public no-arg constructor", e); + } catch (InstantiationException | IllegalAccessException | InvocationTargetException e) { throw new IllegalStateException( - "Multiple CometS3CredentialProvider impls on classpath: " + names); + "Failed to instantiate CometS3CredentialProvider " + providerClassName, e); } - CometS3CredentialProvider provider = impls.get(0); - LOG.info("Registered CometS3CredentialProvider: {}", provider.getClass().getName()); - return provider; } } diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java index 677645a3e7..6181e4b1fa 100644 --- a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java @@ -45,10 +45,11 @@ * ship this logic as Java code, not HTTP. * * - *

Vendors register an implementation via {@code - * META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider}. {@link - * #getCredentialsForPath} may be invoked concurrently from many native tokio tasks, so - * implementations must be thread-safe. + *

Vendors register an implementation by setting {@code + * spark.hadoop.fs.s3a.comet.credential.provider.class} (or the per-bucket form {@code + * spark.hadoop.fs.s3a.bucket..comet.credential.provider.class}) to the implementing FQCN. The + * class must have a public no-arg constructor. {@link #getCredentialsForPath} may be invoked + * concurrently from many native tokio tasks, so implementations must be thread-safe. * *

Returns credentials or throws; there is no fall-through return value. A provider that is only * authoritative for some paths should resolve the default AWS chain itself for the rest. See the diff --git a/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java b/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java index bc360e401d..c0066e2b7a 100644 --- a/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java +++ b/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java @@ -31,6 +31,7 @@ public class CometS3CredentialDispatcherTest { + private static final String TEST_PROVIDER = TestCometS3CredentialProvider.class.getName(); private static final int READ = CometS3AccessMode.READ.ordinal(); private static final int WRITE = CometS3AccessMode.WRITE.ordinal(); @@ -39,15 +40,11 @@ public void resetTestProvider() { TestCometS3CredentialProvider.reset(); } - @Test - public void providerIsRegisteredFromTestClasspath() { - assertTrue(CometS3CredentialDispatcher.isProviderRegistered()); - } - @Test public void getCredentialsRoundTripsThroughProvider() throws Exception { CometS3Credentials creds = - CometS3CredentialDispatcher.getCredentialsForPath("my-bucket", "path/to/object", READ); + CometS3CredentialDispatcher.getCredentialsForPath( + TEST_PROVIDER, "my-bucket", "path/to/object", READ); assertNotNull(creds); assertEquals("AKIATEST", creds.getAccessKeyId()); @@ -63,7 +60,7 @@ public void getCredentialsRoundTripsThroughProvider() throws Exception { @Test public void writeModeIsForwarded() throws Exception { - CometS3CredentialDispatcher.getCredentialsForPath("b", "k", WRITE); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b", "k", WRITE); assertEquals(CometS3AccessMode.WRITE, TestCometS3CredentialProvider.lastMode); } @@ -71,7 +68,50 @@ public void writeModeIsForwarded() throws Exception { public void unknownModeRejected() { assertThrows( IllegalArgumentException.class, - () -> CometS3CredentialDispatcher.getCredentialsForPath("b", "k", 99)); + () -> CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b", "k", 99)); + } + + @Test + public void emptyClassNameRejected() { + assertThrows( + IllegalArgumentException.class, + () -> CometS3CredentialDispatcher.getCredentialsForPath("", "b", "k", READ)); + assertThrows( + IllegalArgumentException.class, + () -> CometS3CredentialDispatcher.getCredentialsForPath(null, "b", "k", READ)); + } + + @Test + public void missingClassReportsActionableError() { + Exception thrown = + assertThrows( + IllegalStateException.class, + () -> + CometS3CredentialDispatcher.getCredentialsForPath( + "com.does.not.Exist", "b", "k", READ)); + assertTrue(thrown.getMessage().contains("not found")); + } + + @Test + public void classNotImplementingInterfaceRejected() { + Exception thrown = + assertThrows( + IllegalStateException.class, + () -> + CometS3CredentialDispatcher.getCredentialsForPath( + NotACredentialProvider.class.getName(), "b", "k", READ)); + assertTrue(thrown.getMessage().contains("does not implement")); + } + + @Test + public void classWithoutNoArgCtorRejected() { + Exception thrown = + assertThrows( + IllegalStateException.class, + () -> + CometS3CredentialDispatcher.getCredentialsForPath( + NoNoArgCtorProvider.class.getName(), "b", "k", READ)); + assertTrue(thrown.getMessage().contains("no-arg constructor")); } @Test @@ -82,7 +122,7 @@ public void providerExceptionsPropagate() { Exception thrown = assertThrows( Exception.class, - () -> CometS3CredentialDispatcher.getCredentialsForPath("b", "k", READ)); + () -> CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b", "k", READ)); assertSame(boom, thrown); } @@ -90,16 +130,17 @@ public void providerExceptionsPropagate() { public void nullSessionTokenAllowed() throws Exception { TestCometS3CredentialProvider.nextResult = new CometS3Credentials("AKIA", "sec", null, 0L); - CometS3Credentials creds = CometS3CredentialDispatcher.getCredentialsForPath("b", "k", READ); + CometS3Credentials creds = + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b", "k", READ); assertNull(creds.getSessionToken()); } @Test public void providerReceivesEachCallSeparately() throws Exception { - CometS3CredentialDispatcher.getCredentialsForPath("b1", "k1", READ); - CometS3CredentialDispatcher.getCredentialsForPath("b2", "k2", READ); - CometS3CredentialDispatcher.getCredentialsForPath("b3", "k3", READ); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b1", "k1", READ); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b2", "k2", READ); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b3", "k3", READ); assertEquals(3, TestCometS3CredentialProvider.callCount.get()); assertEquals("b3", TestCometS3CredentialProvider.lastBucket); diff --git a/common/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java b/common/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java new file mode 100644 index 0000000000..606a482f7d --- /dev/null +++ b/common/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java @@ -0,0 +1,33 @@ +/* + * 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.comet.cloud.s3; + +/** Helper for the dispatcher's "missing public no-arg constructor" rejection path. */ +public class NoNoArgCtorProvider implements CometS3CredentialProvider { + + @SuppressWarnings("unused") + public NoNoArgCtorProvider(String required) {} + + @Override + public CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) { + throw new UnsupportedOperationException("never instantiable"); + } +} diff --git a/common/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java b/common/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java new file mode 100644 index 0000000000..ebe292f677 --- /dev/null +++ b/common/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java @@ -0,0 +1,25 @@ +/* + * 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.comet.cloud.s3; + +/** Helper for the dispatcher's "does not implement interface" rejection path. */ +public class NotACredentialProvider { + public NotACredentialProvider() {} +} diff --git a/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java b/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java index f0429ebf1c..d0eb0a1dd2 100644 --- a/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java +++ b/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java @@ -22,8 +22,8 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * Test-only provider registered via {@code META-INF/services}. State is static because the - * dispatcher caches a single provider instance for the JVM lifetime. + * Test-only provider instantiated by the dispatcher when its FQCN is passed across JNI. State is + * static because the dispatcher caches one instance per class name for the JVM lifetime. */ public class TestCometS3CredentialProvider implements CometS3CredentialProvider { diff --git a/common/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider b/common/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider deleted file mode 100644 index cb6a1717e7..0000000000 --- a/common/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider +++ /dev/null @@ -1 +0,0 @@ -org.apache.comet.cloud.s3.TestCometS3CredentialProvider diff --git a/docs/source/user-guide/latest/cloud-credential-providers.md b/docs/source/user-guide/latest/cloud-credential-providers.md index 7a39c84cfb..5d24322114 100644 --- a/docs/source/user-guide/latest/cloud-credential-providers.md +++ b/docs/source/user-guide/latest/cloud-credential-providers.md @@ -19,11 +19,7 @@ under the License. # Cloud Credential Providers -Comet's native S3 readers normally fetch credentials from the standard AWS credential chain -(static keys, instance profiles, environment variables, etc.). Some clusters use a vendor-managed -mechanism instead, where credentials are issued per request based on a JWT or per S3 path. For -those clusters, Comet supports loading a vendor-supplied bridge JAR that routes every native -credential request through the vendor's Java code. +Comet's native S3 readers normally fetch credentials from the standard AWS credential chain (static keys, instance profiles, environment variables, etc.). Some clusters use a vendor-managed mechanism instead, where credentials are issued per request based on a JWT or per S3 path. For those clusters, Comet supports loading a vendor-supplied bridge class that routes every native credential request through the vendor's Java code. ## Do I need this? @@ -40,58 +36,60 @@ You probably do, if any of these are true: - You have a custom Iceberg `client.factory` that injects a configured S3 client. - Spark queries against your S3 paths work, but the same queries with Comet enabled fail with 403. -## Enabling a bridge JAR +## Enabling a bridge -Add the vendor-supplied bridge JAR to your Spark executor classpath: +A bridge is activated by naming the vendor's class in a Spark config. There is no `META-INF/services` discovery and putting a JAR on the classpath alone has no effect; the config key must be set. -```sh -spark-submit --jars vendor-comet-bridge.jar ... +For raw Parquet (the `object_store` path), set the Hadoop S3A config: + +``` +spark.hadoop.fs.s3a.comet.credential.provider.class=com.vendor.MyCometCredentialProvider ``` -Or via `spark.jars`: +A per-bucket override is supported and follows the same shape as `fs.s3a.bucket..aws.credentials.provider`: ``` -spark.jars=/path/to/vendor-comet-bridge.jar +spark.hadoop.fs.s3a.bucket..comet.credential.provider.class=com.vendor.MyCometCredentialProvider ``` -Comet discovers the bridge through `META-INF/services` at executor startup. There are no Comet -config keys to set. +For Iceberg (the `opendal` path), set the per-catalog property under the `s3.` namespace Iceberg already uses for its S3 settings: -OSS Comet ships no vendor-specific bridges. Get one from the same vendor that supplies your -Hadoop S3A signer or Iceberg client factory. If they do not yet provide one, send them to the -"Writing a bridge" section below. +``` +spark.sql.catalog..s3.comet.credential.provider.class=com.vendor.MyCometCredentialProvider +``` -## Verification +Add the vendor JAR to your Spark executor classpath: -With the bridge on the classpath, executor logs show: +```sh +spark-submit --jars vendor-comet-bridge.jar ... +``` -- At startup: `Registered CometS3CredentialProvider: ` -- On first S3 access (debug level): `Fetching credentials for bucket=... path=... mode=...` +OSS Comet ships no vendor-specific bridges. Get one from the same vendor that supplies your Hadoop S3A signer or Iceberg client factory. If they do not yet provide one, send them to the "Writing a bridge" section below. -Without a bridge registered you get exactly one line at startup: +## Verification -``` -No CometS3CredentialProvider registered; native S3 readers will use the default AWS credential chain -``` +With the config set and the JAR on the classpath, executor logs show on first S3 access: + +- Info level: `Instantiated CometS3CredentialProvider ` +- Debug level: `Fetching credentials via for bucket=... path=... mode=...` + +Without the config set, no credential-related log lines appear at startup; native readers use the default AWS credential chain. ## Troubleshooting -**`Multiple CometS3CredentialProvider impls on classpath: [...]`** at startup. Remove all but one -bridge JAR. Comet does not chain providers; it fails fast to prevent silent ambiguity. +**`CometS3CredentialProvider class not found: `**. The class named in the config is not on the executor classpath. Re-check `--jars` / `spark.jars`. On YARN or Kubernetes, confirm the JAR actually reached the executor and not only the driver. + +**` does not implement org.apache.comet.cloud.s3.CometS3CredentialProvider`**. The configured class exists but does not implement the SPI. Double-check the FQCN against the vendor's documentation. -**`No CometS3CredentialProvider registered`** combined with `403 AccessDenied`. The bridge JAR is -not on the executor classpath. Re-check `--jars` / `spark.jars`. On YARN or Kubernetes, confirm -the JAR actually reached the executor and not only the driver. +**` must declare a public no-arg constructor`**. Vendor classes are instantiated reflectively with `Class.forName(name).getDeclaredConstructor().newInstance()`. A non-default constructor is not supported; ask the vendor to expose a no-arg form that reads any state it needs from environment or system properties. -**Credentials silently going stale during long-running jobs.** The bridge caps opendal's -credential cache at 5 minutes when the vendor does not populate `expirationEpochMillis`. Ask the -vendor to return a real expiry; the 5-minute floor is a safety net, not a knob. +**`403 AccessDenied` with the bridge configured.** The provider returned credentials but they were rejected by S3. Most often a region mismatch (see Iceberg section below) or expired session token; enable debug logging on the vendor's class to confirm what it returned. + +**Credentials silently going stale during long-running jobs.** The bridge caps opendal's credential cache at 5 minutes when the vendor does not populate `expirationEpochMillis`. Ask the vendor to return a real expiry; the 5-minute floor is a safety net, not a knob. ## Iceberg: explicit S3 region required -With the bridge registered, Comet wires a custom credential loader into `iceberg-storage-opendal`. -opendal's built-in S3 region auto-detection only runs when no custom loader is configured, so on -the bridge path the region (and endpoint for non-AWS) must be set explicitly on the Spark catalog: +With the bridge configured, Comet wires a custom credential loader into `iceberg-storage-opendal`. opendal's built-in S3 region auto-detection only runs when no custom loader is configured, so on the bridge path the region (and endpoint for non-AWS) must be set explicitly on the Spark catalog: ``` spark.sql.catalog..s3.region = us-east-1 @@ -99,15 +97,11 @@ spark.sql.catalog..s3.endpoint = https://... (non-AWS only) spark.sql.catalog..s3.path-style-access = true (path-style endpoints only) ``` -If you hit `region is missing. Please find it by S3::detect_region() or set them in env`, this -is the missing config. +If you hit `region is missing. Please find it by S3::detect_region() or set them in env`, this is the missing config. ## Writing a bridge -Comet's native scan paths (`object_store` for raw Parquet, `opendal` via `iceberg-rust` for -Iceberg) bypass Hadoop S3A entirely. The standard `AWSCredentialsProvider.getCredentials()` has no -path argument, so vendors that issue per-path STS credentials cannot expose them through it. The -`CometS3CredentialProvider` SPI fills that gap. +Comet's native scan paths (`object_store` for raw Parquet, `opendal` via `iceberg-rust` for Iceberg) bypass Hadoop S3A entirely. The standard `AWSCredentialsProvider.getCredentials()` has no path argument, so vendors that issue per-path STS credentials cannot expose them through it. The `CometS3CredentialProvider` SPI fills that gap. Implement `org.apache.comet.cloud.s3.CometS3CredentialProvider`: @@ -120,9 +114,7 @@ public interface CometS3CredentialProvider { } ``` -Register via `META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider` with the -fully-qualified class name. `getCredentialsForPath` may be invoked concurrently from many native -tokio worker threads; the implementation must be thread-safe. +The class must have a public no-arg constructor. `getCredentialsForPath` may be invoked concurrently from many native tokio worker threads; the implementation must be thread-safe. ### Returned fields @@ -133,18 +125,13 @@ tokio worker threads; the implementation must be thread-safe. | `sessionToken` | `null` for non-STS credentials. | | `expirationEpochMillis` | Absolute expiry. `0` means "unknown" (see below). | -Provide a real `expirationEpochMillis` whenever you have one. The Iceberg path uses it to decide -when `opendal` must re-call the provider for a fresh credential. `0` is treated as unknown and the -Iceberg path defaults to a 5-minute refresh to bound staleness. +Provide a real `expirationEpochMillis` whenever you have one. The Iceberg path uses it to decide when `opendal` must re-call the provider for a fresh credential. `0` is treated as unknown and the Iceberg path defaults to a 5-minute refresh to bound staleness. ### Returns or throws -The SPI follows the same shape as the other JVM AWS-credential SPIs (AWS SDK v1/v2, -Hadoop S3A, Iceberg `VendedCredentialsProvider`): return credentials or throw. There is no -"fall-through" return value. Chaining is a vendor-side concern. +The SPI follows the same shape as the other JVM AWS-credential SPIs (AWS SDK v1/v2, Hadoop S3A, Iceberg `VendedCredentialsProvider`): return credentials or throw. There is no "fall-through" return value. -If your provider is authoritative only for some paths, resolve the default AWS chain yourself for -the rest: +If your provider is authoritative only for some paths, resolve the default AWS chain yourself for the rest: ```java private final DefaultCredentialsProvider defaultChain = DefaultCredentialsProvider.create(); @@ -163,6 +150,28 @@ public CometS3Credentials getCredentialsForPath( } ``` +### Composing multiple credential backends + +A single configured provider class is the dispatcher. If a vendor needs to route across several credential backends (per bucket, per path prefix, per tenant), the dispatch lives inside the vendor's class: + +```java +public final class MyCometCredentialProvider implements CometS3CredentialProvider { + private final ProdVendor prod = ...; + private final StsVendor sts = ...; + private final DefaultVendor fallback = ...; + + @Override + public CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) throws Exception { + if (bucket.startsWith("data-prod-")) return prod.fetch(bucket, path, mode); + if (bucket.equals("partner-shared")) return sts.assumeRole(bucket, path, mode); + return fallback.fetch(bucket, path); + } +} +``` + +Per-bucket Hadoop overrides (`fs.s3a.bucket..comet.credential.provider.class`) are also available if you prefer to ship multiple vendor classes and pick by bucket in config rather than in code. + ### Access mode | Value | Used for | @@ -170,16 +179,7 @@ public CometS3Credentials getCredentialsForPath( | `READ` | All native scan paths (raw Parquet, Iceberg). Comet today only sends READ. | | `WRITE` | Reserved for future native write paths. | -A `WRITE` credential is not implicitly read-capable. Vendors that need read-during-write -workflows include the required read permissions in the IAM policy attached to their `WRITE` -credentials. - -### Discovery rules - -- Zero impls registered: native readers use the default AWS credential chain. -- One impl registered: cached and used for every request. -- Multiple impls registered: `CometS3CredentialDispatcher` throws `IllegalStateException` at - class-load. Pick one bridge JAR. +A `WRITE` credential is not implicitly read-capable. Vendors that need read-during-write workflows include the required read permissions in the IAM policy attached to their `WRITE` credentials. ### Build setup @@ -196,9 +196,4 @@ Vendor implementations need the Comet SPI classes at compile time only. Use `pro ### Iceberg path: error message fidelity -When the bridge is wired into `iceberg-rust`, the outer `reqsign-core::ProvideCredentialChain` -currently swallows thrown exceptions into "no credential" before the request reaches opendal. The -credential is still not issued and the request still fails, only the message is degraded to an -opaque anonymous-request failure. No Comet change fixes this; it is resolved upstream if -`iceberg-rust` stops wrapping custom loaders in its outer chain or moves its S3 backend to -`object_store`. +When the bridge is wired into `iceberg-rust`, the outer `reqsign-core::ProvideCredentialChain` currently swallows thrown exceptions into "no credential" before the request reaches opendal. The credential is still not issued and the request still fails, only the message is degraded to an opaque anonymous-request failure. No Comet change fixes this; it is resolved upstream if `iceberg-rust` stops wrapping custom loaders in its outer chain or moves its S3 backend to `object_store`. diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 217a28a699..ca27c8c38b 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -203,7 +203,10 @@ impl IcebergScanExec { Ok(Box::pin(wrapped_stream)) } - fn storage_factory_for(path: &str) -> Result, DataFusionError> { + fn storage_factory_for( + path: &str, + catalog_properties: &HashMap, + ) -> Result, DataFusionError> { let scheme = if path.contains("://") { path.split("://").next().unwrap_or("file") } else { @@ -212,7 +215,8 @@ impl IcebergScanExec { match scheme { "file" => Ok(Arc::new(OpenDalStorageFactory::Fs)), "s3" | "s3a" => { - let customized_credential_load = build_s3_credential_loader(path); + let customized_credential_load = + build_s3_credential_loader(path, catalog_properties); Ok(Arc::new(OpenDalStorageFactory::S3 { customized_credential_load, })) @@ -229,7 +233,7 @@ impl IcebergScanExec { catalog_properties: &HashMap, metadata_location: &str, ) -> Result { - let factory = Self::storage_factory_for(metadata_location)?; + let factory = Self::storage_factory_for(metadata_location, catalog_properties)?; let mut file_io_builder = FileIOBuilder::new(factory); for (key, value) in catalog_properties { @@ -240,11 +244,21 @@ impl IcebergScanExec { } } -/// Wires the registered Comet credential provider into opendal's S3 service for this scan, or -/// returns `None` so opendal falls back to its default credential chain. -fn build_s3_credential_loader(metadata_location: &str) -> Option { +/// Wires the configured Comet credential provider into opendal's S3 service for this scan, or +/// returns `None` so opendal falls back to its default credential chain. Iceberg passes its +/// per-catalog properties (`spark.sql.catalog..*` after Spark stripping), so the activation +/// key here is `s3.comet.credential.provider.class` to match Iceberg's `s3.*` namespace. +fn build_s3_credential_loader( + metadata_location: &str, + catalog_properties: &HashMap, +) -> Option { let url = url::Url::parse(metadata_location).ok()?; - let bridge = CometS3CredentialBridge::for_url(&url, AccessMode::Read)?; + let bucket = url.host_str()?; + let provider_class = catalog_properties + .get("s3.comet.credential.provider.class") + .map(|s| s.trim()) + .filter(|s| !s.is_empty())?; + let bridge = CometS3CredentialBridge::new(provider_class, bucket, url.path(), AccessMode::Read); Some(CustomAwsCredentialLoader::new(bridge)) } diff --git a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs index f9f7dcdf1b..750dc77a38 100644 --- a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs +++ b/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs @@ -19,20 +19,24 @@ //! `object_store::CredentialProvider` and `reqsign_core::ProvideCredential` for the raw Parquet //! and Iceberg scan paths respectively. //! +//! The bridge is activated by setting `fs.s3a.comet.credential.provider.class` (optionally +//! per-bucket) in the Hadoop configuration. The vendor's named class is instantiated once on +//! first use inside the JVM dispatcher and reused for the executor lifetime. +//! //! ```text //! JVM Native (Rust) //! --- ------------- //! -//! ServiceLoader s3.rs (object_store) -//! (one-time, at class-load) iceberg_scan.rs (opendal) +//! fs.s3a.comet.credential.provider.class s3.rs (object_store) +//! | iceberg_scan.rs (opendal) +//! v | +//! CometS3CredentialDispatcher v +//! (per-class instance cache) CometS3CredentialBridge +//! ^ impl object_store::CredentialProvider +//! | impl reqsign_core::ProvideCredential //! | | -//! v v -//! CometS3CredentialDispatcher CometS3CredentialBridge -//! (static singleton) impl object_store::CredentialProvider -//! | ^ impl reqsign_core::ProvideCredential -//! | | | -//! | +<---- JNI call -----------------+ -//! | getCredentialsForPath(bucket, path, mode ordinal) +//! +<---- JNI call ----------------------------+ +//! | getCredentialsForPath(className, bucket, path, mode ordinal) //! v //! vendor CometS3CredentialProvider //! | @@ -48,13 +52,12 @@ use crate::execution::operators::ExecutionError; use crate::jvm_bridge::{jni_static_call, JVMClasses}; -use crate::JAVA_VM; use async_trait::async_trait; use iceberg_storage_opendal::AwsCredential as IcebergAwsCredential; use jni::objects::{JFieldID, JObject, JString}; use jni::signature::{Primitive, ReturnType}; -use jni::sys::{jboolean, jint}; -use log::{debug, warn}; +use jni::sys::jint; +use log::warn; use object_store::aws::AwsCredential; use object_store::CredentialProvider; use once_cell::sync::OnceCell; @@ -63,15 +66,19 @@ use reqsign_core::{ Context, Error as ReqsignError, ErrorKind as ReqsignErrorKind, ProvideCredential as IcebergProvideCredential, }; +use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; use url::Url; +/// Hadoop-style config key naming the vendor `CometS3CredentialProvider` FQCN. Per-bucket form is +/// `fs.s3a.bucket..comet.credential.provider.class`. +pub const PROVIDER_CLASS_PROPERTY: &str = "comet.credential.provider.class"; + /// Cap on opendal's credential cache when the provider does not report an expiry. Prevents the /// executor from holding a stale credential for the entire job lifetime. const DEFAULT_EXPIRY_WHEN_UNKNOWN: Duration = Duration::from_secs(300); -static PROVIDER_REGISTERED: OnceCell = OnceCell::new(); /// Once-per-process latch for the "missing expiry" warning; bridges are per-scan so a per-bridge /// latch would log once per scan on the same misbehaving provider. static WARNED_MISSING_EXPIRY: OnceCell<()> = OnceCell::new(); @@ -85,44 +92,44 @@ pub enum AccessMode { Write = 1, } -/// True iff a `CometS3CredentialProvider` was discovered on the JVM classpath. -pub fn is_provider_registered() -> bool { - *PROVIDER_REGISTERED.get_or_init(|| { - // Unit tests construct stores without a JVM. Production init always precedes any store - // construction, so the None branch only fires in tests. - if JAVA_VM.get().is_none() { - return false; - } - JVMClasses::with_env(|env| -> Result { - let registered: jboolean = unsafe { - jni_static_call!(env, - comet_s3_credential_dispatcher.is_provider_registered() -> jboolean - )? - }; - Ok(registered) - }) - .unwrap_or_else(|e| { - debug!( - "CometS3CredentialDispatcher.isProviderRegistered failed; native S3 readers \ - will use the default AWS credential chain: {e}" - ); - false - }) - }) +/// Resolve the configured provider class for the given bucket, applying the per-bucket override +/// before falling back to the global key. Returns the trimmed FQCN if non-empty. +pub fn lookup_provider_class<'a>( + configs: &'a HashMap, + bucket: &str, +) -> Option<&'a str> { + let per_bucket = format!("fs.s3a.bucket.{bucket}.{PROVIDER_CLASS_PROPERTY}"); + let value = configs.get(&per_bucket).or_else(|| { + let global = format!("fs.s3a.{PROVIDER_CLASS_PROPERTY}"); + configs.get(&global) + })?; + let trimmed = value.trim(); + if trimmed.is_empty() { + None + } else { + Some(trimmed) + } } /// Per-request credential provider that delegates to the Java SPI via JNI. Constructed once per -/// S3 store or FileIO and forwards the same `(bucket, path, mode)` tuple on every fetch. +/// S3 store or FileIO and forwards the same `(class, bucket, path, mode)` tuple on every fetch. #[derive(Debug)] pub struct CometS3CredentialBridge { + provider_class: String, bucket: String, path: String, mode: AccessMode, } impl CometS3CredentialBridge { - pub fn new(bucket: impl Into, path: impl Into, mode: AccessMode) -> Self { + pub fn new( + provider_class: impl Into, + bucket: impl Into, + path: impl Into, + mode: AccessMode, + ) -> Self { Self { + provider_class: provider_class.into(), bucket: bucket.into(), path: path.into(), mode, @@ -130,18 +137,18 @@ impl CometS3CredentialBridge { } /// Shared constructor for the s3.rs and iceberg_scan.rs call sites. Returns `None` when no - /// provider is registered so callers can fall through to their default credential path. - pub fn for_url(url: &Url, mode: AccessMode) -> Option { - if !is_provider_registered() { - return None; - } - let bucket = url.host_str()?.to_string(); - debug!("Routing S3 credentials for bucket {bucket} through CometS3CredentialProvider"); - Some(Self::new(bucket, url.path(), mode)) + /// provider class is configured so callers can fall through to their default credential path. + pub fn for_url(url: &Url, configs: &HashMap, mode: AccessMode) -> Option { + let bucket = url.host_str()?; + let provider_class = lookup_provider_class(configs, bucket)?; + Some(Self::new(provider_class, bucket, url.path(), mode)) } fn fetch_raw(&self) -> Result { JVMClasses::with_env(|env| -> Result { + let provider_class = env.new_string(&self.provider_class).map_err(|e| { + ExecutionError::GeneralError(format!("new_string(provider_class): {e}")) + })?; let bucket = env .new_string(&self.bucket) .map_err(|e| ExecutionError::GeneralError(format!("new_string(bucket): {e}")))?; @@ -153,7 +160,7 @@ impl CometS3CredentialBridge { let creds_obj: JObject = unsafe { jni_static_call!(env, comet_s3_credential_dispatcher.get_credentials_for_path( - &bucket, &path, mode + &provider_class, &bucket, &path, mode ) -> JObject )? }; diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index 3e90ad6391..08117aa552 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -81,7 +81,8 @@ pub fn create_store( source: "Missing bucket name in S3 URL".into(), })?; - builder = if let Some(bridge) = CometS3CredentialBridge::for_url(url, AccessMode::Read) { + builder = if let Some(bridge) = CometS3CredentialBridge::for_url(url, configs, AccessMode::Read) + { builder.with_credentials(Arc::new(bridge)) } else { match get_runtime().block_on(build_credential_provider(configs, bucket, min_ttl))? { diff --git a/native/jni-bridge/src/comet_s3_credential_dispatcher.rs b/native/jni-bridge/src/comet_s3_credential_dispatcher.rs index 7e3a606692..8c2e7a57e3 100644 --- a/native/jni-bridge/src/comet_s3_credential_dispatcher.rs +++ b/native/jni-bridge/src/comet_s3_credential_dispatcher.rs @@ -18,7 +18,7 @@ use jni::{ errors::Result as JniResult, objects::{JClass, JFieldID, JStaticMethodID}, - signature::{Primitive, ReturnType}, + signature::ReturnType, strings::JNIString, Env, }; @@ -30,8 +30,6 @@ pub struct CometS3CredentialDispatcher<'a> { /// Retained so the cached POJO `JFieldID`s stay alive for the executor lifetime. #[allow(dead_code)] pub credentials_class: JClass<'a>, - pub method_is_provider_registered: JStaticMethodID, - pub method_is_provider_registered_ret: ReturnType, pub method_get_credentials_for_path: JStaticMethodID, pub method_get_credentials_for_path_ret: ReturnType, pub field_access_key_id: JFieldID, @@ -49,17 +47,11 @@ impl<'a> CometS3CredentialDispatcher<'a> { let credentials_class = env.find_class(JNIString::new(Self::CREDENTIALS_CLASS))?; Ok(CometS3CredentialDispatcher { - method_is_provider_registered: env.get_static_method_id( - JNIString::new(Self::JVM_CLASS), - jni::jni_str!("isProviderRegistered"), - jni::jni_sig!("()Z"), - )?, - method_is_provider_registered_ret: ReturnType::Primitive(Primitive::Boolean), method_get_credentials_for_path: env.get_static_method_id( JNIString::new(Self::JVM_CLASS), jni::jni_str!("getCredentialsForPath"), jni::jni_sig!( - "(Ljava/lang/String;Ljava/lang/String;I)Lorg/apache/comet/cloud/s3/CometS3Credentials;" + "(Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;I)Lorg/apache/comet/cloud/s3/CometS3Credentials;" ), )?, method_get_credentials_for_path_ret: ReturnType::Object, diff --git a/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider b/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider deleted file mode 100644 index 40757b041c..0000000000 --- a/spark/src/test/resources/META-INF/services/org.apache.comet.cloud.s3.CometS3CredentialProvider +++ /dev/null @@ -1 +0,0 @@ -org.apache.comet.cloud.s3.MinioCometS3CredentialProvider diff --git a/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala b/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala index 55647b253d..467a360483 100644 --- a/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala +++ b/spark/src/test/scala/org/apache/comet/CometS3TestBase.scala @@ -29,8 +29,6 @@ import org.testcontainers.utility.DockerImageName import org.apache.spark.SparkConf import org.apache.spark.sql.CometTestBase -import org.apache.comet.cloud.s3.MinioCometS3CredentialProvider - import software.amazon.awssdk.auth.credentials.{AwsBasicCredentials, StaticCredentialsProvider} import software.amazon.awssdk.services.s3.S3Client import software.amazon.awssdk.services.s3.model.{CreateBucketRequest, HeadBucketRequest} @@ -50,10 +48,6 @@ trait CometS3TestBase extends CometTestBase { minioContainer.start() createBucketIfNotExists(testBucketName) - // Install Minio's credentials into the test SPI so every S3 suite sharing this JVM routes - // through the bridge cleanly. - MinioCometS3CredentialProvider.installCredentials(userName, password) - super.beforeAll() } diff --git a/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala b/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala index e13305982d..9bd397aeb7 100644 --- a/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala @@ -39,9 +39,14 @@ class CometS3CredentialBridgeSuite extends CometS3TestBase with AdaptiveSparkPla override protected def sparkConf: SparkConf = { val conf = super.sparkConf + val providerClassName = classOf[MinioCometS3CredentialProvider].getName + // Activate the bridge for the Parquet (object_store) path via the Hadoop S3A namespace. + conf.set("spark.hadoop.fs.s3a.comet.credential.provider.class", providerClassName) + // Activate the bridge for the Iceberg (opendal) path via the per-catalog s3 namespace. conf.set("spark.sql.catalog.s3_catalog", "org.apache.iceberg.spark.SparkCatalog") conf.set("spark.sql.catalog.s3_catalog.type", "hadoop") conf.set("spark.sql.catalog.s3_catalog.warehouse", s"s3a://$testBucketName/warehouse") + conf.set("spark.sql.catalog.s3_catalog.s3.comet.credential.provider.class", providerClassName) applyS3CatalogProps(conf, "s3_catalog") conf.set(CometConf.COMET_ICEBERG_NATIVE_ENABLED.key, "true") conf From 17825c091f3014269f8071556a51592cfda5d34c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 15 May 2026 15:34:05 -0400 Subject: [PATCH 15/24] Cleanup. --- .../cloud/s3/CometS3CredentialDispatcher.java | 8 +++---- .../src/execution/operators/iceberg_scan.rs | 5 ++-- .../objectstore/comet_s3_credential_bridge.rs | 24 +++++++------------ native/core/src/parquet/objectstore/s3.rs | 2 +- 4 files changed, 15 insertions(+), 24 deletions(-) diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java index 38457c927f..369b3bf3a7 100644 --- a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java @@ -28,11 +28,9 @@ /** * JNI entry point invoked from native code to resolve a {@link CometS3CredentialProvider}. * - *

The provider class is named in {@code fs.s3a.comet.credential.provider.class} (optionally - * per-bucket). Native code reads that key from the object-store options it already forwards over - * JNI and passes the FQCN here. We instantiate each named class once via reflection and cache it, - * so a single executor JVM can serve multiple providers (e.g. one for the default config and one - * for a specific bucket) without re-instantiation per request. + *

Native code passes the FQCN named in {@code fs.s3a.comet.credential.provider.class} (or its + * per-bucket / Iceberg-namespaced variants). Each named class is instantiated once via reflection + * and cached, so a single executor JVM can serve multiple providers (e.g. one per bucket). */ public final class CometS3CredentialDispatcher { diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index ca27c8c38b..b71302edb0 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -45,7 +45,7 @@ use iceberg_storage_opendal::OpenDalStorageFactory; use crate::execution::operators::ExecutionError; use crate::parquet::objectstore::comet_s3_credential_bridge::{ - AccessMode, CometS3CredentialBridge, + AccessMode, CometS3CredentialBridge, PROVIDER_CLASS_PROPERTY, }; use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkPhysicalExprAdapterFactory; @@ -254,8 +254,9 @@ fn build_s3_credential_loader( ) -> Option { let url = url::Url::parse(metadata_location).ok()?; let bucket = url.host_str()?; + let key = format!("s3.{PROVIDER_CLASS_PROPERTY}"); let provider_class = catalog_properties - .get("s3.comet.credential.provider.class") + .get(&key) .map(|s| s.trim()) .filter(|s| !s.is_empty())?; let bridge = CometS3CredentialBridge::new(provider_class, bucket, url.path(), AccessMode::Read); diff --git a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs index 750dc77a38..2ed41e53fb 100644 --- a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs +++ b/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs @@ -71,9 +71,9 @@ use std::sync::Arc; use std::time::Duration; use url::Url; -/// Hadoop-style config key naming the vendor `CometS3CredentialProvider` FQCN. Per-bucket form is -/// `fs.s3a.bucket..comet.credential.provider.class`. -pub const PROVIDER_CLASS_PROPERTY: &str = "comet.credential.provider.class"; +/// Hadoop-style config key (without `fs.s3a.` prefix) naming the vendor `CometS3CredentialProvider` +/// FQCN. Iceberg's catalog properties use the same suffix under their `s3.` namespace. +pub(crate) const PROVIDER_CLASS_PROPERTY: &str = "comet.credential.provider.class"; /// Cap on opendal's credential cache when the provider does not report an expiry. Prevents the /// executor from holding a stale credential for the entire job lifetime. @@ -92,23 +92,15 @@ pub enum AccessMode { Write = 1, } -/// Resolve the configured provider class for the given bucket, applying the per-bucket override -/// before falling back to the global key. Returns the trimmed FQCN if non-empty. +/// Resolve the configured provider class for the given bucket via `super::s3::get_config_trimmed`, +/// which already implements the per-bucket-then-global `fs.s3a.` lookup. Returns the trimmed FQCN +/// if non-empty. pub fn lookup_provider_class<'a>( configs: &'a HashMap, bucket: &str, ) -> Option<&'a str> { - let per_bucket = format!("fs.s3a.bucket.{bucket}.{PROVIDER_CLASS_PROPERTY}"); - let value = configs.get(&per_bucket).or_else(|| { - let global = format!("fs.s3a.{PROVIDER_CLASS_PROPERTY}"); - configs.get(&global) - })?; - let trimmed = value.trim(); - if trimmed.is_empty() { - None - } else { - Some(trimmed) - } + super::s3::get_config_trimmed(configs, bucket, PROVIDER_CLASS_PROPERTY) + .filter(|s| !s.is_empty()) } /// Per-request credential provider that delegates to the Java SPI via JNI. Constructed once per diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index 08117aa552..e9a7dff7a1 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -293,7 +293,7 @@ fn get_config<'a>( }) } -fn get_config_trimmed<'a>( +pub(super) fn get_config_trimmed<'a>( configs: &'a HashMap, bucket: &str, property: &str, From d806ef6590cb3d547406646b5ce1467dd89bc662 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 18 May 2026 12:37:01 -0400 Subject: [PATCH 16/24] try to incorporate changes from #4335 and add IcebergRESTVendedS3Provider. --- .../cloud/s3/CometS3CredentialDispatcher.java | 91 ++++++-- .../cloud/s3/CometS3CredentialProvider.java | 39 +++- .../s3/CometS3CredentialDispatcherTest.java | 94 ++++++-- .../s3/TestCometS3CredentialProvider.java | 16 +- .../latest/cloud-credential-providers.md | 70 +++++- .../src/execution/operators/iceberg_scan.rs | 100 +++++++- native/core/src/execution/planner.rs | 2 + .../objectstore/comet_s3_credential_bridge.rs | 221 +++++++++++++++--- native/core/src/parquet/objectstore/s3.rs | 23 +- .../src/comet_s3_credential_dispatcher.rs | 12 +- native/proto/src/proto/operator.proto | 6 + .../comet/iceberg/IcebergReflection.scala | 77 ++++++ .../apache/comet/rules/CometScanRule.scala | 9 +- .../operator/CometIcebergNativeScan.scala | 16 +- .../cloud/s3/IcebergRESTVendedS3Provider.java | 70 ++++++ .../s3/IcebergRESTVendedS3ProviderTest.java | 90 +++++++ .../s3/MinioCometS3CredentialProvider.java | 48 ++++ .../s3/CometS3CredentialBridgeSuite.scala | 121 +++++++++- 18 files changed, 1008 insertions(+), 97 deletions(-) create mode 100644 spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java create mode 100644 spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java index 369b3bf3a7..66da294107 100644 --- a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java @@ -20,6 +20,9 @@ package org.apache.comet.cloud.s3; import java.lang.reflect.InvocationTargetException; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import org.slf4j.Logger; @@ -28,27 +31,56 @@ /** * JNI entry point invoked from native code to resolve a {@link CometS3CredentialProvider}. * - *

Native code passes the FQCN named in {@code fs.s3a.comet.credential.provider.class} (or its - * per-bucket / Iceberg-namespaced variants). Each named class is instantiated once via reflection - * and cached, so a single executor JVM can serve multiple providers (e.g. one per bucket). + *

Native code names a vendor class via the activation knob ({@code + * fs.s3a.comet.credential.provider.class} for the Parquet path, {@code + * s3.comet.credential.provider.class} on a Spark catalog property for the Iceberg path) and a + * {@code dispatchKey} that scopes the instance: catalog name on the Iceberg path, bucket name on + * the Parquet path. Each {@code (FQCN, dispatchKey)} key gets its own instance, so two catalogs + * sharing one provider class get isolated state. */ public final class CometS3CredentialDispatcher { private static final Logger LOG = LoggerFactory.getLogger(CometS3CredentialDispatcher.class); - private static final ConcurrentHashMap INSTANCES = + private static final ConcurrentHashMap INSTANCES = new ConcurrentHashMap<>(); private static final CometS3AccessMode[] MODES = CometS3AccessMode.values(); private CometS3CredentialDispatcher() {} /** - * Invoked by native code. {@code mode} is the {@link CometS3AccessMode} ordinal. - * - * @param providerClassName FQCN configured in {@code fs.s3a.comet.credential.provider.class} + * Reflects and initializes the named provider for {@code (FQCN, dispatchKey)} exactly once per + * JVM. Subsequent calls with the same key are no-ops. Native code invokes this synchronously when + * {@code CometS3CredentialBridge} is constructed at plan time, before any per-request {@link + * #getCredentialsForPath} call. {@code catalogProperties} carries the unfiltered FileIO property + * bag on the Iceberg path and is empty on the Parquet path. + */ + public static void ensureInitialized( + String providerClassName, String dispatchKey, Map catalogProperties) { + if (providerClassName == null || providerClassName.isEmpty()) { + throw new IllegalArgumentException( + "providerClassName is empty; native side should not call without a configured class"); + } + InstanceKey key = new InstanceKey(providerClassName, dispatchKey == null ? "" : dispatchKey); + Map props = + catalogProperties == null ? Collections.emptyMap() : catalogProperties; + INSTANCES.computeIfAbsent( + key, + k -> { + CometS3CredentialProvider provider = instantiate(k.providerClassName); + provider.initialize(props); + return provider; + }); + } + + /** + * Invoked by native code on every per-request credential fetch. The instance must have been + * created by a prior {@link #ensureInitialized} call; otherwise this throws. {@code mode} is the + * {@link CometS3AccessMode} ordinal. */ public static CometS3Credentials getCredentialsForPath( - String providerClassName, String bucket, String path, int mode) throws Exception { + String providerClassName, String dispatchKey, String bucket, String path, int mode) + throws Exception { if (providerClassName == null || providerClassName.isEmpty()) { throw new IllegalArgumentException( "providerClassName is empty; native side should not call without a configured class"); @@ -56,12 +88,23 @@ public static CometS3Credentials getCredentialsForPath( if (mode < 0 || mode >= MODES.length) { throw new IllegalArgumentException("Invalid CometS3AccessMode ordinal: " + mode); } - CometS3CredentialProvider provider = resolve(providerClassName); + InstanceKey key = new InstanceKey(providerClassName, dispatchKey == null ? "" : dispatchKey); + CometS3CredentialProvider provider = INSTANCES.get(key); + if (provider == null) { + throw new IllegalStateException( + "CometS3CredentialProvider " + + providerClassName + + " (dispatchKey=" + + key.dispatchKey + + ") was not initialized; ensureInitialized must be called before" + + " getCredentialsForPath"); + } CometS3AccessMode accessMode = MODES[mode]; if (LOG.isDebugEnabled()) { LOG.debug( - "Fetching credentials via {} for bucket={} path={} mode={}", + "Fetching credentials via {} (dispatchKey={}) for bucket={} path={} mode={}", providerClassName, + key.dispatchKey, bucket, path, accessMode); @@ -69,10 +112,6 @@ public static CometS3Credentials getCredentialsForPath( return provider.getCredentialsForPath(bucket, path, accessMode); } - private static CometS3CredentialProvider resolve(String providerClassName) { - return INSTANCES.computeIfAbsent(providerClassName, CometS3CredentialDispatcher::instantiate); - } - private static CometS3CredentialProvider instantiate(String providerClassName) { Class clazz; try { @@ -100,4 +139,28 @@ private static CometS3CredentialProvider instantiate(String providerClassName) { "Failed to instantiate CometS3CredentialProvider " + providerClassName, e); } } + + private static final class InstanceKey { + final String providerClassName; + final String dispatchKey; + + InstanceKey(String providerClassName, String dispatchKey) { + this.providerClassName = providerClassName; + this.dispatchKey = dispatchKey; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof InstanceKey)) return false; + InstanceKey other = (InstanceKey) o; + return providerClassName.equals(other.providerClassName) + && dispatchKey.equals(other.dispatchKey); + } + + @Override + public int hashCode() { + return Objects.hash(providerClassName, dispatchKey); + } + } } diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java index 6181e4b1fa..536df65bbb 100644 --- a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java +++ b/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java @@ -19,6 +19,8 @@ package org.apache.comet.cloud.s3; +import java.util.Map; + /** * SPI for supplying AWS credentials to Comet's native S3 readers, which bypass Spark's Hadoop S3A * code path and cannot reach signer-based or path-aware credential mechanisms through the standard @@ -47,9 +49,29 @@ * *

Vendors register an implementation by setting {@code * spark.hadoop.fs.s3a.comet.credential.provider.class} (or the per-bucket form {@code - * spark.hadoop.fs.s3a.bucket..comet.credential.provider.class}) to the implementing FQCN. The - * class must have a public no-arg constructor. {@link #getCredentialsForPath} may be invoked - * concurrently from many native tokio tasks, so implementations must be thread-safe. + * spark.hadoop.fs.s3a.bucket..comet.credential.provider.class}) for the Parquet path, or + * {@code spark.sql.catalog..s3.comet.credential.provider.class} for the Iceberg path. The + * class must have a public no-arg constructor. + * + *

Lifecycle

+ * + *

Comet keys provider instances by {@code (FQCN, dispatchKey)}, where {@code dispatchKey} is the + * Spark V2 catalog name on the Iceberg path and the bucket on the Parquet path. The first time a + * given key is seen on an executor, Comet reflects the class, calls {@link #initialize(Map)} once, + * and caches the instance. Subsequent requests for the same key reuse it. Two catalogs that share + * one FQCN therefore get isolated instances with their own {@code initialize} maps. + * + *

{@link #initialize(Map)} should be cheap and non-blocking; defer real credential fetches to + * the first {@link #getCredentialsForPath} call. {@link #getCredentialsForPath} may be invoked + * concurrently from many native tokio worker threads, so implementations must be thread-safe. + * + *

Caching, refresh, and distribution are the vendor's job

+ * + *

Comet does not maintain a TTL cache, broadcast catalog state, or schedule refresh. Vendors + * decide whether to cache (e.g. by wrapping {@code + * org.apache.iceberg.aws.s3.VendedCredentialsProvider}'s {@code CachedSupplier}), when to refresh, + * and how to distribute driver-side state to executors (typically by reading {@link #initialize}'s + * {@code catalogProperties}, which Comet has already serialized through the native plan op). * *

Returns credentials or throws; there is no fall-through return value. A provider that is only * authoritative for some paths should resolve the default AWS chain itself for the rest. See the @@ -57,6 +79,17 @@ */ public interface CometS3CredentialProvider { + /** + * Called once per {@code (FQCN, dispatchKey)} on each executor before any {@link + * #getCredentialsForPath} call. The {@code catalogProperties} map carries the full FileIO + * property bag for the Iceberg path (including {@code credentials.uri}, OAuth tokens, vendor keys + * like {@code tenant-id}) and is empty on the Parquet path. The default no-op keeps Parquet + * vendors source-compatible. + * + * @param catalogProperties unfiltered FileIO/catalog properties; may contain secrets, do not log + */ + default void initialize(Map catalogProperties) {} + /** * @param bucket S3 bucket name (no scheme, no path) * @param path object key or prefix, leading slash included (matches the URL path component) diff --git a/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java b/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java index c0066e2b7a..1bf09712bf 100644 --- a/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java +++ b/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java @@ -19,6 +19,10 @@ package org.apache.comet.cloud.s3; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + import org.junit.Before; import org.junit.Test; @@ -32,6 +36,7 @@ public class CometS3CredentialDispatcherTest { private static final String TEST_PROVIDER = TestCometS3CredentialProvider.class.getName(); + private static final String DK = "test-dispatch-key"; private static final int READ = CometS3AccessMode.READ.ordinal(); private static final int WRITE = CometS3AccessMode.WRITE.ordinal(); @@ -40,11 +45,16 @@ public void resetTestProvider() { TestCometS3CredentialProvider.reset(); } + private void init() { + CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, DK, Collections.emptyMap()); + } + @Test public void getCredentialsRoundTripsThroughProvider() throws Exception { + init(); CometS3Credentials creds = CometS3CredentialDispatcher.getCredentialsForPath( - TEST_PROVIDER, "my-bucket", "path/to/object", READ); + TEST_PROVIDER, DK, "my-bucket", "path/to/object", READ); assertNotNull(creds); assertEquals("AKIATEST", creds.getAccessKeyId()); @@ -60,25 +70,30 @@ public void getCredentialsRoundTripsThroughProvider() throws Exception { @Test public void writeModeIsForwarded() throws Exception { - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b", "k", WRITE); + init(); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b", "k", WRITE); assertEquals(CometS3AccessMode.WRITE, TestCometS3CredentialProvider.lastMode); } @Test public void unknownModeRejected() { + init(); assertThrows( IllegalArgumentException.class, - () -> CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b", "k", 99)); + () -> CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b", "k", 99)); } @Test public void emptyClassNameRejected() { assertThrows( IllegalArgumentException.class, - () -> CometS3CredentialDispatcher.getCredentialsForPath("", "b", "k", READ)); + () -> CometS3CredentialDispatcher.ensureInitialized("", DK, Collections.emptyMap())); + assertThrows( + IllegalArgumentException.class, + () -> CometS3CredentialDispatcher.ensureInitialized(null, DK, Collections.emptyMap())); assertThrows( IllegalArgumentException.class, - () -> CometS3CredentialDispatcher.getCredentialsForPath(null, "b", "k", READ)); + () -> CometS3CredentialDispatcher.getCredentialsForPath("", DK, "b", "k", READ)); } @Test @@ -87,8 +102,8 @@ public void missingClassReportsActionableError() { assertThrows( IllegalStateException.class, () -> - CometS3CredentialDispatcher.getCredentialsForPath( - "com.does.not.Exist", "b", "k", READ)); + CometS3CredentialDispatcher.ensureInitialized( + "com.does.not.Exist", DK, Collections.emptyMap())); assertTrue(thrown.getMessage().contains("not found")); } @@ -98,8 +113,8 @@ public void classNotImplementingInterfaceRejected() { assertThrows( IllegalStateException.class, () -> - CometS3CredentialDispatcher.getCredentialsForPath( - NotACredentialProvider.class.getName(), "b", "k", READ)); + CometS3CredentialDispatcher.ensureInitialized( + NotACredentialProvider.class.getName(), DK, Collections.emptyMap())); assertTrue(thrown.getMessage().contains("does not implement")); } @@ -109,38 +124,85 @@ public void classWithoutNoArgCtorRejected() { assertThrows( IllegalStateException.class, () -> - CometS3CredentialDispatcher.getCredentialsForPath( - NoNoArgCtorProvider.class.getName(), "b", "k", READ)); + CometS3CredentialDispatcher.ensureInitialized( + NoNoArgCtorProvider.class.getName(), DK, Collections.emptyMap())); assertTrue(thrown.getMessage().contains("no-arg constructor")); } + @Test + public void getWithoutEnsureInitializedThrows() { + Exception thrown = + assertThrows( + IllegalStateException.class, + () -> + CometS3CredentialDispatcher.getCredentialsForPath( + TEST_PROVIDER, "never-initialized", "b", "k", READ)); + assertTrue(thrown.getMessage().contains("not initialized")); + } + + @Test + public void initializeCalledExactlyOncePerKey() { + Map props = new HashMap<>(); + props.put("tenant-id", "T1"); + + CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); + CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); + CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); + + assertEquals(1, TestCometS3CredentialProvider.initCount.get()); + } + + @Test + public void distinctDispatchKeysIsolateInstances() throws Exception { + Map propsA = new HashMap<>(); + propsA.put("tenant-id", "T-A"); + Map propsB = new HashMap<>(); + propsB.put("tenant-id", "T-B"); + + CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "iso-cat-a", propsA); + CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "iso-cat-b", propsB); + + assertEquals(2, TestCometS3CredentialProvider.initCount.get()); + + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "iso-cat-a", "b", "k", READ); + assertEquals("T-A", TestCometS3CredentialProvider.lastTenantSeen); + + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "iso-cat-b", "b", "k", READ); + assertEquals("T-B", TestCometS3CredentialProvider.lastTenantSeen); + } + @Test public void providerExceptionsPropagate() { + init(); IllegalStateException boom = new IllegalStateException("simulated STS failure"); TestCometS3CredentialProvider.throwOnNext = boom; Exception thrown = assertThrows( Exception.class, - () -> CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b", "k", READ)); + () -> + CometS3CredentialDispatcher.getCredentialsForPath( + TEST_PROVIDER, DK, "b", "k", READ)); assertSame(boom, thrown); } @Test public void nullSessionTokenAllowed() throws Exception { + init(); TestCometS3CredentialProvider.nextResult = new CometS3Credentials("AKIA", "sec", null, 0L); CometS3Credentials creds = - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b", "k", READ); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b", "k", READ); assertNull(creds.getSessionToken()); } @Test public void providerReceivesEachCallSeparately() throws Exception { - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b1", "k1", READ); - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b2", "k2", READ); - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "b3", "k3", READ); + init(); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b1", "k1", READ); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b2", "k2", READ); + CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b3", "k3", READ); assertEquals(3, TestCometS3CredentialProvider.callCount.get()); assertEquals("b3", TestCometS3CredentialProvider.lastBucket); diff --git a/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java b/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java index d0eb0a1dd2..53595a9cf7 100644 --- a/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java +++ b/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java @@ -19,31 +19,44 @@ package org.apache.comet.cloud.s3; +import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; /** * Test-only provider instantiated by the dispatcher when its FQCN is passed across JNI. State is - * static because the dispatcher caches one instance per class name for the JVM lifetime. + * static because the dispatcher caches one instance per (FQCN, dispatchKey) for the JVM lifetime. */ public class TestCometS3CredentialProvider implements CometS3CredentialProvider { + static final AtomicInteger initCount = new AtomicInteger(0); static final AtomicInteger callCount = new AtomicInteger(0); static volatile String lastBucket; static volatile String lastPath; static volatile CometS3AccessMode lastMode; + static volatile String lastTenantSeen; static volatile RuntimeException throwOnNext; static volatile CometS3Credentials nextResult = new CometS3Credentials("AKIATEST", "secret", "session-tok", 0L); + private volatile String tenantId; + static void reset() { + initCount.set(0); callCount.set(0); lastBucket = null; lastPath = null; lastMode = null; + lastTenantSeen = null; throwOnNext = null; nextResult = new CometS3Credentials("AKIATEST", "secret", "session-tok", 0L); } + @Override + public void initialize(Map catalogProperties) { + initCount.incrementAndGet(); + this.tenantId = catalogProperties.get("tenant-id"); + } + @Override public CometS3Credentials getCredentialsForPath( String bucket, String path, CometS3AccessMode mode) { @@ -51,6 +64,7 @@ public CometS3Credentials getCredentialsForPath( lastBucket = bucket; lastPath = path; lastMode = mode; + lastTenantSeen = tenantId; RuntimeException toThrow = throwOnNext; if (toThrow != null) { throwOnNext = null; diff --git a/docs/source/user-guide/latest/cloud-credential-providers.md b/docs/source/user-guide/latest/cloud-credential-providers.md index 5d24322114..0bf404d8fd 100644 --- a/docs/source/user-guide/latest/cloud-credential-providers.md +++ b/docs/source/user-guide/latest/cloud-credential-providers.md @@ -71,7 +71,7 @@ OSS Comet ships no vendor-specific bridges. Get one from the same vendor that su With the config set and the JAR on the classpath, executor logs show on first S3 access: - Info level: `Instantiated CometS3CredentialProvider ` -- Debug level: `Fetching credentials via for bucket=... path=... mode=...` +- Debug level: `Fetching credentials via (dispatchKey=) for bucket=... path=... mode=...` Without the config set, no credential-related log lines appear at startup; native readers use the default AWS credential chain. @@ -81,7 +81,9 @@ Without the config set, no credential-related log lines appear at startup; nativ **` does not implement org.apache.comet.cloud.s3.CometS3CredentialProvider`**. The configured class exists but does not implement the SPI. Double-check the FQCN against the vendor's documentation. -**` must declare a public no-arg constructor`**. Vendor classes are instantiated reflectively with `Class.forName(name).getDeclaredConstructor().newInstance()`. A non-default constructor is not supported; ask the vendor to expose a no-arg form that reads any state it needs from environment or system properties. +**` must declare a public no-arg constructor`**. Vendor classes are instantiated reflectively with `Class.forName(name).getDeclaredConstructor().newInstance()`. A non-default constructor is not supported; ask the vendor to expose a no-arg form that reads any state it needs from `initialize(Map)` or environment. + +**`CometS3CredentialProvider (dispatchKey=...) was not initialized`**. `initialize(Map)` was not called before a credential request. Comet should always invoke `ensureInitialized` synchronously when it builds the bridge at plan time, so this indicates the bridge skipped the init call (a Comet bug) or the vendor's `initialize` threw and the bridge fell through to the default chain. **`403 AccessDenied` with the bridge configured.** The provider returned credentials but they were rejected by S3. Most often a region mismatch (see Iceberg section below) or expired session token; enable debug logging on the vendor's class to confirm what it returned. @@ -109,6 +111,9 @@ Implement `org.apache.comet.cloud.s3.CometS3CredentialProvider`: package org.apache.comet.cloud.s3; public interface CometS3CredentialProvider { + /** Called once per (FQCN, dispatchKey) before any per-request call. Optional. */ + default void initialize(java.util.Map catalogProperties) {} + CometS3Credentials getCredentialsForPath( String bucket, String path, CometS3AccessMode mode) throws Exception; } @@ -116,6 +121,22 @@ public interface CometS3CredentialProvider { The class must have a public no-arg constructor. `getCredentialsForPath` may be invoked concurrently from many native tokio worker threads; the implementation must be thread-safe. +### Lifecycle + +Comet keys provider instances by `(FQCN, dispatchKey)`. The dispatch key is the Spark V2 catalog name on the Iceberg path and the S3 bucket name on the Parquet path. The first time a given key is seen on an executor, Comet reflects the class, calls `initialize(Map)` exactly once, and caches the instance for the JVM lifetime. Two catalogs sharing one provider FQCN therefore get isolated instances with their own `initialize` maps. + +`initialize` should be cheap and non-blocking. Defer real credential fetches (REST round-trips, STS calls) to the first `getCredentialsForPath` invocation. On the Iceberg path the supplied `catalogProperties` carries the unfiltered FileIO bag, including REST-vended fields like `credentials.uri`, OAuth tokens, and any vendor-custom keys you set on the catalog config. The map may contain secrets, so do not log it. + +### Caching, refresh, and distribution + +Comet does not maintain a TTL cache, broadcast catalog state, or schedule refresh. Vendors decide: + +- Whether to cache credentials and for how long. Iceberg vendors get `software.amazon.awssdk.utils.cache.CachedSupplier` for free inside `VendedCredentialsProvider`; vendors with custom STS write whatever cache fits. +- When to refresh: proactive timer, on-demand at expiry, on `403` retry, etc. +- How to distribute driver-only state. Read it from `initialize`'s `catalogProperties` (which Comet has already serialized through the native plan op), call back to a vendor service from the executor, or run your own Spark broadcast inside the class. + +`expirationEpochMillis` on the returned `CometS3Credentials` is metadata pass-through, not a Comet-owned cache. Comet forwards it to `object_store` and opendal, which already have their own credential caches and use the expiry to schedule the next refresh. Publish a real expiry when you have one; return `0` if you do not, and a conservative 5-minute floor is applied so a stale credential cannot live indefinitely. + ### Returned fields | Field | Notes | @@ -172,6 +193,51 @@ public final class MyCometCredentialProvider implements CometS3CredentialProvide Per-bucket Hadoop overrides (`fs.s3a.bucket..comet.credential.provider.class`) are also available if you prefer to ship multiple vendor classes and pick by bucket in config rather than in code. +For Iceberg deployments where two catalogs share one provider class but need isolated state, configure the same FQCN on both catalogs and read your discriminator from `initialize`'s `catalogProperties`. Each catalog gets its own provider instance because Comet keys by `(FQCN, catalogName)`: + +```java +public final class MyMultiTenantProvider implements CometS3CredentialProvider { + private volatile String tenantId; + + @Override + public void initialize(Map catalogProperties) { + this.tenantId = catalogProperties.get("vendor.tenant-id"); + } + + @Override + public CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) { + return mintForTenant(tenantId, bucket, path, mode); + } +} +``` + +### Reference implementation: Iceberg REST vended credentials + +For Iceberg REST catalogs that vend AWS credentials (`LoadTableResponse.credentials`), the canonical implementation wraps Iceberg's existing `VendedCredentialsProvider`: + +```java +public final class IcebergRESTVendedS3Provider implements CometS3CredentialProvider { + private volatile VendedCredentialsProvider provider; + + @Override + public void initialize(Map catalogProperties) { + this.provider = VendedCredentialsProvider.create(catalogProperties); + } + + @Override + public CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) { + AwsCredentials c = provider.resolveCredentials(); + String token = (c instanceof AwsSessionCredentials) + ? ((AwsSessionCredentials) c).sessionToken() : null; + return new CometS3Credentials(c.accessKeyId(), c.secretAccessKey(), token, 0L); + } +} +``` + +`VendedCredentialsProvider` reads `credentials.uri`, the catalog endpoint, and OAuth tokens from the supplied map (Comet forwards the unfiltered FileIO bag to `initialize`), and refreshes through its own `CachedSupplier`. Caching, refresh-near-expiry, and the REST round-trip all live in Iceberg, not in Comet. Comet ships a copy of this class under `spark/src/test` for now; promote it to your runtime classpath alongside `iceberg-aws` and AWS SDK v2. + ### Access mode | Value | Used for | diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index b71302edb0..a7e46adc99 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -45,7 +45,7 @@ use iceberg_storage_opendal::OpenDalStorageFactory; use crate::execution::operators::ExecutionError; use crate::parquet::objectstore::comet_s3_credential_bridge::{ - AccessMode, CometS3CredentialBridge, PROVIDER_CLASS_PROPERTY, + AccessMode, CometS3CredentialBridge, ICEBERG_PROVIDER_CLASS_PROPERTY, }; use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkPhysicalExprAdapterFactory; @@ -56,7 +56,6 @@ use iceberg::scan::FileScanTask; /// Iceberg table scan operator that uses iceberg-rust to read Iceberg tables. /// /// Executes pre-planned FileScanTasks for efficient parallel scanning. -#[derive(Debug)] pub struct IcebergScanExec { /// Iceberg table metadata location for FileIO initialization metadata_location: String, @@ -64,8 +63,14 @@ pub struct IcebergScanExec { output_schema: SchemaRef, /// Cached execution plan properties plan_properties: Arc, - /// Catalog-specific configuration for FileIO + /// Catalog-specific configuration for FileIO. Holds the unfiltered FileIO property bag, which + /// may contain OAuth tokens, REST `credentials.uri`, and other secrets the credential bridge + /// needs. Redacted in `Debug` so plan dumps and tracing do not leak credentials. catalog_properties: HashMap, + /// Spark V2 catalog name; forwarded as dispatchKey to the credential bridge so multiple + /// catalogs sharing one provider FQCN get isolated provider instances. Empty when the table + /// has no catalog identity. + catalog_name: String, /// Pre-planned file scan tasks tasks: Vec, /// Number of data files to read concurrently @@ -79,6 +84,7 @@ impl IcebergScanExec { metadata_location: String, schema: SchemaRef, catalog_properties: HashMap, + catalog_name: String, tasks: Vec, data_file_concurrency_limit: usize, ) -> Result { @@ -92,6 +98,7 @@ impl IcebergScanExec { output_schema, plan_properties, catalog_properties, + catalog_name, tasks, data_file_concurrency_limit, metrics, @@ -158,7 +165,11 @@ impl IcebergScanExec { context: Arc, ) -> DFResult { let output_schema = Arc::clone(&self.output_schema); - let file_io = Self::load_file_io(&self.catalog_properties, &self.metadata_location)?; + let file_io = Self::load_file_io( + &self.catalog_properties, + &self.metadata_location, + &self.catalog_name, + )?; let batch_size = context.session_config().batch_size(); let metrics = IcebergScanMetrics::new(&self.metrics); @@ -206,6 +217,7 @@ impl IcebergScanExec { fn storage_factory_for( path: &str, catalog_properties: &HashMap, + catalog_name: &str, ) -> Result, DataFusionError> { let scheme = if path.contains("://") { path.split("://").next().unwrap_or("file") @@ -216,7 +228,7 @@ impl IcebergScanExec { "file" => Ok(Arc::new(OpenDalStorageFactory::Fs)), "s3" | "s3a" => { let customized_credential_load = - build_s3_credential_loader(path, catalog_properties); + build_s3_credential_loader(path, catalog_properties, catalog_name); Ok(Arc::new(OpenDalStorageFactory::S3 { customized_credential_load, })) @@ -232,18 +244,27 @@ impl IcebergScanExec { fn load_file_io( catalog_properties: &HashMap, metadata_location: &str, + catalog_name: &str, ) -> Result { - let factory = Self::storage_factory_for(metadata_location, catalog_properties)?; + let factory = + Self::storage_factory_for(metadata_location, catalog_properties, catalog_name)?; let mut file_io_builder = FileIOBuilder::new(factory); + // Narrow to storage-prefix keys before forwarding to iceberg-rust's FileIO. The full + // unfiltered bag (catalog URI, OAuth tokens, credentials.uri, tenant-id, etc.) is kept + // upstream so CometS3CredentialBridge can read whatever the vendor needs. for (key, value) in catalog_properties { - file_io_builder = file_io_builder.with_prop(key, value); + if STORAGE_PROPERTY_PREFIXES.iter().any(|p| key.starts_with(p)) { + file_io_builder = file_io_builder.with_prop(key, value); + } } Ok(file_io_builder.build()) } } +const STORAGE_PROPERTY_PREFIXES: &[&str] = &["s3.", "gcs.", "adls.", "client."]; + /// Wires the configured Comet credential provider into opendal's S3 service for this scan, or /// returns `None` so opendal falls back to its default credential chain. Iceberg passes its /// per-catalog properties (`spark.sql.catalog..*` after Spark stripping), so the activation @@ -251,16 +272,41 @@ impl IcebergScanExec { fn build_s3_credential_loader( metadata_location: &str, catalog_properties: &HashMap, + catalog_name: &str, ) -> Option { let url = url::Url::parse(metadata_location).ok()?; let bucket = url.host_str()?; - let key = format!("s3.{PROVIDER_CLASS_PROPERTY}"); let provider_class = catalog_properties - .get(&key) + .get(ICEBERG_PROVIDER_CLASS_PROPERTY) .map(|s| s.trim()) .filter(|s| !s.is_empty())?; - let bridge = CometS3CredentialBridge::new(provider_class, bucket, url.path(), AccessMode::Read); - Some(CustomAwsCredentialLoader::new(bridge)) + // Catalog name scopes provider instances on the JVM dispatcher so two catalogs sharing one + // provider class get isolated state. Falls back to the bucket when the table has no catalog + // identity (e.g. HadoopTables loaded by raw path), keeping the previous behavior in that + // case. + let dispatch_key: &str = if catalog_name.is_empty() { + bucket + } else { + catalog_name + }; + let bridge = CometS3CredentialBridge::new( + provider_class, + dispatch_key, + bucket, + url.path(), + AccessMode::Read, + catalog_properties, + ); + match bridge { + Ok(b) => Some(CustomAwsCredentialLoader::new(b)), + Err(e) => { + log::warn!( + "Failed to initialize CometS3CredentialBridge for {provider_class}: {e}; \ + falling back to default opendal credential chain" + ); + None + } + } } /// Metrics for IcebergScanExec @@ -391,6 +437,38 @@ impl DisplayAs for IcebergScanExec { } } +impl fmt::Debug for IcebergScanExec { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("IcebergScanExec") + .field("metadata_location", &self.metadata_location) + .field("catalog_name", &self.catalog_name) + .field( + "catalog_properties", + &RedactedProperties(&self.catalog_properties), + ) + .field("num_tasks", &self.tasks.len()) + .field( + "data_file_concurrency_limit", + &self.data_file_concurrency_limit, + ) + .finish_non_exhaustive() + } +} + +/// Wraps a property map so `Debug` shows keys but elides values, since the unfiltered FileIO bag +/// may contain bearer tokens and OAuth secrets. +struct RedactedProperties<'a>(&'a HashMap); + +impl fmt::Debug for RedactedProperties<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let mut m = f.debug_map(); + for k in self.0.keys() { + m.key(k).value(&""); + } + m.finish() + } +} + /// Build projection expressions that adapt batches from a file schema to the target schema. /// /// The returned expressions can be cached and reused across multiple batches diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index debe47ba04..183ce463a4 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1472,6 +1472,7 @@ impl PhysicalPlanner { .map(|(k, v)| (k.clone(), v.clone())) .collect(); let metadata_location = common.metadata_location.clone(); + let catalog_name = common.catalog_name.clone(); let tasks = parse_file_scan_tasks_from_common(common, &scan.file_scan_tasks)?; let data_file_concurrency_limit = common.data_file_concurrency_limit as usize; @@ -1479,6 +1480,7 @@ impl PhysicalPlanner { metadata_location, required_schema, catalog_properties, + catalog_name, tasks, data_file_concurrency_limit, )?; diff --git a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs b/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs index 2ed41e53fb..995b632011 100644 --- a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs +++ b/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs @@ -51,11 +51,12 @@ //! ``` use crate::execution::operators::ExecutionError; -use crate::jvm_bridge::{jni_static_call, JVMClasses}; +use crate::jvm_bridge::{jni_new_global_ref, jni_static_call, JVMClasses}; use async_trait::async_trait; use iceberg_storage_opendal::AwsCredential as IcebergAwsCredential; -use jni::objects::{JFieldID, JObject, JString}; +use jni::objects::{Global, JFieldID, JObject, JString, JValue}; use jni::signature::{Primitive, ReturnType}; +use jni::strings::JNIString; use jni::sys::jint; use log::warn; use object_store::aws::AwsCredential; @@ -67,6 +68,7 @@ use reqsign_core::{ ProvideCredential as IcebergProvideCredential, }; use std::collections::HashMap; +use std::fmt; use std::sync::Arc; use std::time::Duration; use url::Url; @@ -75,6 +77,10 @@ use url::Url; /// FQCN. Iceberg's catalog properties use the same suffix under their `s3.` namespace. pub(crate) const PROVIDER_CLASS_PROPERTY: &str = "comet.credential.provider.class"; +/// Iceberg-namespaced form of [`PROVIDER_CLASS_PROPERTY`], read from a Spark catalog's `s3.*` +/// property bag (`spark.sql.catalog..s3.comet.credential.provider.class`). +pub(crate) const ICEBERG_PROVIDER_CLASS_PROPERTY: &str = "s3.comet.credential.provider.class"; + /// Cap on opendal's credential cache when the provider does not report an expiry. Prevents the /// executor from holding a stale credential for the entire job lifetime. const DEFAULT_EXPIRY_WHEN_UNKNOWN: Duration = Duration::from_secs(300); @@ -103,56 +109,140 @@ pub fn lookup_provider_class<'a>( .filter(|s| !s.is_empty()) } -/// Per-request credential provider that delegates to the Java SPI via JNI. Constructed once per -/// S3 store or FileIO and forwards the same `(class, bucket, path, mode)` tuple on every fetch. -#[derive(Debug)] +/// Per-request credential provider that delegates to the Java SPI via JNI. Constructed once per S3 +/// store or FileIO; calls `ensureInitialized` synchronously at construction so the JVM provider +/// instance is ready before any per-request fetch. +/// +/// The four String arguments threaded through every `getCredentialsForPath` call (provider class, +/// dispatch key, bucket, path) are immutable for the bridge's lifetime, so we cache them once at +/// construction as JNI global refs to avoid per-call `env.new_string` allocations on the hot path. pub struct CometS3CredentialBridge { provider_class: String, + dispatch_key: String, bucket: String, path: String, mode: AccessMode, + /// Cached JNI globals for the four constant String arguments to `getCredentialsForPath`. + provider_class_jstr: Arc>>, + dispatch_key_jstr: Arc>>, + bucket_jstr: Arc>>, + path_jstr: Arc>>, +} + +impl fmt::Debug for CometS3CredentialBridge { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("CometS3CredentialBridge") + .field("provider_class", &self.provider_class) + .field("dispatch_key", &self.dispatch_key) + .field("bucket", &self.bucket) + .field("path", &self.path) + .field("mode", &self.mode) + .finish() + } } impl CometS3CredentialBridge { + /// Construct the bridge and run a one-shot `ensureInitialized` call against the JVM + /// dispatcher. `dispatch_key` scopes provider instances on the JVM side: bucket name on the + /// Parquet path, catalog name on the Iceberg path. `catalog_properties` is forwarded to + /// `CometS3CredentialProvider.initialize(Map)` exactly once per `(class, dispatchKey)` pair. pub fn new( provider_class: impl Into, + dispatch_key: impl Into, bucket: impl Into, path: impl Into, mode: AccessMode, - ) -> Self { - Self { - provider_class: provider_class.into(), - bucket: bucket.into(), - path: path.into(), + catalog_properties: &HashMap, + ) -> Result { + let provider_class = provider_class.into(); + let dispatch_key = dispatch_key.into(); + let bucket = bucket.into(); + let path = path.into(); + + let (provider_class_jstr, dispatch_key_jstr, bucket_jstr, path_jstr) = + JVMClasses::with_env(|env| -> Result<_, ExecutionError> { + let pc = env.new_string(&provider_class).map_err(|e| { + ExecutionError::GeneralError(format!("new_string(provider_class): {e}")) + })?; + let dk = env.new_string(&dispatch_key).map_err(|e| { + ExecutionError::GeneralError(format!("new_string(dispatch_key): {e}")) + })?; + let b = env.new_string(&bucket).map_err(|e| { + ExecutionError::GeneralError(format!("new_string(bucket): {e}")) + })?; + let p = env + .new_string(&path) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(path): {e}")))?; + let pc_g = Arc::new(jni_new_global_ref!(env, pc).map_err(|e| { + ExecutionError::GeneralError(format!("global_ref(provider_class): {e}")) + })?); + let dk_g = Arc::new(jni_new_global_ref!(env, dk).map_err(|e| { + ExecutionError::GeneralError(format!("global_ref(dispatch_key): {e}")) + })?); + let b_g = Arc::new(jni_new_global_ref!(env, b).map_err(|e| { + ExecutionError::GeneralError(format!("global_ref(bucket): {e}")) + })?); + let p_g = + Arc::new(jni_new_global_ref!(env, p).map_err(|e| { + ExecutionError::GeneralError(format!("global_ref(path): {e}")) + })?); + Ok((pc_g, dk_g, b_g, p_g)) + })?; + + ensure_initialized(&provider_class, &dispatch_key, catalog_properties)?; + Ok(Self { + provider_class, + dispatch_key, + bucket, + path, mode, - } + provider_class_jstr, + dispatch_key_jstr, + bucket_jstr, + path_jstr, + }) } - /// Shared constructor for the s3.rs and iceberg_scan.rs call sites. Returns `None` when no + /// Shared constructor for the s3.rs and iceberg_scan.rs call sites. Returns `Ok(None)` when no /// provider class is configured so callers can fall through to their default credential path. - pub fn for_url(url: &Url, configs: &HashMap, mode: AccessMode) -> Option { - let bucket = url.host_str()?; - let provider_class = lookup_provider_class(configs, bucket)?; - Some(Self::new(provider_class, bucket, url.path(), mode)) + pub fn for_url( + url: &Url, + configs: &HashMap, + mode: AccessMode, + dispatch_key: &str, + catalog_properties: &HashMap, + ) -> Result, ExecutionError> { + let bucket = match url.host_str() { + Some(b) => b, + None => return Ok(None), + }; + let provider_class = match lookup_provider_class(configs, bucket) { + Some(c) => c.to_string(), + None => return Ok(None), + }; + Self::new( + provider_class, + dispatch_key, + bucket, + url.path(), + mode, + catalog_properties, + ) + .map(Some) } fn fetch_raw(&self) -> Result { JVMClasses::with_env(|env| -> Result { - let provider_class = env.new_string(&self.provider_class).map_err(|e| { - ExecutionError::GeneralError(format!("new_string(provider_class): {e}")) - })?; - let bucket = env - .new_string(&self.bucket) - .map_err(|e| ExecutionError::GeneralError(format!("new_string(bucket): {e}")))?; - let path = env - .new_string(&self.path) - .map_err(|e| ExecutionError::GeneralError(format!("new_string(path): {e}")))?; let mode = self.mode as jint; let creds_obj: JObject = unsafe { jni_static_call!(env, comet_s3_credential_dispatcher.get_credentials_for_path( - &provider_class, &bucket, &path, mode + self.provider_class_jstr.as_obj(), + self.dispatch_key_jstr.as_obj(), + self.bucket_jstr.as_obj(), + self.path_jstr.as_obj(), + mode ) -> JObject )? }; @@ -193,6 +283,85 @@ impl CometS3CredentialBridge { } } +fn ensure_initialized( + provider_class: &str, + dispatch_key: &str, + catalog_properties: &HashMap, +) -> Result<(), ExecutionError> { + JVMClasses::with_env(|env| -> Result<(), ExecutionError> { + let provider_class_jstr = env.new_string(provider_class).map_err(|e| { + ExecutionError::GeneralError(format!("new_string(provider_class): {e}")) + })?; + let dispatch_key_jstr = env + .new_string(dispatch_key) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(dispatch_key): {e}")))?; + let props_obj = build_java_string_map(env, catalog_properties)?; + + unsafe { + jni_static_call!(env, + comet_s3_credential_dispatcher.ensure_initialized( + &provider_class_jstr, &dispatch_key_jstr, &props_obj + ) -> () + )?; + } + Ok(()) + }) +} + +/// Construct a `java.util.HashMap` and populate it. Called once per bridge at +/// construction (per-scan), so the per-call HashMap/put cost is amortized away from the hot path. +fn build_java_string_map<'a>( + env: &mut jni::Env<'a>, + map: &HashMap, +) -> Result, ExecutionError> { + let hashmap_class = env + .find_class(JNIString::new("java/util/HashMap")) + .map_err(|e| ExecutionError::GeneralError(format!("find_class(HashMap): {e}")))?; + let ctor = env + .get_method_id( + &hashmap_class, + jni::jni_str!(""), + jni::jni_sig!("(I)V"), + ) + .map_err(|e| ExecutionError::GeneralError(format!("HashMap.(I): {e}")))?; + let put = env + .get_method_id( + &hashmap_class, + jni::jni_str!("put"), + jni::jni_sig!("(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;"), + ) + .map_err(|e| ExecutionError::GeneralError(format!("HashMap.put: {e}")))?; + let initial_capacity = JValue::Int(map.len() as jint); + let instance = + unsafe { env.new_object_unchecked(&hashmap_class, ctor, &[initial_capacity.as_jni()]) } + .map_err(|e| ExecutionError::GeneralError(format!("new HashMap(int): {e}")))?; + + for (k, v) in map { + let k_jstr = env + .new_string(k) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(key): {e}")))?; + let v_jstr = env + .new_string(v) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(value): {e}")))?; + let prev = unsafe { + env.call_method_unchecked( + &instance, + put, + ReturnType::Object, + &[ + JValue::Object(&k_jstr).as_jni(), + JValue::Object(&v_jstr).as_jni(), + ], + ) + } + .map_err(|e| ExecutionError::GeneralError(format!("HashMap.put call: {e}")))?; + // Discard return value; Java would have reused the existing key but our maps have no dupes. + let _ = prev.l(); + } + + Ok(instance) +} + struct RawCredentials { access_key_id: String, secret_access_key: String, diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index e9a7dff7a1..e524db81cc 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -81,8 +81,27 @@ pub fn create_store( source: "Missing bucket name in S3 URL".into(), })?; - builder = if let Some(bridge) = CometS3CredentialBridge::for_url(url, configs, AccessMode::Read) - { + // Parquet path: dispatch_key = bucket (matches the per-bucket override config namespace); + // catalog_properties is empty since vendors on the Parquet path read from Hadoop conf, not + // catalog props. + let empty_props: HashMap = HashMap::new(); + let bridge = match CometS3CredentialBridge::for_url( + url, + configs, + AccessMode::Read, + bucket, + &empty_props, + ) { + Ok(b) => b, + Err(e) => { + log::warn!( + "Failed to initialize CometS3CredentialBridge for {bucket}: {e}; \ + falling back to default credential chain" + ); + None + } + }; + builder = if let Some(bridge) = bridge { builder.with_credentials(Arc::new(bridge)) } else { match get_runtime().block_on(build_credential_provider(configs, bucket, min_ttl))? { diff --git a/native/jni-bridge/src/comet_s3_credential_dispatcher.rs b/native/jni-bridge/src/comet_s3_credential_dispatcher.rs index 8c2e7a57e3..1fb2fef47f 100644 --- a/native/jni-bridge/src/comet_s3_credential_dispatcher.rs +++ b/native/jni-bridge/src/comet_s3_credential_dispatcher.rs @@ -18,7 +18,7 @@ use jni::{ errors::Result as JniResult, objects::{JClass, JFieldID, JStaticMethodID}, - signature::ReturnType, + signature::{Primitive, ReturnType}, strings::JNIString, Env, }; @@ -30,6 +30,8 @@ pub struct CometS3CredentialDispatcher<'a> { /// Retained so the cached POJO `JFieldID`s stay alive for the executor lifetime. #[allow(dead_code)] pub credentials_class: JClass<'a>, + pub method_ensure_initialized: JStaticMethodID, + pub method_ensure_initialized_ret: ReturnType, pub method_get_credentials_for_path: JStaticMethodID, pub method_get_credentials_for_path_ret: ReturnType, pub field_access_key_id: JFieldID, @@ -47,11 +49,17 @@ impl<'a> CometS3CredentialDispatcher<'a> { let credentials_class = env.find_class(JNIString::new(Self::CREDENTIALS_CLASS))?; Ok(CometS3CredentialDispatcher { + method_ensure_initialized: env.get_static_method_id( + JNIString::new(Self::JVM_CLASS), + jni::jni_str!("ensureInitialized"), + jni::jni_sig!("(Ljava/lang/String;Ljava/lang/String;Ljava/util/Map;)V"), + )?, + method_ensure_initialized_ret: ReturnType::Primitive(Primitive::Void), method_get_credentials_for_path: env.get_static_method_id( JNIString::new(Self::JVM_CLASS), jni::jni_str!("getCredentialsForPath"), jni::jni_sig!( - "(Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;I)Lorg/apache/comet/cloud/s3/CometS3Credentials;" + "(Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;I)Lorg/apache/comet/cloud/s3/CometS3Credentials;" ), )?, method_get_credentials_for_path_ret: ReturnType::Object, diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 7a33d46282..5bb9261622 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -205,6 +205,12 @@ message IcebergScanCommon { // Number of data files to read concurrently within a single task uint32 data_file_concurrency_limit = 12; + + // Spark V2 catalog name that loaded this table. Forwarded as the dispatchKey to + // CometS3CredentialDispatcher.ensureInitialized so two catalogs sharing one provider class get + // isolated provider instances. Empty string when the table has no catalog identity (e.g. + // HadoopTables loaded by raw path). + string catalog_name = 13; } message IcebergScan { diff --git a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala index e85eac2c40..8ee2291361 100644 --- a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala +++ b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala @@ -20,6 +20,7 @@ package org.apache.comet.iceberg import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession /** * Shared reflection utilities for Iceberg operations. @@ -672,6 +673,11 @@ object IcebergReflection extends Logging { * Mapping from column names to Iceberg field IDs (built from scanSchema) * @param catalogProperties * Catalog properties for FileIO (S3 credentials, regions, etc.) + * @param catalogName + * Spark V2 catalog name that loaded this table, if it can be derived. Forwarded as + * `dispatchKey` to the native CometS3CredentialBridge so two catalogs sharing one provider FQCN + * get isolated provider instances. `None` when the table has no catalog identity (e.g. + * HadoopTables loaded by raw path). */ case class CometIcebergNativeScanMetadata( table: Any, @@ -682,6 +688,7 @@ case class CometIcebergNativeScanMetadata( tableSchema: Any, globalFieldIdMapping: Map[String, Int], catalogProperties: Map[String, String], + catalogName: Option[String], fileFormat: String) object CometIcebergNativeScanMetadata extends Logging { @@ -737,7 +744,77 @@ object CometIcebergNativeScanMetadata extends Logging { tableSchema = tableSchema, globalFieldIdMapping = globalFieldIdMapping, catalogProperties = catalogProperties, + catalogName = deriveCatalogName(table), fileFormat = FileFormats.PARQUET) } } + + /** + * Best-effort extraction of the Spark V2 catalog name from an Iceberg `Table`. Iceberg's + * `Table.name()` returns `catalog.namespace.table` for tables loaded through a catalog. We + * intersect that name against the V2 catalogs Spark has registered so a value like `s3.foo` is + * not mistaken for a catalog `s3` when no such catalog exists. Falls back to the dotted-prefix + * split when the catalog manager is not reachable or the name does not match. Returns `None` + * when the table has no catalog identity (e.g. HadoopTables loaded by raw path) or when + * reflection fails. + */ + private[iceberg] def deriveCatalogName(table: Any): Option[String] = + deriveCatalogName(table, registeredCatalogNames _) + + /** + * Test seam for [[deriveCatalogName(table:Any)]]. The `knownCatalogNames` thunk lets tests + * inject a fixed catalog set without bootstrapping a SparkSession. + */ + private[iceberg] def deriveCatalogName( + table: Any, + knownCatalogNames: () => Iterable[String]): Option[String] = { + if (table == null) return None + invokeTableName(table).flatMap { name => + if (name.isEmpty || name == "null") { + None + } else { + knownCatalogNames() + .find(c => name == c || name.startsWith(c + ".")) + .orElse { + val idx = name.indexOf('.') + if (idx > 0) Some(name.substring(0, idx)) else None + } + } + } + } + + /** + * Calls Iceberg's public `Table.name()` reflectively. Uses `getMethod` so the interface default + * is reachable when a concrete table class does not override it. Matches the pattern used for + * `Field.name()` / `Column.name()` elsewhere in this file. + * + * `name()` is a default method on `org.apache.iceberg.Table`. A thrown exception here means the + * classpath is wrong or the object is not actually an Iceberg Table, so log at `warn` to make + * it visible. A `null` return is legitimate (anonymous tables) and not noteworthy. + */ + private def invokeTableName(table: Any): Option[String] = { + try { + table.getClass.getMethod("name").invoke(table) match { + case s: String => Some(s) + case other if other != null => Some(other.toString) + case null => None + } + } catch { + case e: Exception => + logWarning( + s"Iceberg reflection: Table.name() not callable on ${table.getClass.getName}; " + + "native S3 credential dispatch will fall back to bucket-keyed isolation: " + + s"${e.getMessage}") + None + } + } + + private def registeredCatalogNames(): Iterable[String] = + try { + SparkSession.active.sessionState.catalogManager.listCatalogs(None) + } catch { + case e: Exception => + logDebug(s"Could not list V2 catalogs from SparkSession: ${e.getMessage}") + Nil + } } diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 64b69be1e9..09f7b0ea45 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -377,12 +377,13 @@ case class CometScanRule(session: SparkSession) val hadoopDerivedProperties = CometIcebergNativeScan.hadoopToIcebergS3Properties(hadoopS3Options) - // Extract vended credentials from FileIO (REST catalog credential vending). - // FileIO properties take precedence over Hadoop-derived properties because - // they contain per-table credentials vended by the REST catalog. + // Forward the full FileIO property bag (including credentials.uri, OAuth tokens, + // tenant-id, etc.) so a CometS3CredentialProvider can see everything LoadTableResponse + // returned. The storage-prefix narrowing happens native-side just before + // FileIOBuilder.with_prop, since iceberg-rust's FileIO is the only consumer that + // requires the narrowed view. val fileIOProperties = tableOpt .flatMap(IcebergReflection.getFileIOProperties) - .map(CometIcebergNativeScan.filterStorageProperties) .getOrElse(Map.empty) val catalogProperties = hadoopDerivedProperties ++ fileIOProperties diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala index b0de978af8..8c88c19f1f 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala @@ -500,21 +500,6 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit } } - /** Storage-related property prefixes passed through to native FileIO. */ - private val storagePropertyPrefixes = - Seq("s3.", "gcs.", "adls.", "client.") - - /** - * Filters a properties map to only include storage-related keys. FileIO.properties() may - * contain catalog URIs, bearer tokens, and other non-storage settings that should not be passed - * to the native FileIO builder. - */ - def filterStorageProperties(props: Map[String, String]): Map[String, String] = { - props.filter { case (key, _) => - storagePropertyPrefixes.exists(prefix => key.startsWith(prefix)) - } - } - /** * Transforms Hadoop S3A configuration keys to Iceberg FileIO property keys. * @@ -771,6 +756,7 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit commonBuilder.setMetadataLocation(metadata.metadataLocation) commonBuilder.setDataFileConcurrencyLimit( CometConf.COMET_ICEBERG_DATA_FILE_CONCURRENCY_LIMIT.get()) + metadata.catalogName.foreach(commonBuilder.setCatalogName) metadata.catalogProperties.foreach { case (key, value) => commonBuilder.putCatalogProperties(key, value) } diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java b/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java new file mode 100644 index 0000000000..24f11fecbe --- /dev/null +++ b/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java @@ -0,0 +1,70 @@ +/* + * 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.comet.cloud.s3; + +import java.util.Map; + +import org.apache.iceberg.aws.s3.VendedCredentialsProvider; + +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; + +/** + * Reference implementation of {@link CometS3CredentialProvider} for Iceberg REST catalogs that vend + * AWS credentials. Wraps Iceberg's existing {@code VendedCredentialsProvider}, so caching and + * refresh-near-expiry come from {@code software.amazon.awssdk.utils.cache.CachedSupplier} for free. + * Comet contributes only the JNI shape and the one-shot {@code initialize} call. + * + *

Lives in {@code spark/src/test} for now to keep iceberg-aws and AWS SDK v2 off Comet's runtime + * classpath. A future PR can promote this to a runtime artifact behind an optional {@code + * iceberg-aws} dependency once we settle on a packaging story. + * + *

Activation: set {@code spark.sql.catalog..s3.comet.credential.provider.class = + * org.apache.comet.cloud.s3.IcebergRESTVendedS3Provider}. Comet calls {@link #initialize} once per + * catalog with the unfiltered FileIO property bag, which already carries {@code credentials.uri} + * and {@code uri} as required by {@code VendedCredentialsProvider.create}. + */ +public final class IcebergRESTVendedS3Provider implements CometS3CredentialProvider { + + private volatile VendedCredentialsProvider provider; + + @Override + public void initialize(Map catalogProperties) { + this.provider = VendedCredentialsProvider.create(catalogProperties); + } + + @Override + public CometS3Credentials getCredentialsForPath( + String bucket, String path, CometS3AccessMode mode) { + VendedCredentialsProvider p = provider; + if (p == null) { + throw new IllegalStateException( + "IcebergRESTVendedS3Provider used before initialize(Map) was called; " + + "Comet should always invoke initialize before getCredentialsForPath"); + } + AwsCredentials c = p.resolveCredentials(); + String sessionToken = + (c instanceof AwsSessionCredentials) ? ((AwsSessionCredentials) c).sessionToken() : null; + // Expiration is owned by VendedCredentialsProvider's CachedSupplier; we publish 0 so the + // native bridge applies its conservative floor to opendal's cache while the inner + // CachedSupplier handles refresh on its own schedule. + return new CometS3Credentials(c.accessKeyId(), c.secretAccessKey(), sessionToken, 0L); + } +} diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java b/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java new file mode 100644 index 0000000000..28b851fb18 --- /dev/null +++ b/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java @@ -0,0 +1,90 @@ +/* + * 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.comet.cloud.s3; + +import java.util.HashMap; +import java.util.Map; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +/** + * Unit tests for {@link IcebergRESTVendedS3Provider}. Feeds the wrapped {@code + * VendedCredentialsProvider} a fully-populated {@code s3.*} bag so its {@code CachedSupplier} + * short-circuits without calling the REST {@code credentials.uri} endpoint. Refresh-via-REST + * coverage lives upstream in Iceberg. + */ +public class IcebergRESTVendedS3ProviderTest { + + private static Map staticVendedProps() { + Map props = new HashMap<>(); + // Required by VendedCredentialsProvider.create; not actually contacted in this test. + props.put("uri", "http://localhost:0/catalog"); + props.put("credentials.uri", "http://localhost:0/credentials"); + // Required four keys for the in-properties short-circuit path. + props.put("s3.access-key-id", "AKIA_TEST"); + props.put("s3.secret-access-key", "secret_TEST"); + props.put("s3.session-token", "token_TEST"); + props.put( + "s3.session-token-expires-at-ms", Long.toString(System.currentTimeMillis() + 3_600_000L)); + return props; + } + + @Test + public void initializeThenGetReturnsVendedCredentials() { + IcebergRESTVendedS3Provider p = new IcebergRESTVendedS3Provider(); + p.initialize(staticVendedProps()); + + CometS3Credentials c = p.getCredentialsForPath("bucket", "/k", CometS3AccessMode.READ); + + assertEquals("AKIA_TEST", c.getAccessKeyId()); + assertEquals("secret_TEST", c.getSecretAccessKey()); + assertEquals("token_TEST", c.getSessionToken()); + // Wrapper publishes 0 since VendedCredentialsProvider's CachedSupplier owns the expiry. + assertEquals(0L, c.getExpirationEpochMillis()); + } + + @Test + public void getBeforeInitializeThrows() { + IcebergRESTVendedS3Provider p = new IcebergRESTVendedS3Provider(); + assertThrows( + IllegalStateException.class, + () -> p.getCredentialsForPath("bucket", "/k", CometS3AccessMode.READ)); + } + + @Test + public void multipleCallsServedByCache() { + IcebergRESTVendedS3Provider p = new IcebergRESTVendedS3Provider(); + p.initialize(staticVendedProps()); + + CometS3Credentials a = p.getCredentialsForPath("b", "/k", CometS3AccessMode.READ); + CometS3Credentials b = p.getCredentialsForPath("b", "/k2", CometS3AccessMode.READ); + CometS3Credentials c = p.getCredentialsForPath("b", "/k3", CometS3AccessMode.READ); + + // CachedSupplier hands out the same identity-equal credential until expiry; we assert by + // value so the test does not depend on AWS SDK internal caching semantics. + assertEquals(a.getAccessKeyId(), b.getAccessKeyId()); + assertEquals(a.getAccessKeyId(), c.getAccessKeyId()); + assertEquals(a.getSecretAccessKey(), b.getSecretAccessKey()); + assertEquals(a.getSessionToken(), b.getSessionToken()); + } +} diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java index f255d9cb27..ed83cc914e 100644 --- a/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java +++ b/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java @@ -19,6 +19,9 @@ package org.apache.comet.cloud.s3; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -29,10 +32,28 @@ */ public final class MinioCometS3CredentialProvider implements CometS3CredentialProvider { + /** + * Property key used by tests to tag a per-catalog provider instance. The dispatcher key alone is + * not visible to {@code initialize(Map)}, so suites that want to identify which catalog's + * instance they are inspecting put a discriminator under this key on the catalog config. + */ + public static final String TEST_INSTANCE_TAG = "comet.test-instance-tag"; + private static final AtomicReference CREDS = new AtomicReference<>(); private static final AtomicInteger CALL_COUNT = new AtomicInteger(0); + private static final AtomicInteger INIT_COUNT = new AtomicInteger(0); private static final AtomicReference LAST_BUCKET = new AtomicReference<>(); private static final AtomicReference LAST_PATH = new AtomicReference<>(); + private static final AtomicReference> LAST_INIT_PROPS = + new AtomicReference<>(); + + /** + * Captures one entry per `initialize(Map)` invocation, keyed by the value of {@link + * #TEST_INSTANCE_TAG} in the supplied map. Lets multi-catalog tests look up the per-instance init + * bag without relying on the most-recent-wins {@link #LAST_INIT_PROPS}. + */ + private static final ConcurrentHashMap> INIT_BY_TAG = + new ConcurrentHashMap<>(); public static void installCredentials(String accessKeyId, String secretAccessKey) { CREDS.set(new Credentials(accessKeyId, secretAccessKey)); @@ -42,6 +63,10 @@ public static int callCount() { return CALL_COUNT.get(); } + public static int initCount() { + return INIT_COUNT.get(); + } + public static String lastBucket() { return LAST_BUCKET.get(); } @@ -50,10 +75,33 @@ public static String lastPath() { return LAST_PATH.get(); } + public static Map lastInitProperties() { + return LAST_INIT_PROPS.get(); + } + + public static Map initPropertiesForTag(String tag) { + return INIT_BY_TAG.get(tag); + } + public static void resetCounters() { CALL_COUNT.set(0); + INIT_COUNT.set(0); LAST_BUCKET.set(null); LAST_PATH.set(null); + LAST_INIT_PROPS.set(null); + INIT_BY_TAG.clear(); + } + + @Override + public void initialize(Map catalogProperties) { + INIT_COUNT.incrementAndGet(); + Map snapshot = + Collections.unmodifiableMap(new java.util.HashMap<>(catalogProperties)); + LAST_INIT_PROPS.set(snapshot); + String tag = catalogProperties.get(TEST_INSTANCE_TAG); + if (tag != null) { + INIT_BY_TAG.put(tag, snapshot); + } } @Override diff --git a/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala b/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala index 9bd397aeb7..172d339901 100644 --- a/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/cloud/s3/CometS3CredentialBridgeSuite.scala @@ -27,13 +27,17 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.{col, sum} import org.apache.comet.{CometConf, CometS3TestBase} +import org.apache.comet.iceberg.RESTCatalogHelper /** * End-to-end test that exercises [[CometS3CredentialDispatcher]] and the Rust JNI bridge against * a real S3 server (Minio). Asserts the test SPI was actually invoked rather than the default AWS * credential chain. */ -class CometS3CredentialBridgeSuite extends CometS3TestBase with AdaptiveSparkPlanHelper { +class CometS3CredentialBridgeSuite + extends CometS3TestBase + with AdaptiveSparkPlanHelper + with RESTCatalogHelper { override protected val testBucketName = "bridge-test-bucket" @@ -115,4 +119,119 @@ class CometS3CredentialBridgeSuite extends CometS3TestBase with AdaptiveSparkPla spark.sql("DROP TABLE s3_catalog.db.bridge_iceberg") } + + test("REST catalog vended creds reach CometS3CredentialProvider.initialize via JNI") { + assume(icebergAvailable, "Iceberg not available in classpath") + + // Mix storage-prefixed creds (which the old filter would have allowed through) with a + // non-storage-prefixed sentinel key. The sentinel only reaches initialize(Map) if Comet + // forwards the unfiltered FileIO property bag end-to-end. + val vendedCreds = Map( + "s3.access-key-id" -> userName, + "s3.secret-access-key" -> password, + "s3.endpoint" -> minioContainer.getS3URL, + "s3.region" -> "us-east-1", + "s3.path-style-access" -> "true", + "comet.test-tenant-id" -> "tenant-A") + val warehouse = s"s3a://$testBucketName/warehouse-rest-spi" + val providerClassName = classOf[MinioCometS3CredentialProvider].getName + + withRESTCatalog(vendedCredentials = vendedCreds, warehouseLocation = Some(warehouse)) { + (restUri, _, _) => + withSQLConf( + "spark.sql.catalog.rest_spi" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.rest_spi.catalog-impl" -> "org.apache.iceberg.rest.RESTCatalog", + "spark.sql.catalog.rest_spi.uri" -> restUri, + "spark.sql.catalog.rest_spi.warehouse" -> warehouse, + "spark.sql.catalog.rest_spi.s3.comet.credential.provider.class" -> providerClassName) { + + spark.sql("CREATE NAMESPACE rest_spi.db") + spark.sql("CREATE TABLE rest_spi.db.tbl (id INT) USING iceberg") + spark.sql("INSERT INTO rest_spi.db.tbl VALUES (1), (2), (3)") + + MinioCometS3CredentialProvider.resetCounters() + val df = spark.sql("SELECT * FROM rest_spi.db.tbl ORDER BY id") + assertHasCometIcebergScan(df.queryExecution.executedPlan) + assert(df.count() == 3) + + assert( + MinioCometS3CredentialProvider.callCount() > 0, + "Bridge was not invoked during REST catalog read") + + val captured = MinioCometS3CredentialProvider.lastInitProperties() + assert(captured != null, "initialize(Map) was never called") + assert( + captured.containsKey("comet.test-tenant-id"), + s"Unfiltered FileIO key did not reach initialize(Map). Captured keys: ${captured + .keySet()}") + assert(captured.get("comet.test-tenant-id") == "tenant-A") + + spark.sql("DROP TABLE rest_spi.db.tbl") + spark.sql("DROP NAMESPACE rest_spi.db") + } + } + } + + test( + "two catalogs sharing one provider FQCN get isolated CometS3CredentialProvider instances") { + assume(icebergAvailable, "Iceberg not available in classpath") + + val providerClassName = classOf[MinioCometS3CredentialProvider].getName + val s3Endpoint = minioContainer.getS3URL + + withSQLConf( + "spark.sql.catalog.iso_a" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.iso_a.type" -> "hadoop", + "spark.sql.catalog.iso_a.warehouse" -> s"s3a://$testBucketName/iso-warehouse-a", + "spark.sql.catalog.iso_a.s3.comet.credential.provider.class" -> providerClassName, + "spark.sql.catalog.iso_a.s3.endpoint" -> s3Endpoint, + "spark.sql.catalog.iso_a.s3.region" -> "us-east-1", + "spark.sql.catalog.iso_a.s3.path-style-access" -> "true", + s"spark.sql.catalog.iso_a.${MinioCometS3CredentialProvider.TEST_INSTANCE_TAG}" -> "tag-A", + "spark.sql.catalog.iso_b" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.iso_b.type" -> "hadoop", + "spark.sql.catalog.iso_b.warehouse" -> s"s3a://$testBucketName/iso-warehouse-b", + "spark.sql.catalog.iso_b.s3.comet.credential.provider.class" -> providerClassName, + "spark.sql.catalog.iso_b.s3.endpoint" -> s3Endpoint, + "spark.sql.catalog.iso_b.s3.region" -> "us-east-1", + "spark.sql.catalog.iso_b.s3.path-style-access" -> "true", + s"spark.sql.catalog.iso_b.${MinioCometS3CredentialProvider.TEST_INSTANCE_TAG}" -> "tag-B") { + + MinioCometS3CredentialProvider.resetCounters() + + spark.sql("CREATE TABLE iso_a.db.t (id INT) USING iceberg") + spark.sql("INSERT INTO iso_a.db.t VALUES (1), (2)") + spark.sql("CREATE TABLE iso_b.db.t (id INT) USING iceberg") + spark.sql("INSERT INTO iso_b.db.t VALUES (10), (20), (30)") + + val dfA = spark.sql("SELECT * FROM iso_a.db.t ORDER BY id") + assertHasCometIcebergScan(dfA.queryExecution.executedPlan) + assert(dfA.count() == 2) + + val dfB = spark.sql("SELECT * FROM iso_b.db.t ORDER BY id") + assertHasCometIcebergScan(dfB.queryExecution.executedPlan) + assert(dfB.count() == 3) + + // Two distinct dispatchKeys (catalog names "iso_a" and "iso_b") must yield two separate + // initialize(Map) calls even though the provider FQCN is shared. + assert( + MinioCometS3CredentialProvider.initCount() >= 2, + "Expected at least 2 initialize() calls across two catalogs, got " + + s"${MinioCometS3CredentialProvider.initCount()}") + + val initA = MinioCometS3CredentialProvider.initPropertiesForTag("tag-A") + val initB = MinioCometS3CredentialProvider.initPropertiesForTag("tag-B") + assert(initA != null, "initialize(Map) for catalog iso_a never landed in INIT_BY_TAG") + assert(initB != null, "initialize(Map) for catalog iso_b never landed in INIT_BY_TAG") + assert( + initA.get(MinioCometS3CredentialProvider.TEST_INSTANCE_TAG) == "tag-A", + s"iso_a init map tagged wrong: ${initA.get(MinioCometS3CredentialProvider.TEST_INSTANCE_TAG)}") + assert( + initB.get(MinioCometS3CredentialProvider.TEST_INSTANCE_TAG) == "tag-B", + s"iso_b init map tagged wrong: ${initB.get(MinioCometS3CredentialProvider.TEST_INSTANCE_TAG)}") + + spark.sql("DROP TABLE iso_a.db.t") + spark.sql("DROP TABLE iso_b.db.t") + } + } } From 8551acad3501fa321509fa735e35aa974a13f0cc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 18 May 2026 16:02:01 -0400 Subject: [PATCH 17/24] change IcebergRESTVendedS3Provider and its test to Spark 4.0+ --- .../cloud/s3/IcebergRESTVendedS3Provider.java | 26 ++++++++++++------- .../s3/IcebergRESTVendedS3ProviderTest.java | 0 2 files changed, 17 insertions(+), 9 deletions(-) rename spark/src/test/{ => spark-4.x}/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java (66%) rename spark/src/test/{ => spark-4.x}/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java (100%) diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java similarity index 66% rename from spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java rename to spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java index 24f11fecbe..614019c68a 100644 --- a/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java +++ b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java @@ -27,19 +27,27 @@ import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; /** - * Reference implementation of {@link CometS3CredentialProvider} for Iceberg REST catalogs that vend - * AWS credentials. Wraps Iceberg's existing {@code VendedCredentialsProvider}, so caching and - * refresh-near-expiry come from {@code software.amazon.awssdk.utils.cache.CachedSupplier} for free. - * Comet contributes only the JNI shape and the one-shot {@code initialize} call. + * Example implementation of {@link CometS3CredentialProvider} for Iceberg REST catalogs that vend + * AWS credentials. Wraps Iceberg's {@code VendedCredentialsProvider} so caching and + * refresh-near-expiry come from its {@code CachedSupplier}; Comet adds only the JNI shape and the + * one-shot {@code initialize} call. * - *

Lives in {@code spark/src/test} for now to keep iceberg-aws and AWS SDK v2 off Comet's runtime - * classpath. A future PR can promote this to a runtime artifact behind an optional {@code - * iceberg-aws} dependency once we settle on a packaging story. + *

Test scope only, to keep iceberg-aws and AWS SDK v2 off Comet's runtime classpath. Production + * users should copy this into their own jar. + * + *

Built only on the Spark 4.x profiles (placed under {@code src/test/spark-4.x}, picked up via + * {@code shims.majorVerSrc}). Excluded from Spark 3.4 because {@code iceberg-spark-runtime-3.4_*} + * does not expose {@code VendedCredentialsProvider} on its test classpath. Excluded from Spark 3.5 + * because Comet pins Iceberg 1.8.1 there, and the in-properties short-circuit that the unit test + * relies on was added in Iceberg 1.9.0 (apache/iceberg#12504); on 1.8.x {@code refreshCredential} + * always issues an HTTP GET against {@code credentials.uri}. + * + *

Test exercised in CI against Iceberg 1.10.0 (the Spark 4.x profile pin). * *

Activation: set {@code spark.sql.catalog..s3.comet.credential.provider.class = * org.apache.comet.cloud.s3.IcebergRESTVendedS3Provider}. Comet calls {@link #initialize} once per - * catalog with the unfiltered FileIO property bag, which already carries {@code credentials.uri} - * and {@code uri} as required by {@code VendedCredentialsProvider.create}. + * catalog with the unfiltered FileIO property bag, which carries {@code credentials.uri} and + * {@code uri} as required by {@code VendedCredentialsProvider.create}. */ public final class IcebergRESTVendedS3Provider implements CometS3CredentialProvider { diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java similarity index 100% rename from spark/src/test/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java rename to spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java From 9f58cb09004fac989800f6dbee7156211cce02f9 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 18 May 2026 19:34:55 -0400 Subject: [PATCH 18/24] Update file structure after #4325. --- .../main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java | 0 .../org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java | 0 .../java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java | 0 .../main/java/org/apache/comet/cloud/s3/CometS3Credentials.java | 0 .../apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java | 0 .../test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java | 0 .../java/org/apache/comet/cloud/s3/NotACredentialProvider.java | 0 .../org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java | 0 8 files changed, 0 insertions(+), 0 deletions(-) rename {common => spark}/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java (100%) rename {common => spark}/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java (100%) rename {common => spark}/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java (100%) rename {common => spark}/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java (100%) rename {common => spark}/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java (100%) rename {common => spark}/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java (100%) rename {common => spark}/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java (100%) rename {common => spark}/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java (100%) diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java similarity index 100% rename from common/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java rename to spark/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java similarity index 100% rename from common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java rename to spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java similarity index 100% rename from common/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java rename to spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java diff --git a/common/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java similarity index 100% rename from common/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java rename to spark/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java diff --git a/common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java b/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java similarity index 100% rename from common/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java rename to spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java diff --git a/common/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java similarity index 100% rename from common/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java rename to spark/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java diff --git a/common/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java similarity index 100% rename from common/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java rename to spark/src/test/java/org/apache/comet/cloud/s3/NotACredentialProvider.java diff --git a/common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java similarity index 100% rename from common/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java rename to spark/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java From 5853dffedb7992157f3c1fbf4c5932cd9bb64fa3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 21 May 2026 14:10:22 -0400 Subject: [PATCH 19/24] Update docs, add contributor guide page about credential provider. --- docs/source/contributor-guide/index.md | 1 + .../s3-credential-provider-design.md | 99 +++++++++++++++++++ .../latest/cloud-credential-providers.md | 28 +++--- .../cloud/s3/CometS3CredentialProvider.java | 47 ++------- .../cloud/s3/IcebergRESTVendedS3Provider.java | 13 +-- 5 files changed, 128 insertions(+), 60 deletions(-) create mode 100644 docs/source/contributor-guide/s3-credential-provider-design.md diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index f3bbfba044..0b8269e7b9 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -47,6 +47,7 @@ Benchmarking Guide Adding a New Operator Adding a New Expression Adding a New Spark Version +S3 Credential Provider Design Supported Spark Expressions Supported Spark Configurations Tracing diff --git a/docs/source/contributor-guide/s3-credential-provider-design.md b/docs/source/contributor-guide/s3-credential-provider-design.md new file mode 100644 index 0000000000..4860b2fe7e --- /dev/null +++ b/docs/source/contributor-guide/s3-credential-provider-design.md @@ -0,0 +1,99 @@ + + +# S3 Credential Provider SPI: Design Notes + +This page captures why the `org.apache.comet.cloud.s3.CometS3CredentialProvider` SPI is shaped the way it is. The user-facing contract and operator setup live in the user guide page on cloud credential providers; this page is for maintainers and reviewers who want the design rationale. + +## The gap the SPI fills + +Comet's native scan paths (`object_store` for raw Parquet, `opendal` via `iceberg-rust` for Iceberg) bypass Spark's Hadoop S3A code path. That means credentials cannot flow through any of the contracts that vendors typically wire into for S3A: + +- `org.apache.spark.deploy.security.cloud.CloudCredentialsProvider` yields a single JWT per service name. No path argument, no AWS credential. +- Hadoop S3A custom signers hide path-aware logic inside `Signer.sign(request, credentials)`. The credential never leaves the signing pipeline, and the underlying secret is an HMAC key that is not present in the signed output, so running the signer against a synthesized request cannot recover it. +- `AWSCredentialsProvider.getCredentials()` (AWS SDK v1) and `AwsCredentialsProvider.resolveCredentials()` (v2) are parameterless. They cannot vend per-path credentials. +- Reflecting into vendor singletons would encode per-vendor class names and lifecycles in Comet and would silently break on vendor upgrades. + +A Comet-specific SPI is the narrowest fit: a single Java method that takes `(bucket, path, mode)` and returns `CometS3Credentials`. + +## Why config-driven activation, not `META-INF/services` + +An earlier iteration used `ServiceLoader` discovery. That was rejected because: + +- Peer SPIs in the same space (Hadoop `AWSCredentialsProvider`, AWS SDK v2 `AwsCredentialsProvider`, Iceberg `AwsClientFactory`, S3A custom signers) are all class-name-in-config. Vendors are already familiar with that model. +- ServiceLoader makes activation implicit on classpath presence. A vendor JAR drifting onto a cluster could silently change S3 auth behavior. The config key makes activation explicit. +- The activation key (`fs.s3a.comet.credential.provider.class`, with per-bucket override) follows the same shape as `fs.s3a.bucket..aws.credentials.provider`, so operators do not learn a new pattern. + +Activation is modeled on `parquet.crypto.factory.class` (Parquet Modular Encryption KMS, see Comet #2447): the user names a single vendor class and the vendor dispatches across multiple credential backends inside that class if they need to. This mirrors how Iceberg's `DecryptionPropertiesFactory` already behaves for Parquet keys. + +## Why `(FQCN, dispatchKey)` keying + +Comet caches one provider instance per `(FQCN, dispatchKey)`. The dispatch key is the Spark V2 catalog name on the Iceberg path and the bucket on the Parquet path. + +- Two catalogs that share one provider class (typical in multi-tenant deployments) need isolated `initialize` maps because their `catalogProperties` differ. Without `dispatchKey`, the second `initialize` would either overwrite the first or be silently skipped. +- The bucket as `dispatchKey` for Parquet gives vendors per-bucket isolation when the same provider is named under several `fs.s3a.bucket..comet.credential.provider.class` keys. +- Keying solely by FQCN would force vendors to encode multi-tenant routing in static state. The pair-key keeps each call site independent. + +## Why fresh construction in `initialize`, not probing a JVM-wide static + +A provider implementation might be tempted to probe an existing static populated elsewhere (e.g. by a Hadoop S3A signer's `registerStore` callback) and reuse the credential cache that the Hadoop path uses. That fails on Comet-only executors: + +- The driver JVM hits `S3AFileSystem.initialize` during analysis (raw `s3a://` paths) or during Hadoop catalog manifest reads (Iceberg with Hadoop catalog), so the static is populated there. +- The driver may not hit `S3AFileSystem` at all under Iceberg with REST catalog plus `S3FileIO`, because `S3FileIO` calls AWS SDK directly without going through the Hadoop layer. The static stays null. +- Executors with Comet-only reads never instantiate `S3AFileSystem`. The data path is `object_store` (raw Parquet) or `opendal` via `iceberg-rust` (Iceberg native scan). Neither touches Hadoop S3A. The static stays null on every executor. + +Constructing a fresh provider from `catalogProperties` plus `SparkEnv` is the only strategy that works across all four cases. The trade-off is that on the driver (and any JVM where Hadoop S3A is also active), two credential caches now exist for the same identity: one inside the Hadoop signer's provider, one inside the SPI implementation's. The vendor pays for this with a small number of extra AS round-trips on cold starts and TTL boundaries. A future optional optimization could probe the static first and reuse if non-null, falling back to fresh construction otherwise. + +## Why no Comet-side cache + +Comet's bridge does not maintain a TTL cache, schedule refresh, or broadcast catalog state. All of that is the vendor's responsibility: + +- Iceberg vendors get `software.amazon.awssdk.utils.cache.CachedSupplier` for free inside `org.apache.iceberg.aws.s3.VendedCredentialsProvider`. +- Custom-STS vendors write whatever cache fits their refresh model. +- Driver-only state (e.g. a JWT minted at plan time) is distributed via `initialize`'s `catalogProperties`, which Comet has already serialized through the native plan op for the Iceberg path. On the Parquet path vendors read from Hadoop conf via `SparkEnv`. + +A Comet-side cache would have to either expose a tuning knob (TTL, max size, eviction policy) and grow over time, or be hardcoded and surprise vendors whose policies disagree. The bridge intentionally has neither and forwards every call. + +## Path-specific behavior + +`object_store::CredentialProvider` and `reqsign_core::ProvideCredential` differ in what they consume: + +| Concern | Parquet (`object_store`) | Iceberg (`opendal` via `reqsign-core`) | +| ----------------------- | -------------------------------------------------- | ------------------------------------------------------------ | +| Trait method | `get_credential() -> AwsCredential` | `provide_credential(...) -> Option` | +| Returns expiry? | No (only key/secret/token) | Yes (`expires_in: Option`) | +| Comet-side TTL wrapper? | None. Bridge passed straight to `with_credentials` | None. `opendal` schedules the next refresh from `expires_in` | +| When SPI is called | Every `get_credential()` call | When `expires_in` is exceeded | +| Vendor returns 0 expiry | Field has no use | Bridge substitutes 5 minutes to bound staleness | + +The 5-minute fallback is a safety net so a vendor that omits expiry cannot leave `opendal` caching a stale token indefinitely. It is intentionally not a configuration knob. + +## Property-bag handling on the Iceberg path + +The full unfiltered FileIO property bag crosses JNI as `catalog_properties`. The storage-prefix filter (`s3.`/`gcs.`/`adls.`/`client.`) is applied native-side in `iceberg_scan.rs::load_file_io` immediately before `FileIOBuilder.with_prop`. This means the bridge sees `credentials.uri`, OAuth tokens, and any vendor-custom keys with no parallel field on the operator and no driver-side broadcast. Vendors set their own keys on the catalog config and read them back inside `initialize(Map)`. + +`IcebergScanExec` derives a redacting `Debug` so plan dumps and tracing do not leak the property bag. + +## Returns or throws, not a fall-through value + +The SPI returns a `CometS3Credentials` or throws. There is no sentinel "I do not know" return. Vendors that are only authoritative for some paths resolve the default AWS chain themselves for the rest and return the result. This matches the contract on every other AWS credential SPI in the JVM ecosystem (AWS SDK v1/v2, Hadoop S3A, Iceberg `VendedCredentialsProvider`). + +## Iceberg path: error message fidelity caveat + +When the bridge is wired into `iceberg-rust`, the outer `reqsign-core::ProvideCredentialChain` currently swallows thrown exceptions into "no credential" before the request reaches `opendal`. The credential is still not issued and the request still fails, but the message is degraded to an opaque anonymous-request failure. No Comet change fixes this; it is resolved upstream when `iceberg-rust` stops wrapping custom loaders in its outer chain or moves its S3 backend to `object_store`. diff --git a/docs/source/user-guide/latest/cloud-credential-providers.md b/docs/source/user-guide/latest/cloud-credential-providers.md index 0bf404d8fd..ccbef886e4 100644 --- a/docs/source/user-guide/latest/cloud-credential-providers.md +++ b/docs/source/user-guide/latest/cloud-credential-providers.md @@ -38,7 +38,7 @@ You probably do, if any of these are true: ## Enabling a bridge -A bridge is activated by naming the vendor's class in a Spark config. There is no `META-INF/services` discovery and putting a JAR on the classpath alone has no effect; the config key must be set. +A bridge is activated by naming the vendor's class in a Spark config. Putting a JAR on the classpath alone has no effect; the config key must be set. For raw Parquet (the `object_store` path), set the Hadoop S3A config: @@ -87,11 +87,11 @@ Without the config set, no credential-related log lines appear at startup; nativ **`403 AccessDenied` with the bridge configured.** The provider returned credentials but they were rejected by S3. Most often a region mismatch (see Iceberg section below) or expired session token; enable debug logging on the vendor's class to confirm what it returned. -**Credentials silently going stale during long-running jobs.** The bridge caps opendal's credential cache at 5 minutes when the vendor does not populate `expirationEpochMillis`. Ask the vendor to return a real expiry; the 5-minute floor is a safety net, not a knob. +**Credentials silently going stale during long-running jobs.** When a vendor returns `expirationEpochMillis=0`, the bridge substitutes a 5-minute expiry before handing the credential to `opendal`, so `opendal`'s cache cannot hold a stale credential indefinitely. Returning a real expiry is preferred; the 5-minute fallback is a safety net, not a knob. ## Iceberg: explicit S3 region required -With the bridge configured, Comet wires a custom credential loader into `iceberg-storage-opendal`. opendal's built-in S3 region auto-detection only runs when no custom loader is configured, so on the bridge path the region (and endpoint for non-AWS) must be set explicitly on the Spark catalog: +With the bridge configured, Comet wires a custom credential loader into `iceberg-storage-opendal`. `opendal`'s built-in S3 region auto-detection only runs when no custom loader is configured, so on the bridge path the region (and endpoint for non-AWS) must be set explicitly on the Spark catalog: ``` spark.sql.catalog..s3.region = us-east-1 @@ -135,18 +135,20 @@ Comet does not maintain a TTL cache, broadcast catalog state, or schedule refres - When to refresh: proactive timer, on-demand at expiry, on `403` retry, etc. - How to distribute driver-only state. Read it from `initialize`'s `catalogProperties` (which Comet has already serialized through the native plan op), call back to a vendor service from the executor, or run your own Spark broadcast inside the class. -`expirationEpochMillis` on the returned `CometS3Credentials` is metadata pass-through, not a Comet-owned cache. Comet forwards it to `object_store` and opendal, which already have their own credential caches and use the expiry to schedule the next refresh. Publish a real expiry when you have one; return `0` if you do not, and a conservative 5-minute floor is applied so a stale credential cannot live indefinitely. +`expirationEpochMillis` only matters on the Iceberg/`opendal` path. There the bridge implements `reqsign_core::ProvideCredential`, which carries an `expires_in` field that `opendal` uses to schedule the next refresh. Publish a real expiry when you have one. `0` means "unknown"; the bridge then substitutes a 5-minute expiry to bound staleness. + +The Parquet/`object_store` path has no expiry concept: `object_store::CredentialProvider` returns just `AwsCredential` (key/secret/token). The bridge is passed to `with_credentials` without a TTL wrapper, so `object_store` calls into the SPI on every request and relies on the vendor's own cache for hit rates. Expiry handling is fully the vendor's responsibility: the vendor decides when its internal cache refreshes. If `object_store` receives a 403 from an expired session token, its retry layer calls `get_credential()` again, giving the vendor another chance to mint fresh credentials. ### Returned fields -| Field | Notes | -| ----------------------- | ------------------------------------------------- | -| `accessKeyId` | Required. | -| `secretAccessKey` | Required. | -| `sessionToken` | `null` for non-STS credentials. | -| `expirationEpochMillis` | Absolute expiry. `0` means "unknown" (see below). | +| Field | Notes | +| ----------------------- | ------------------------------------------------------------------------------------------------------------------------- | +| `accessKeyId` | Required. | +| `secretAccessKey` | Required. | +| `sessionToken` | `null` for non-STS credentials. | +| `expirationEpochMillis` | Iceberg path only. `0` means "unknown"; the bridge substitutes a 5-minute expiry. The Parquet path has no expiry concept. | -Provide a real `expirationEpochMillis` whenever you have one. The Iceberg path uses it to decide when `opendal` must re-call the provider for a fresh credential. `0` is treated as unknown and the Iceberg path defaults to a 5-minute refresh to bound staleness. +Provide a real `expirationEpochMillis` whenever you have one on the Iceberg path. The Parquet path's `object_store::CredentialProvider` does not consume an expiry, and the bridge invokes the SPI on every `get_credential()` call. ### Returns or throws @@ -236,7 +238,7 @@ public final class IcebergRESTVendedS3Provider implements CometS3CredentialProvi } ``` -`VendedCredentialsProvider` reads `credentials.uri`, the catalog endpoint, and OAuth tokens from the supplied map (Comet forwards the unfiltered FileIO bag to `initialize`), and refreshes through its own `CachedSupplier`. Caching, refresh-near-expiry, and the REST round-trip all live in Iceberg, not in Comet. Comet ships a copy of this class under `spark/src/test` for now; promote it to your runtime classpath alongside `iceberg-aws` and AWS SDK v2. +`VendedCredentialsProvider` reads `credentials.uri`, the catalog endpoint, and OAuth tokens from the supplied map (Comet forwards the unfiltered FileIO bag to `initialize`), and refreshes through its own `CachedSupplier`. Caching, refresh-near-expiry, and the REST round-trip all live in Iceberg, not in Comet. Comet ships a copy of this class under `spark/src/test` as a reference; copy it into your runtime jar alongside `iceberg-aws` and AWS SDK v2. ### Access mode @@ -262,4 +264,4 @@ Vendor implementations need the Comet SPI classes at compile time only. Use `pro ### Iceberg path: error message fidelity -When the bridge is wired into `iceberg-rust`, the outer `reqsign-core::ProvideCredentialChain` currently swallows thrown exceptions into "no credential" before the request reaches opendal. The credential is still not issued and the request still fails, only the message is degraded to an opaque anonymous-request failure. No Comet change fixes this; it is resolved upstream if `iceberg-rust` stops wrapping custom loaders in its outer chain or moves its S3 backend to `object_store`. +When the bridge is wired into `iceberg-rust`, the outer `reqsign-core::ProvideCredentialChain` currently swallows thrown exceptions into "no credential" before the request reaches `opendal`. The credential is still not issued and the request still fails, only the message is degraded to an opaque anonymous-request failure. No Comet change fixes this; it is resolved upstream if `iceberg-rust` stops wrapping custom loaders in its outer chain or moves its S3 backend to `object_store`. diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java index 536df65bbb..9730ed6851 100644 --- a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java @@ -23,29 +23,9 @@ /** * SPI for supplying AWS credentials to Comet's native S3 readers, which bypass Spark's Hadoop S3A - * code path and cannot reach signer-based or path-aware credential mechanisms through the standard - * parameterless {@code AWSCredentialsProvider.getCredentials()} contract. - * - *

Peer to {@code org.apache.hadoop.fs.s3a.AwsSignerInitializer} (Hadoop S3A) and {@code - * org.apache.iceberg.aws.AwsClientFactory} (Iceberg-Java): the same shape vendors already implement - * for those two, with a smaller surface (one method). - * - *

Why a new SPI?

- * - * No existing contract carries per-path AWS credentials from vendor code to Comet's native readers: - * - * + * code path. Vendors implement this when path-aware or vendor-managed credential mechanisms cannot + * be reached through the standard parameterless {@code AWSCredentialsProvider.getCredentials()} + * contract. * *

Vendors register an implementation by setting {@code * spark.hadoop.fs.s3a.comet.credential.provider.class} (or the per-bucket form {@code @@ -53,29 +33,20 @@ * {@code spark.sql.catalog..s3.comet.credential.provider.class} for the Iceberg path. The * class must have a public no-arg constructor. * - *

Lifecycle

- * *

Comet keys provider instances by {@code (FQCN, dispatchKey)}, where {@code dispatchKey} is the * Spark V2 catalog name on the Iceberg path and the bucket on the Parquet path. The first time a * given key is seen on an executor, Comet reflects the class, calls {@link #initialize(Map)} once, - * and caches the instance. Subsequent requests for the same key reuse it. Two catalogs that share - * one FQCN therefore get isolated instances with their own {@code initialize} maps. + * and caches the instance. Two catalogs that share one FQCN therefore get isolated instances with + * their own {@code initialize} maps. * *

{@link #initialize(Map)} should be cheap and non-blocking; defer real credential fetches to * the first {@link #getCredentialsForPath} call. {@link #getCredentialsForPath} may be invoked - * concurrently from many native tokio worker threads, so implementations must be thread-safe. - * - *

Caching, refresh, and distribution are the vendor's job

+ * concurrently from many native worker threads, so implementations must be thread-safe. * *

Comet does not maintain a TTL cache, broadcast catalog state, or schedule refresh. Vendors - * decide whether to cache (e.g. by wrapping {@code - * org.apache.iceberg.aws.s3.VendedCredentialsProvider}'s {@code CachedSupplier}), when to refresh, - * and how to distribute driver-side state to executors (typically by reading {@link #initialize}'s - * {@code catalogProperties}, which Comet has already serialized through the native plan op). - * - *

Returns credentials or throws; there is no fall-through return value. A provider that is only - * authoritative for some paths should resolve the default AWS chain itself for the rest. See the - * user guide on cloud credential providers. + * own caching, refresh, and any executor-side state distribution. Returns credentials or throws; + * there is no fall-through return value. See the user guide on cloud credential providers for the + * full contract and examples. */ public interface CometS3CredentialProvider { diff --git a/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java index 614019c68a..8c8c7d6c0d 100644 --- a/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java +++ b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java @@ -35,14 +35,9 @@ *

Test scope only, to keep iceberg-aws and AWS SDK v2 off Comet's runtime classpath. Production * users should copy this into their own jar. * - *

Built only on the Spark 4.x profiles (placed under {@code src/test/spark-4.x}, picked up via - * {@code shims.majorVerSrc}). Excluded from Spark 3.4 because {@code iceberg-spark-runtime-3.4_*} - * does not expose {@code VendedCredentialsProvider} on its test classpath. Excluded from Spark 3.5 - * because Comet pins Iceberg 1.8.1 there, and the in-properties short-circuit that the unit test - * relies on was added in Iceberg 1.9.0 (apache/iceberg#12504); on 1.8.x {@code refreshCredential} - * always issues an HTTP GET against {@code credentials.uri}. - * - *

Test exercised in CI against Iceberg 1.10.0 (the Spark 4.x profile pin). + *

Spark 4.x build only (Iceberg 1.9.0+ is required for the in-properties caching short-circuit + * in {@code VendedCredentialsProvider}; earlier pins issue an HTTP GET against {@code + * credentials.uri} on every refresh). * *

Activation: set {@code spark.sql.catalog..s3.comet.credential.provider.class = * org.apache.comet.cloud.s3.IcebergRESTVendedS3Provider}. Comet calls {@link #initialize} once per @@ -71,7 +66,7 @@ public CometS3Credentials getCredentialsForPath( String sessionToken = (c instanceof AwsSessionCredentials) ? ((AwsSessionCredentials) c).sessionToken() : null; // Expiration is owned by VendedCredentialsProvider's CachedSupplier; we publish 0 so the - // native bridge applies its conservative floor to opendal's cache while the inner + // native bridge applies its conservative floor to `opendal`'s cache while the inner // CachedSupplier handles refresh on its own schedule. return new CometS3Credentials(c.accessKeyId(), c.secretAccessKey(), sessionToken, 0L); } From c7766133d295f9aadaa2691d253d16fc33ea11e0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 21 May 2026 14:12:11 -0400 Subject: [PATCH 20/24] fix format --- .../apache/comet/cloud/s3/CometS3CredentialProvider.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java index 9730ed6851..47b0b82a60 100644 --- a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java @@ -43,10 +43,10 @@ * the first {@link #getCredentialsForPath} call. {@link #getCredentialsForPath} may be invoked * concurrently from many native worker threads, so implementations must be thread-safe. * - *

Comet does not maintain a TTL cache, broadcast catalog state, or schedule refresh. Vendors - * own caching, refresh, and any executor-side state distribution. Returns credentials or throws; - * there is no fall-through return value. See the user guide on cloud credential providers for the - * full contract and examples. + *

Comet does not maintain a TTL cache, broadcast catalog state, or schedule refresh. Vendors own + * caching, refresh, and any executor-side state distribution. Returns credentials or throws; there + * is no fall-through return value. See the user guide on cloud credential providers for the full + * contract and examples. */ public interface CometS3CredentialProvider { From 6738ca03c51a805df0f10ea65f9c9279761cb459 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 21 May 2026 14:19:36 -0400 Subject: [PATCH 21/24] rename docs --- docs/source/contributor-guide/s3-credential-provider-design.md | 2 +- docs/source/user-guide/latest/index.rst | 2 +- ...cloud-credential-providers.md => s3-credential-providers.md} | 2 +- .../org/apache/comet/cloud/s3/CometS3CredentialProvider.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) rename docs/source/user-guide/latest/{cloud-credential-providers.md => s3-credential-providers.md} (99%) diff --git a/docs/source/contributor-guide/s3-credential-provider-design.md b/docs/source/contributor-guide/s3-credential-provider-design.md index 4860b2fe7e..34dd2465d2 100644 --- a/docs/source/contributor-guide/s3-credential-provider-design.md +++ b/docs/source/contributor-guide/s3-credential-provider-design.md @@ -19,7 +19,7 @@ under the License. # S3 Credential Provider SPI: Design Notes -This page captures why the `org.apache.comet.cloud.s3.CometS3CredentialProvider` SPI is shaped the way it is. The user-facing contract and operator setup live in the user guide page on cloud credential providers; this page is for maintainers and reviewers who want the design rationale. +This page captures why the `org.apache.comet.cloud.s3.CometS3CredentialProvider` SPI is shaped the way it is. The user-facing contract and operator setup live in the user guide page on S3 credential providers; this page is for maintainers and reviewers who want the design rationale. ## The gap the SPI fills diff --git a/docs/source/user-guide/latest/index.rst b/docs/source/user-guide/latest/index.rst index 10a48e772e..e284ea1df9 100644 --- a/docs/source/user-guide/latest/index.rst +++ b/docs/source/user-guide/latest/index.rst @@ -50,5 +50,5 @@ to read more. Tuning Guide Metrics Guide Iceberg Guide - Cloud Credential Providers + S3 Credential Providers Kubernetes Guide diff --git a/docs/source/user-guide/latest/cloud-credential-providers.md b/docs/source/user-guide/latest/s3-credential-providers.md similarity index 99% rename from docs/source/user-guide/latest/cloud-credential-providers.md rename to docs/source/user-guide/latest/s3-credential-providers.md index ccbef886e4..510b03a7d6 100644 --- a/docs/source/user-guide/latest/cloud-credential-providers.md +++ b/docs/source/user-guide/latest/s3-credential-providers.md @@ -17,7 +17,7 @@ specific language governing permissions and limitations under the License. --> -# Cloud Credential Providers +# S3 Credential Providers Comet's native S3 readers normally fetch credentials from the standard AWS credential chain (static keys, instance profiles, environment variables, etc.). Some clusters use a vendor-managed mechanism instead, where credentials are issued per request based on a JWT or per S3 path. For those clusters, Comet supports loading a vendor-supplied bridge class that routes every native credential request through the vendor's Java code. diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java index 47b0b82a60..ac3de483da 100644 --- a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java @@ -45,7 +45,7 @@ * *

Comet does not maintain a TTL cache, broadcast catalog state, or schedule refresh. Vendors own * caching, refresh, and any executor-side state distribution. Returns credentials or throws; there - * is no fall-through return value. See the user guide on cloud credential providers for the full + * is no fall-through return value. See the user guide on S3 credential providers for the full * contract and examples. */ public interface CometS3CredentialProvider { From 1895a175556fdb3b165658c0a895880830de3408 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 21 May 2026 17:45:22 -0400 Subject: [PATCH 22/24] Address PR feedback. --- .../org/apache/comet/util/ClassLoaders.java | 43 +++++ .../s3-credential-provider-design.md | 15 +- .../latest/s3-credential-providers.md | 42 ++--- native/core/src/cloud/mod.rs | 23 +++ .../s3/credential_bridge.rs} | 175 +++++------------- native/core/src/cloud/s3/mod.rs | 18 ++ .../src/execution/operators/iceberg_scan.rs | 10 +- native/core/src/lib.rs | 1 + native/core/src/parquet/objectstore/mod.rs | 1 - native/core/src/parquet/objectstore/s3.rs | 53 ++++-- .../src/comet_s3_credential_dispatcher.rs | 6 +- .../cloud/s3/CometS3CredentialContext.java | 62 +++++++ .../cloud/s3/CometS3CredentialDispatcher.java | 129 ++++++++----- .../cloud/s3/CometS3CredentialProvider.java | 38 ++-- .../comet/iceberg/IcebergReflection.scala | 16 +- .../s3/CometS3CredentialDispatcherTest.java | 133 ++++++++++--- .../s3/MinioCometS3CredentialProvider.java | 7 +- .../comet/cloud/s3/NoNoArgCtorProvider.java | 3 +- .../s3/TestCometS3CredentialProvider.java | 23 ++- .../cloud/s3/IcebergRESTVendedS3Provider.java | 3 +- .../s3/IcebergRESTVendedS3ProviderTest.java | 17 +- 21 files changed, 517 insertions(+), 301 deletions(-) create mode 100644 common/src/main/java/org/apache/comet/util/ClassLoaders.java create mode 100644 native/core/src/cloud/mod.rs rename native/core/src/{parquet/objectstore/comet_s3_credential_bridge.rs => cloud/s3/credential_bridge.rs} (64%) create mode 100644 native/core/src/cloud/s3/mod.rs create mode 100644 spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialContext.java diff --git a/common/src/main/java/org/apache/comet/util/ClassLoaders.java b/common/src/main/java/org/apache/comet/util/ClassLoaders.java new file mode 100644 index 0000000000..ee7e620bdc --- /dev/null +++ b/common/src/main/java/org/apache/comet/util/ClassLoaders.java @@ -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.comet.util; + +/** ClassLoader helpers shared across Comet modules. */ +public final class ClassLoaders { + + private ClassLoaders() {} + + /** + * Loads a class using the thread context ClassLoader if available, falling back to the system + * ClassLoader. Spark wires user JARs onto the context ClassLoader, so vendor classes named in + * Spark configs are reachable through this path. + */ + public static Class loadClass(String className) throws ClassNotFoundException { + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + if (classLoader != null) { + // scalastyle:off classforname + return Class.forName(className, true, classLoader); + // scalastyle:on classforname + } + // scalastyle:off classforname + return Class.forName(className); + // scalastyle:on classforname + } +} diff --git a/docs/source/contributor-guide/s3-credential-provider-design.md b/docs/source/contributor-guide/s3-credential-provider-design.md index 34dd2465d2..33f1b46932 100644 --- a/docs/source/contributor-guide/s3-credential-provider-design.md +++ b/docs/source/contributor-guide/s3-credential-provider-design.md @@ -30,7 +30,7 @@ Comet's native scan paths (`object_store` for raw Parquet, `opendal` via `iceber - `AWSCredentialsProvider.getCredentials()` (AWS SDK v1) and `AwsCredentialsProvider.resolveCredentials()` (v2) are parameterless. They cannot vend per-path credentials. - Reflecting into vendor singletons would encode per-vendor class names and lifecycles in Comet and would silently break on vendor upgrades. -A Comet-specific SPI is the narrowest fit: a single Java method that takes `(bucket, path, mode)` and returns `CometS3Credentials`. +A Comet-specific SPI is the narrowest fit: a single Java method that takes a `CometS3CredentialContext` (today wrapping `bucket`, `path`, and access `mode`; new fields can be added without breaking vendors compiled against earlier versions) and returns `CometS3Credentials`. ## Why config-driven activation, not `META-INF/services` @@ -42,13 +42,16 @@ An earlier iteration used `ServiceLoader` discovery. That was rejected because: Activation is modeled on `parquet.crypto.factory.class` (Parquet Modular Encryption KMS, see Comet #2447): the user names a single vendor class and the vendor dispatches across multiple credential backends inside that class if they need to. This mirrors how Iceberg's `DecryptionPropertiesFactory` already behaves for Parquet keys. -## Why `(FQCN, dispatchKey)` keying +## Why `(FQCN, dispatchKey, catalogProperties)` keying -Comet caches one provider instance per `(FQCN, dispatchKey)`. The dispatch key is the Spark V2 catalog name on the Iceberg path and the bucket on the Parquet path. +Comet caches one provider instance per `(FQCN, dispatchKey, catalogProperties)` triple. The dispatch key is the Spark V2 catalog name on the Iceberg path and the bucket on the Parquet path. - Two catalogs that share one provider class (typical in multi-tenant deployments) need isolated `initialize` maps because their `catalogProperties` differ. Without `dispatchKey`, the second `initialize` would either overwrite the first or be silently skipped. - The bucket as `dispatchKey` for Parquet gives vendors per-bucket isolation when the same provider is named under several `fs.s3a.bucket..comet.credential.provider.class` keys. -- Keying solely by FQCN would force vendors to encode multi-tenant routing in static state. The pair-key keeps each call site independent. +- `catalogProperties` enters the key to handle multi-tenant JVMs (Spark Connect, Thrift Server, `SparkSession.newSession()`) where two sessions can configure the same provider class against the same `dispatchKey` but with different REST endpoints, OAuth tokens, or vendor keys. Without it the second session would silently use the first session's credentials. +- Keying solely by FQCN would force vendors to encode multi-tenant routing in static state. The triple-key keeps each call site independent. + +`ensureInitialized` returns a `long` handle that the native bridge stashes and replays on every per-request call. Routing per-request lookups by handle avoids re-sending the property bag across JNI on the hot path and unambiguously selects the right provider when the same `(FQCN, dispatchKey)` pair maps to multiple instances. ## Why fresh construction in `initialize`, not probing a JVM-wide static @@ -94,6 +97,10 @@ The full unfiltered FileIO property bag crosses JNI as `catalog_properties`. The The SPI returns a `CometS3Credentials` or throws. There is no sentinel "I do not know" return. Vendors that are only authoritative for some paths resolve the default AWS chain themselves for the rest and return the result. This matches the contract on every other AWS credential SPI in the JVM ecosystem (AWS SDK v1/v2, Hadoop S3A, Iceberg `VendedCredentialsProvider`). +## Lifecycle: `AutoCloseable` plus a JVM shutdown hook + +`CometS3CredentialProvider` extends `AutoCloseable` with a default no-op `close()`. The dispatcher installs a JVM shutdown hook that iterates every cached instance and calls `close()`, swallowing per-provider exceptions so a slow or buggy vendor cannot block other providers from cleaning up. Stateless providers ignore this entirely; vendors that hold long-lived resources (HTTP clients, scheduled-refresh executors, STS connection pools) override `close()` to release them. Shutdown hooks are best-effort, so a `SIGKILL` or abrupt JVM termination skips them; vendors must not rely on `close()` for correctness, only for resource hygiene. + ## Iceberg path: error message fidelity caveat When the bridge is wired into `iceberg-rust`, the outer `reqsign-core::ProvideCredentialChain` currently swallows thrown exceptions into "no credential" before the request reaches `opendal`. The credential is still not issued and the request still fails, but the message is degraded to an opaque anonymous-request failure. No Comet change fixes this; it is resolved upstream when `iceberg-rust` stops wrapping custom loaders in its outer chain or moves its S3 backend to `object_store`. diff --git a/docs/source/user-guide/latest/s3-credential-providers.md b/docs/source/user-guide/latest/s3-credential-providers.md index 510b03a7d6..eeb9e4ffb7 100644 --- a/docs/source/user-guide/latest/s3-credential-providers.md +++ b/docs/source/user-guide/latest/s3-credential-providers.md @@ -110,23 +110,27 @@ Implement `org.apache.comet.cloud.s3.CometS3CredentialProvider`: ```java package org.apache.comet.cloud.s3; -public interface CometS3CredentialProvider { - /** Called once per (FQCN, dispatchKey) before any per-request call. Optional. */ +public interface CometS3CredentialProvider extends AutoCloseable { + /** Called once per (FQCN, dispatchKey, catalogProperties) before any per-request call. Optional. */ default void initialize(java.util.Map catalogProperties) {} - CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) throws Exception; + CometS3Credentials getCredentialsForPath(CometS3CredentialContext context) throws Exception; + + /** Invoked from the dispatcher's JVM shutdown hook. Default is a no-op. */ + @Override default void close() throws Exception {} } ``` -The class must have a public no-arg constructor. `getCredentialsForPath` may be invoked concurrently from many native tokio worker threads; the implementation must be thread-safe. +The class must have a public no-arg constructor. `getCredentialsForPath` may be invoked concurrently from many native tokio worker threads; the implementation must be thread-safe. The supplied `CometS3CredentialContext` exposes `getBucket()`, `getPath()`, and `getMode()`; future Comet releases may add accessors here without changing the method signature, so vendors compiled against today's API stay binary-compatible. ### Lifecycle -Comet keys provider instances by `(FQCN, dispatchKey)`. The dispatch key is the Spark V2 catalog name on the Iceberg path and the S3 bucket name on the Parquet path. The first time a given key is seen on an executor, Comet reflects the class, calls `initialize(Map)` exactly once, and caches the instance for the JVM lifetime. Two catalogs sharing one provider FQCN therefore get isolated instances with their own `initialize` maps. +Comet keys provider instances by `(FQCN, dispatchKey, catalogProperties)`. The dispatch key is the Spark V2 catalog name on the Iceberg path and the S3 bucket name on the Parquet path. The first time a given key is seen on an executor, Comet reflects the class, calls `initialize(Map)` exactly once, and caches the instance for the JVM lifetime. Two catalogs sharing one provider FQCN therefore get isolated instances with their own `initialize` maps. Including `catalogProperties` in the key matters in multi-tenant JVMs (Spark Connect, Thrift Server, `SparkSession.newSession()`) where two sessions can otherwise collide on the same `(FQCN, dispatchKey)` and have the second session silently use the first session's credentials. `initialize` should be cheap and non-blocking. Defer real credential fetches (REST round-trips, STS calls) to the first `getCredentialsForPath` invocation. On the Iceberg path the supplied `catalogProperties` carries the unfiltered FileIO bag, including REST-vended fields like `credentials.uri`, OAuth tokens, and any vendor-custom keys you set on the catalog config. The map may contain secrets, so do not log it. +`close()` is invoked from a JVM shutdown hook installed by the dispatcher. The default no-op is fine for stateless providers. Override it to release HTTP clients, scheduled-refresh executors, or STS connection pools. Shutdown hooks are best-effort: a `SIGKILL` or abrupt JVM termination skips them, so do not depend on `close()` for correctness. + ### Caching, refresh, and distribution Comet does not maintain a TTL cache, broadcast catalog state, or schedule refresh. Vendors decide: @@ -160,10 +164,9 @@ If your provider is authoritative only for some paths, resolve the default AWS c private final DefaultCredentialsProvider defaultChain = DefaultCredentialsProvider.create(); @Override -public CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) throws Exception { - if (handlesPath(bucket, path)) { - return mintFromMyVendorService(bucket, path, mode); +public CometS3Credentials getCredentialsForPath(CometS3CredentialContext ctx) throws Exception { + if (handlesPath(ctx.getBucket(), ctx.getPath())) { + return mintFromMyVendorService(ctx.getBucket(), ctx.getPath(), ctx.getMode()); } AwsCredentials c = defaultChain.resolveCredentials(); String token = (c instanceof AwsSessionCredentials) @@ -184,18 +187,17 @@ public final class MyCometCredentialProvider implements CometS3CredentialProvide private final DefaultVendor fallback = ...; @Override - public CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) throws Exception { - if (bucket.startsWith("data-prod-")) return prod.fetch(bucket, path, mode); - if (bucket.equals("partner-shared")) return sts.assumeRole(bucket, path, mode); - return fallback.fetch(bucket, path); + public CometS3Credentials getCredentialsForPath(CometS3CredentialContext ctx) throws Exception { + if (ctx.getBucket().startsWith("data-prod-")) return prod.fetch(ctx); + if (ctx.getBucket().equals("partner-shared")) return sts.assumeRole(ctx); + return fallback.fetch(ctx.getBucket(), ctx.getPath()); } } ``` Per-bucket Hadoop overrides (`fs.s3a.bucket..comet.credential.provider.class`) are also available if you prefer to ship multiple vendor classes and pick by bucket in config rather than in code. -For Iceberg deployments where two catalogs share one provider class but need isolated state, configure the same FQCN on both catalogs and read your discriminator from `initialize`'s `catalogProperties`. Each catalog gets its own provider instance because Comet keys by `(FQCN, catalogName)`: +For Iceberg deployments where two catalogs share one provider class but need isolated state, configure the same FQCN on both catalogs and read your discriminator from `initialize`'s `catalogProperties`. Each catalog gets its own provider instance because Comet keys by `(FQCN, catalogName, catalogProperties)`: ```java public final class MyMultiTenantProvider implements CometS3CredentialProvider { @@ -207,9 +209,8 @@ public final class MyMultiTenantProvider implements CometS3CredentialProvider { } @Override - public CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) { - return mintForTenant(tenantId, bucket, path, mode); + public CometS3Credentials getCredentialsForPath(CometS3CredentialContext ctx) { + return mintForTenant(tenantId, ctx.getBucket(), ctx.getPath(), ctx.getMode()); } } ``` @@ -228,8 +229,7 @@ public final class IcebergRESTVendedS3Provider implements CometS3CredentialProvi } @Override - public CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) { + public CometS3Credentials getCredentialsForPath(CometS3CredentialContext ctx) { AwsCredentials c = provider.resolveCredentials(); String token = (c instanceof AwsSessionCredentials) ? ((AwsSessionCredentials) c).sessionToken() : null; diff --git a/native/core/src/cloud/mod.rs b/native/core/src/cloud/mod.rs new file mode 100644 index 0000000000..634d37b26d --- /dev/null +++ b/native/core/src/cloud/mod.rs @@ -0,0 +1,23 @@ +// 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. + +//! Cloud-side helpers shared across Comet's native scan paths. +//! +//! Today this hosts the JNI bridge to the JVM `CometS3CredentialDispatcher` SPI, which is reused +//! by the raw Parquet (`object_store`) path and the Iceberg (`opendal` via `iceberg-rust`) path. + +pub mod s3; diff --git a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs b/native/core/src/cloud/s3/credential_bridge.rs similarity index 64% rename from native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs rename to native/core/src/cloud/s3/credential_bridge.rs index 995b632011..7de2f2c0d6 100644 --- a/native/core/src/parquet/objectstore/comet_s3_credential_bridge.rs +++ b/native/core/src/cloud/s3/credential_bridge.rs @@ -15,39 +15,24 @@ // specific language governing permissions and limitations // under the License. -//! JNI bridge to the `CometS3CredentialDispatcher` SPI, exposed as -//! `object_store::CredentialProvider` and `reqsign_core::ProvideCredential` for the raw Parquet -//! and Iceberg scan paths respectively. -//! -//! The bridge is activated by setting `fs.s3a.comet.credential.provider.class` (optionally -//! per-bucket) in the Hadoop configuration. The vendor's named class is instantiated once on -//! first use inside the JVM dispatcher and reused for the executor lifetime. +//! JNI bridge to the JVM `CometS3CredentialDispatcher` SPI, exposed as +//! `object_store::CredentialProvider` (raw Parquet path) and `reqsign_core::ProvideCredential` +//! (Iceberg via `opendal`). //! //! ```text //! JVM Native (Rust) //! --- ------------- //! -//! fs.s3a.comet.credential.provider.class s3.rs (object_store) -//! | iceberg_scan.rs (opendal) -//! v | -//! CometS3CredentialDispatcher v -//! (per-class instance cache) CometS3CredentialBridge -//! ^ impl object_store::CredentialProvider -//! | impl reqsign_core::ProvideCredential +//! spark.hadoop.fs.s3a.comet.credential parquet/objectstore/s3.rs (object_store) +//! .provider.class execution/operators/iceberg_scan.rs (opendal) //! | | -//! +<---- JNI call ----------------------------+ -//! | getCredentialsForPath(className, bucket, path, mode ordinal) -//! v -//! vendor CometS3CredentialProvider -//! | +//! v v +//! CometS3CredentialDispatcher CometS3CredentialBridge +//! ^ | +//! | ensureInitialized -> long handle | +//! +<--- getCredentialsForPath(handle, ...) ------+ //! v -//! CometS3Credentials POJO -//! | -//! +------- JNI field reads ---------------->+ -//! | -//! v -//! AwsCredential / IcebergAwsCredential -//! (used to sign S3 requests) +//! vendor CometS3CredentialProvider -> CometS3Credentials //! ``` use crate::execution::operators::ExecutionError; @@ -71,15 +56,6 @@ use std::collections::HashMap; use std::fmt; use std::sync::Arc; use std::time::Duration; -use url::Url; - -/// Hadoop-style config key (without `fs.s3a.` prefix) naming the vendor `CometS3CredentialProvider` -/// FQCN. Iceberg's catalog properties use the same suffix under their `s3.` namespace. -pub(crate) const PROVIDER_CLASS_PROPERTY: &str = "comet.credential.provider.class"; - -/// Iceberg-namespaced form of [`PROVIDER_CLASS_PROPERTY`], read from a Spark catalog's `s3.*` -/// property bag (`spark.sql.catalog..s3.comet.credential.provider.class`). -pub(crate) const ICEBERG_PROVIDER_CLASS_PROPERTY: &str = "s3.comet.credential.provider.class"; /// Cap on opendal's credential cache when the provider does not report an expiry. Prevents the /// executor from holding a stale credential for the entire job lifetime. @@ -98,33 +74,22 @@ pub enum AccessMode { Write = 1, } -/// Resolve the configured provider class for the given bucket via `super::s3::get_config_trimmed`, -/// which already implements the per-bucket-then-global `fs.s3a.` lookup. Returns the trimmed FQCN -/// if non-empty. -pub fn lookup_provider_class<'a>( - configs: &'a HashMap, - bucket: &str, -) -> Option<&'a str> { - super::s3::get_config_trimmed(configs, bucket, PROVIDER_CLASS_PROPERTY) - .filter(|s| !s.is_empty()) -} - -/// Per-request credential provider that delegates to the Java SPI via JNI. Constructed once per S3 -/// store or FileIO; calls `ensureInitialized` synchronously at construction so the JVM provider -/// instance is ready before any per-request fetch. +/// Per-scan credential provider that delegates to the JVM SPI via JNI. /// -/// The four String arguments threaded through every `getCredentialsForPath` call (provider class, -/// dispatch key, bucket, path) are immutable for the bridge's lifetime, so we cache them once at -/// construction as JNI global refs to avoid per-call `env.new_string` allocations on the hot path. +/// `handle` is the JVM-allocated identity for the `(provider_class, dispatch_key, +/// catalog_properties)` triple, returned by `ensureInitialized` at construction. Per-request +/// calls carry `(handle, bucket, path, mode)`, which lets the JVM disambiguate multi-tenant +/// providers without re-sending the property bag and saves one JNI string allocation on the hot +/// path. `bucket` and `path` are immutable for the bridge's lifetime so we cache them as JNI +/// global refs. pub struct CometS3CredentialBridge { provider_class: String, dispatch_key: String, bucket: String, path: String, mode: AccessMode, - /// Cached JNI globals for the four constant String arguments to `getCredentialsForPath`. - provider_class_jstr: Arc>>, - dispatch_key_jstr: Arc>>, + handle: i64, + /// Cached JNI globals for the two constant String arguments to `getCredentialsForPath`. bucket_jstr: Arc>>, path_jstr: Arc>>, } @@ -134,6 +99,7 @@ impl fmt::Debug for CometS3CredentialBridge { f.debug_struct("CometS3CredentialBridge") .field("provider_class", &self.provider_class) .field("dispatch_key", &self.dispatch_key) + .field("handle", &self.handle) .field("bucket", &self.bucket) .field("path", &self.path) .field("mode", &self.mode) @@ -142,10 +108,9 @@ impl fmt::Debug for CometS3CredentialBridge { } impl CometS3CredentialBridge { - /// Construct the bridge and run a one-shot `ensureInitialized` call against the JVM - /// dispatcher. `dispatch_key` scopes provider instances on the JVM side: bucket name on the - /// Parquet path, catalog name on the Iceberg path. `catalog_properties` is forwarded to - /// `CometS3CredentialProvider.initialize(Map)` exactly once per `(class, dispatchKey)` pair. + /// Run `ensureInitialized` synchronously and stash the returned handle for the bridge's + /// lifetime. `dispatch_key` is the bucket on the Parquet path, the catalog name on the Iceberg + /// path. `catalog_properties` is forwarded into the vendor's `initialize(Map)`. pub fn new( provider_class: impl Into, dispatch_key: impl Into, @@ -159,78 +124,37 @@ impl CometS3CredentialBridge { let bucket = bucket.into(); let path = path.into(); - let (provider_class_jstr, dispatch_key_jstr, bucket_jstr, path_jstr) = - JVMClasses::with_env(|env| -> Result<_, ExecutionError> { - let pc = env.new_string(&provider_class).map_err(|e| { - ExecutionError::GeneralError(format!("new_string(provider_class): {e}")) - })?; - let dk = env.new_string(&dispatch_key).map_err(|e| { - ExecutionError::GeneralError(format!("new_string(dispatch_key): {e}")) - })?; - let b = env.new_string(&bucket).map_err(|e| { - ExecutionError::GeneralError(format!("new_string(bucket): {e}")) - })?; - let p = env - .new_string(&path) - .map_err(|e| ExecutionError::GeneralError(format!("new_string(path): {e}")))?; - let pc_g = Arc::new(jni_new_global_ref!(env, pc).map_err(|e| { - ExecutionError::GeneralError(format!("global_ref(provider_class): {e}")) - })?); - let dk_g = Arc::new(jni_new_global_ref!(env, dk).map_err(|e| { - ExecutionError::GeneralError(format!("global_ref(dispatch_key): {e}")) - })?); - let b_g = Arc::new(jni_new_global_ref!(env, b).map_err(|e| { + let (bucket_jstr, path_jstr) = JVMClasses::with_env(|env| -> Result<_, ExecutionError> { + let b = env + .new_string(&bucket) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(bucket): {e}")))?; + let p = env + .new_string(&path) + .map_err(|e| ExecutionError::GeneralError(format!("new_string(path): {e}")))?; + let b_g = + Arc::new(jni_new_global_ref!(env, b).map_err(|e| { ExecutionError::GeneralError(format!("global_ref(bucket): {e}")) })?); - let p_g = - Arc::new(jni_new_global_ref!(env, p).map_err(|e| { - ExecutionError::GeneralError(format!("global_ref(path): {e}")) - })?); - Ok((pc_g, dk_g, b_g, p_g)) - })?; + let p_g = Arc::new( + jni_new_global_ref!(env, p) + .map_err(|e| ExecutionError::GeneralError(format!("global_ref(path): {e}")))?, + ); + Ok((b_g, p_g)) + })?; - ensure_initialized(&provider_class, &dispatch_key, catalog_properties)?; + let handle = ensure_initialized(&provider_class, &dispatch_key, catalog_properties)?; Ok(Self { provider_class, dispatch_key, bucket, path, mode, - provider_class_jstr, - dispatch_key_jstr, + handle, bucket_jstr, path_jstr, }) } - /// Shared constructor for the s3.rs and iceberg_scan.rs call sites. Returns `Ok(None)` when no - /// provider class is configured so callers can fall through to their default credential path. - pub fn for_url( - url: &Url, - configs: &HashMap, - mode: AccessMode, - dispatch_key: &str, - catalog_properties: &HashMap, - ) -> Result, ExecutionError> { - let bucket = match url.host_str() { - Some(b) => b, - None => return Ok(None), - }; - let provider_class = match lookup_provider_class(configs, bucket) { - Some(c) => c.to_string(), - None => return Ok(None), - }; - Self::new( - provider_class, - dispatch_key, - bucket, - url.path(), - mode, - catalog_properties, - ) - .map(Some) - } - fn fetch_raw(&self) -> Result { JVMClasses::with_env(|env| -> Result { let mode = self.mode as jint; @@ -238,8 +162,7 @@ impl CometS3CredentialBridge { let creds_obj: JObject = unsafe { jni_static_call!(env, comet_s3_credential_dispatcher.get_credentials_for_path( - self.provider_class_jstr.as_obj(), - self.dispatch_key_jstr.as_obj(), + self.handle, self.bucket_jstr.as_obj(), self.path_jstr.as_obj(), mode @@ -287,8 +210,8 @@ fn ensure_initialized( provider_class: &str, dispatch_key: &str, catalog_properties: &HashMap, -) -> Result<(), ExecutionError> { - JVMClasses::with_env(|env| -> Result<(), ExecutionError> { +) -> Result { + JVMClasses::with_env(|env| -> Result { let provider_class_jstr = env.new_string(provider_class).map_err(|e| { ExecutionError::GeneralError(format!("new_string(provider_class): {e}")) })?; @@ -297,14 +220,14 @@ fn ensure_initialized( .map_err(|e| ExecutionError::GeneralError(format!("new_string(dispatch_key): {e}")))?; let props_obj = build_java_string_map(env, catalog_properties)?; - unsafe { + let handle: i64 = unsafe { jni_static_call!(env, comet_s3_credential_dispatcher.ensure_initialized( &provider_class_jstr, &dispatch_key_jstr, &props_obj - ) -> () - )?; - } - Ok(()) + ) -> i64 + )? + }; + Ok(handle) }) } diff --git a/native/core/src/cloud/s3/mod.rs b/native/core/src/cloud/s3/mod.rs new file mode 100644 index 0000000000..054919a691 --- /dev/null +++ b/native/core/src/cloud/s3/mod.rs @@ -0,0 +1,18 @@ +// 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. + +pub mod credential_bridge; diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index a7e46adc99..5bcd168af7 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -43,16 +43,20 @@ use iceberg::io::{FileIO, FileIOBuilder, StorageFactory}; use iceberg_storage_opendal::CustomAwsCredentialLoader; use iceberg_storage_opendal::OpenDalStorageFactory; +use crate::cloud::s3::credential_bridge::{AccessMode, CometS3CredentialBridge}; use crate::execution::operators::ExecutionError; -use crate::parquet::objectstore::comet_s3_credential_bridge::{ - AccessMode, CometS3CredentialBridge, ICEBERG_PROVIDER_CLASS_PROPERTY, -}; use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkPhysicalExprAdapterFactory; use datafusion_comet_spark_expr::EvalMode; use datafusion_physical_expr_adapter::{PhysicalExprAdapter, PhysicalExprAdapterFactory}; use iceberg::scan::FileScanTask; +/// Iceberg-namespaced activation knob for the `CometS3CredentialProvider` SPI, read from a Spark +/// catalog's `s3.*` property bag (`spark.sql.catalog..s3.comet.credential.provider.class`). +/// Mirrors the Hadoop-style `comet.credential.provider.class` used on the Parquet/object_store +/// path, but lives here because only the Iceberg path consumes it. +const ICEBERG_PROVIDER_CLASS_PROPERTY: &str = "s3.comet.credential.provider.class"; + /// Iceberg table scan operator that uses iceberg-rust to read Iceberg tables. /// /// Executes pre-planned FileScanTasks for efficient parallel scanning. diff --git a/native/core/src/lib.rs b/native/core/src/lib.rs index f4ae0b8834..19a2d774a0 100644 --- a/native/core/src/lib.rs +++ b/native/core/src/lib.rs @@ -65,6 +65,7 @@ pub mod jvm_bridge { use errors::{try_unwrap_or_throw, CometError, CometResult}; +pub mod cloud; pub mod execution; pub mod parquet; // this module is for non release only. Intended for debugging/profiling purposes diff --git a/native/core/src/parquet/objectstore/mod.rs b/native/core/src/parquet/objectstore/mod.rs index a12835d5cf..bedae08f69 100644 --- a/native/core/src/parquet/objectstore/mod.rs +++ b/native/core/src/parquet/objectstore/mod.rs @@ -15,5 +15,4 @@ // specific language governing permissions and limitations // under the License. -pub mod comet_s3_credential_bridge; pub mod s3; diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index e524db81cc..16d60440fd 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -20,10 +20,8 @@ use std::collections::HashMap; use std::sync::OnceLock; use url::Url; +use crate::cloud::s3::credential_bridge::{AccessMode, CometS3CredentialBridge}; use crate::execution::jni_api::get_runtime; -use crate::parquet::objectstore::comet_s3_credential_bridge::{ - AccessMode, CometS3CredentialBridge, -}; use async_trait::async_trait; use aws_config::{ ecs::EcsCredentialsProvider, environment::EnvironmentVariableCredentialsProvider, @@ -85,21 +83,25 @@ pub fn create_store( // catalog_properties is empty since vendors on the Parquet path read from Hadoop conf, not // catalog props. let empty_props: HashMap = HashMap::new(); - let bridge = match CometS3CredentialBridge::for_url( - url, - configs, - AccessMode::Read, - bucket, - &empty_props, - ) { - Ok(b) => b, - Err(e) => { - log::warn!( - "Failed to initialize CometS3CredentialBridge for {bucket}: {e}; \ - falling back to default credential chain" - ); - None - } + let bridge = match lookup_provider_class(configs, bucket) { + Some(provider_class) => match CometS3CredentialBridge::new( + provider_class, + bucket, + bucket, + url.path(), + AccessMode::Read, + &empty_props, + ) { + Ok(b) => Some(b), + Err(e) => { + log::warn!( + "Failed to initialize CometS3CredentialBridge for {bucket}: {e}; \ + falling back to default credential chain" + ); + None + } + }, + None => None, }; builder = if let Some(bridge) = bridge { builder.with_credentials(Arc::new(bridge)) @@ -320,6 +322,21 @@ pub(super) fn get_config_trimmed<'a>( get_config(configs, bucket, property).map(|s| s.trim()) } +/// Hadoop-style config key (without `fs.s3a.` prefix) naming the vendor `CometS3CredentialProvider` +/// FQCN. Looked up via [`get_config_trimmed`] so the per-bucket override +/// (`fs.s3a.bucket..comet.credential.provider.class`) is honored before the global +/// (`fs.s3a.comet.credential.provider.class`). +const PROVIDER_CLASS_PROPERTY: &str = "comet.credential.provider.class"; + +/// Returns the configured `CometS3CredentialProvider` FQCN for the given bucket, or `None` when +/// no provider is registered. Trims surrounding whitespace and treats an empty string as unset. +fn lookup_provider_class<'a>( + configs: &'a HashMap, + bucket: &str, +) -> Option<&'a str> { + get_config_trimmed(configs, bucket, PROVIDER_CLASS_PROPERTY).filter(|s| !s.is_empty()) +} + // Hadoop S3A credential provider constants const HADOOP_IAM_INSTANCE: &str = "org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider"; const HADOOP_SIMPLE: &str = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider"; diff --git a/native/jni-bridge/src/comet_s3_credential_dispatcher.rs b/native/jni-bridge/src/comet_s3_credential_dispatcher.rs index 1fb2fef47f..b38ca0348a 100644 --- a/native/jni-bridge/src/comet_s3_credential_dispatcher.rs +++ b/native/jni-bridge/src/comet_s3_credential_dispatcher.rs @@ -52,14 +52,14 @@ impl<'a> CometS3CredentialDispatcher<'a> { method_ensure_initialized: env.get_static_method_id( JNIString::new(Self::JVM_CLASS), jni::jni_str!("ensureInitialized"), - jni::jni_sig!("(Ljava/lang/String;Ljava/lang/String;Ljava/util/Map;)V"), + jni::jni_sig!("(Ljava/lang/String;Ljava/lang/String;Ljava/util/Map;)J"), )?, - method_ensure_initialized_ret: ReturnType::Primitive(Primitive::Void), + method_ensure_initialized_ret: ReturnType::Primitive(Primitive::Long), method_get_credentials_for_path: env.get_static_method_id( JNIString::new(Self::JVM_CLASS), jni::jni_str!("getCredentialsForPath"), jni::jni_sig!( - "(Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;I)Lorg/apache/comet/cloud/s3/CometS3Credentials;" + "(JLjava/lang/String;Ljava/lang/String;I)Lorg/apache/comet/cloud/s3/CometS3Credentials;" ), )?, method_get_credentials_for_path_ret: ReturnType::Object, diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialContext.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialContext.java new file mode 100644 index 0000000000..4dfcb3adb7 --- /dev/null +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialContext.java @@ -0,0 +1,62 @@ +/* + * 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.comet.cloud.s3; + +import java.util.Objects; + +/** + * Per-request context passed to {@link + * CometS3CredentialProvider#getCredentialsForPath(CometS3CredentialContext)}. New fields can be + * added here without breaking vendors compiled against earlier Comet versions, so the SPI method + * signature does not change when Comet learns to forward additional per-request information (e.g. + * region, write-target ARN). + */ +public final class CometS3CredentialContext { + + private final String bucket; + private final String path; + private final CometS3AccessMode mode; + + public CometS3CredentialContext(String bucket, String path, CometS3AccessMode mode) { + this.bucket = Objects.requireNonNull(bucket, "bucket"); + this.path = Objects.requireNonNull(path, "path"); + this.mode = Objects.requireNonNull(mode, "mode"); + } + + /** S3 bucket name (no scheme, no path). */ + public String getBucket() { + return bucket; + } + + /** Object key or prefix, leading slash included (matches the URL path component). */ + public String getPath() { + return path; + } + + /** Access intent for this request. */ + public CometS3AccessMode getMode() { + return mode; + } + + @Override + public String toString() { + return "CometS3CredentialContext{bucket=" + bucket + ", path=" + path + ", mode=" + mode + "}"; + } +} diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java index 66da294107..311b36a64a 100644 --- a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java @@ -21,101 +21,109 @@ import java.lang.reflect.InvocationTargetException; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.comet.util.ClassLoaders; + /** - * JNI entry point invoked from native code to resolve a {@link CometS3CredentialProvider}. + * JNI entry point that resolves a {@link CometS3CredentialProvider} for native code. * - *

Native code names a vendor class via the activation knob ({@code - * fs.s3a.comet.credential.provider.class} for the Parquet path, {@code - * s3.comet.credential.provider.class} on a Spark catalog property for the Iceberg path) and a - * {@code dispatchKey} that scopes the instance: catalog name on the Iceberg path, bucket name on - * the Parquet path. Each {@code (FQCN, dispatchKey)} key gets its own instance, so two catalogs - * sharing one provider class get isolated state. + *

{@link #ensureInitialized} reflects the named class, runs {@code initialize(Map)} once, and + * returns a {@code long} handle. {@link #getCredentialsForPath} takes that handle on every + * per-request call. See the design notes in the contributor guide for why the SPI is shaped this + * way (keying, multi-tenant isolation, shutdown lifecycle). */ public final class CometS3CredentialDispatcher { private static final Logger LOG = LoggerFactory.getLogger(CometS3CredentialDispatcher.class); - private static final ConcurrentHashMap INSTANCES = + private static final ConcurrentHashMap KEY_TO_HANDLE = + new ConcurrentHashMap<>(); + private static final ConcurrentHashMap INSTANCES = new ConcurrentHashMap<>(); + private static final AtomicLong HANDLE_SEQ = new AtomicLong(1L); private static final CometS3AccessMode[] MODES = CometS3AccessMode.values(); + static { + Runtime.getRuntime() + .addShutdownHook( + new Thread(CometS3CredentialDispatcher::closeAll, "comet-s3-credential-shutdown")); + } + private CometS3CredentialDispatcher() {} /** - * Reflects and initializes the named provider for {@code (FQCN, dispatchKey)} exactly once per - * JVM. Subsequent calls with the same key are no-ops. Native code invokes this synchronously when - * {@code CometS3CredentialBridge} is constructed at plan time, before any per-request {@link - * #getCredentialsForPath} call. {@code catalogProperties} carries the unfiltered FileIO property - * bag on the Iceberg path and is empty on the Parquet path. + * Reflects and initializes the named provider on first call for the {@code (FQCN, dispatchKey, + * catalogProperties)} triple, and returns a handle reused by subsequent {@link + * #getCredentialsForPath} calls. */ - public static void ensureInitialized( + public static long ensureInitialized( String providerClassName, String dispatchKey, Map catalogProperties) { if (providerClassName == null || providerClassName.isEmpty()) { throw new IllegalArgumentException( "providerClassName is empty; native side should not call without a configured class"); } - InstanceKey key = new InstanceKey(providerClassName, dispatchKey == null ? "" : dispatchKey); - Map props = - catalogProperties == null ? Collections.emptyMap() : catalogProperties; - INSTANCES.computeIfAbsent( + Map snapshot = + catalogProperties == null + ? Collections.emptyMap() + : Collections.unmodifiableMap(new HashMap<>(catalogProperties)); + InstanceKey key = + new InstanceKey(providerClassName, dispatchKey == null ? "" : dispatchKey, snapshot); + return KEY_TO_HANDLE.computeIfAbsent( key, k -> { CometS3CredentialProvider provider = instantiate(k.providerClassName); - provider.initialize(props); - return provider; + provider.initialize(k.catalogProperties); + long handle = HANDLE_SEQ.getAndIncrement(); + INSTANCES.put(handle, new RegisteredProvider(provider, k)); + return handle; }); } /** - * Invoked by native code on every per-request credential fetch. The instance must have been - * created by a prior {@link #ensureInitialized} call; otherwise this throws. {@code mode} is the + * Invoked by native code on every per-request credential fetch. {@code handle} must be a value + * returned by a prior {@link #ensureInitialized} call; otherwise this throws. {@code mode} is the * {@link CometS3AccessMode} ordinal. */ public static CometS3Credentials getCredentialsForPath( - String providerClassName, String dispatchKey, String bucket, String path, int mode) - throws Exception { - if (providerClassName == null || providerClassName.isEmpty()) { - throw new IllegalArgumentException( - "providerClassName is empty; native side should not call without a configured class"); - } + long handle, String bucket, String path, int mode) throws Exception { if (mode < 0 || mode >= MODES.length) { throw new IllegalArgumentException("Invalid CometS3AccessMode ordinal: " + mode); } - InstanceKey key = new InstanceKey(providerClassName, dispatchKey == null ? "" : dispatchKey); - CometS3CredentialProvider provider = INSTANCES.get(key); - if (provider == null) { + RegisteredProvider registered = INSTANCES.get(handle); + if (registered == null) { throw new IllegalStateException( - "CometS3CredentialProvider " - + providerClassName - + " (dispatchKey=" - + key.dispatchKey - + ") was not initialized; ensureInitialized must be called before" - + " getCredentialsForPath"); + "CometS3CredentialProvider handle " + + handle + + " was not initialized; " + + "ensureInitialized must be called before getCredentialsForPath"); } CometS3AccessMode accessMode = MODES[mode]; if (LOG.isDebugEnabled()) { LOG.debug( - "Fetching credentials via {} (dispatchKey={}) for bucket={} path={} mode={}", - providerClassName, - key.dispatchKey, + "Fetching credentials via {} (dispatchKey={}, handle={}) for bucket={} path={} mode={}", + registered.key.providerClassName, + registered.key.dispatchKey, + handle, bucket, path, accessMode); } - return provider.getCredentialsForPath(bucket, path, accessMode); + return registered.provider.getCredentialsForPath( + new CometS3CredentialContext(bucket, path, accessMode)); } private static CometS3CredentialProvider instantiate(String providerClassName) { Class clazz; try { - clazz = Class.forName(providerClassName); + clazz = ClassLoaders.loadClass(providerClassName); } catch (ClassNotFoundException e) { throw new IllegalStateException( "CometS3CredentialProvider class not found: " @@ -140,13 +148,33 @@ private static CometS3CredentialProvider instantiate(String providerClassName) { } } + /** Visible for tests; otherwise invoked from the JVM shutdown hook. */ + static void closeAll() { + for (RegisteredProvider registered : INSTANCES.values()) { + try { + registered.provider.close(); + } catch (Throwable t) { + LOG.warn( + "Exception from {} (dispatchKey={}).close() during shutdown", + registered.key.providerClassName, + registered.key.dispatchKey, + t); + } + } + INSTANCES.clear(); + KEY_TO_HANDLE.clear(); + } + private static final class InstanceKey { final String providerClassName; final String dispatchKey; + final Map catalogProperties; - InstanceKey(String providerClassName, String dispatchKey) { + InstanceKey( + String providerClassName, String dispatchKey, Map catalogProperties) { this.providerClassName = providerClassName; this.dispatchKey = dispatchKey; + this.catalogProperties = catalogProperties; } @Override @@ -155,12 +183,23 @@ public boolean equals(Object o) { if (!(o instanceof InstanceKey)) return false; InstanceKey other = (InstanceKey) o; return providerClassName.equals(other.providerClassName) - && dispatchKey.equals(other.dispatchKey); + && dispatchKey.equals(other.dispatchKey) + && catalogProperties.equals(other.catalogProperties); } @Override public int hashCode() { - return Objects.hash(providerClassName, dispatchKey); + return Objects.hash(providerClassName, dispatchKey, catalogProperties); + } + } + + private static final class RegisteredProvider { + final CometS3CredentialProvider provider; + final InstanceKey key; + + RegisteredProvider(CometS3CredentialProvider provider, InstanceKey key) { + this.provider = provider; + this.key = key; } } } diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java index ac3de483da..62155bd760 100644 --- a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java @@ -33,22 +33,15 @@ * {@code spark.sql.catalog..s3.comet.credential.provider.class} for the Iceberg path. The * class must have a public no-arg constructor. * - *

Comet keys provider instances by {@code (FQCN, dispatchKey)}, where {@code dispatchKey} is the - * Spark V2 catalog name on the Iceberg path and the bucket on the Parquet path. The first time a - * given key is seen on an executor, Comet reflects the class, calls {@link #initialize(Map)} once, - * and caches the instance. Two catalogs that share one FQCN therefore get isolated instances with - * their own {@code initialize} maps. + *

{@link #initialize(Map)} runs once per Comet-cached instance before any {@link + * #getCredentialsForPath} call, must be cheap and non-blocking, and may receive secrets in its map. + * {@link #getCredentialsForPath} may be invoked concurrently from many native worker threads so + * implementations must be thread-safe; it returns credentials or throws (no fall-through). * - *

{@link #initialize(Map)} should be cheap and non-blocking; defer real credential fetches to - * the first {@link #getCredentialsForPath} call. {@link #getCredentialsForPath} may be invoked - * concurrently from many native worker threads, so implementations must be thread-safe. - * - *

Comet does not maintain a TTL cache, broadcast catalog state, or schedule refresh. Vendors own - * caching, refresh, and any executor-side state distribution. Returns credentials or throws; there - * is no fall-through return value. See the user guide on S3 credential providers for the full - * contract and examples. + *

See the user guide on S3 credential providers for caching, refresh, and multi-tenant isolation + * guidance. */ -public interface CometS3CredentialProvider { +public interface CometS3CredentialProvider extends AutoCloseable { /** * Called once per {@code (FQCN, dispatchKey)} on each executor before any {@link @@ -62,11 +55,18 @@ public interface CometS3CredentialProvider { default void initialize(Map catalogProperties) {} /** - * @param bucket S3 bucket name (no scheme, no path) - * @param path object key or prefix, leading slash included (matches the URL path component) - * @param mode access intent for this request + * @param context per-request context (bucket, path, access mode). Fields can be added to {@link + * CometS3CredentialContext} in future Comet releases without changing this method signature, + * so vendors compiled against today's API stay binary-compatible. * @return non-null credentials; {@code null} is a contract violation */ - CometS3Credentials getCredentialsForPath(String bucket, String path, CometS3AccessMode mode) - throws Exception; + CometS3Credentials getCredentialsForPath(CometS3CredentialContext context) throws Exception; + + /** + * Releases vendor-owned resources (HTTP clients, refresh executors, STS connection pools). + * Invoked from a best-effort JVM shutdown hook installed by the dispatcher; default no-op suits + * stateless providers. The hook swallows exceptions thrown here. + */ + @Override + default void close() throws Exception {} } diff --git a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala index 8ee2291361..e1a468bf23 100644 --- a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala +++ b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala @@ -22,6 +22,8 @@ package org.apache.comet.iceberg import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession +import org.apache.comet.util.ClassLoaders + /** * Shared reflection utilities for Iceberg operations. * @@ -101,19 +103,7 @@ object IcebergReflection extends Logging { * @return * The loaded Class object */ - def loadClass(className: String): Class[_] = { - val classLoader = Thread.currentThread().getContextClassLoader - if (classLoader != null) { - // scalastyle:off classforname - Class.forName(className, true, classLoader) - // scalastyle:on classforname - } else { - // Fallback to default classloader if context classloader is null - // scalastyle:off classforname - Class.forName(className) - // scalastyle:on classforname - } - } + def loadClass(className: String): Class[_] = ClassLoaders.loadClass(className) /** * Searches through class hierarchy to find a method (including protected methods). diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java b/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java index 1bf09712bf..d527b4866f 100644 --- a/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java +++ b/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java @@ -27,6 +27,7 @@ import org.junit.Test; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; @@ -42,19 +43,22 @@ public class CometS3CredentialDispatcherTest { @Before public void resetTestProvider() { + // Each test starts with an empty dispatcher cache so providers and counters from prior tests + // do not leak through closeAll() / initCount / closeCount assertions. + CometS3CredentialDispatcher.closeAll(); TestCometS3CredentialProvider.reset(); } - private void init() { - CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, DK, Collections.emptyMap()); + private long init() { + return CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, DK, Collections.emptyMap()); } @Test public void getCredentialsRoundTripsThroughProvider() throws Exception { - init(); + long handle = init(); CometS3Credentials creds = CometS3CredentialDispatcher.getCredentialsForPath( - TEST_PROVIDER, DK, "my-bucket", "path/to/object", READ); + handle, "my-bucket", "path/to/object", READ); assertNotNull(creds); assertEquals("AKIATEST", creds.getAccessKeyId()); @@ -70,17 +74,17 @@ public void getCredentialsRoundTripsThroughProvider() throws Exception { @Test public void writeModeIsForwarded() throws Exception { - init(); - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b", "k", WRITE); + long handle = init(); + CometS3CredentialDispatcher.getCredentialsForPath(handle, "b", "k", WRITE); assertEquals(CometS3AccessMode.WRITE, TestCometS3CredentialProvider.lastMode); } @Test public void unknownModeRejected() { - init(); + long handle = init(); assertThrows( IllegalArgumentException.class, - () -> CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b", "k", 99)); + () -> CometS3CredentialDispatcher.getCredentialsForPath(handle, "b", "k", 99)); } @Test @@ -91,9 +95,6 @@ public void emptyClassNameRejected() { assertThrows( IllegalArgumentException.class, () -> CometS3CredentialDispatcher.ensureInitialized(null, DK, Collections.emptyMap())); - assertThrows( - IllegalArgumentException.class, - () -> CometS3CredentialDispatcher.getCredentialsForPath("", DK, "b", "k", READ)); } @Test @@ -135,8 +136,7 @@ public void getWithoutEnsureInitializedThrows() { assertThrows( IllegalStateException.class, () -> - CometS3CredentialDispatcher.getCredentialsForPath( - TEST_PROVIDER, "never-initialized", "b", "k", READ)); + CometS3CredentialDispatcher.getCredentialsForPath(Long.MAX_VALUE, "b", "k", READ)); assertTrue(thrown.getMessage().contains("not initialized")); } @@ -145,11 +145,13 @@ public void initializeCalledExactlyOncePerKey() { Map props = new HashMap<>(); props.put("tenant-id", "T1"); - CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); - CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); - CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); + long h1 = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); + long h2 = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); + long h3 = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "cat-a", props); assertEquals(1, TestCometS3CredentialProvider.initCount.get()); + assertEquals(h1, h2); + assertEquals(h1, h3); } @Test @@ -159,53 +161,124 @@ public void distinctDispatchKeysIsolateInstances() throws Exception { Map propsB = new HashMap<>(); propsB.put("tenant-id", "T-B"); - CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "iso-cat-a", propsA); - CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "iso-cat-b", propsB); + long ha = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "iso-cat-a", propsA); + long hb = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "iso-cat-b", propsB); + assertNotEquals(ha, hb); assertEquals(2, TestCometS3CredentialProvider.initCount.get()); - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "iso-cat-a", "b", "k", READ); + CometS3CredentialDispatcher.getCredentialsForPath(ha, "b", "k", READ); assertEquals("T-A", TestCometS3CredentialProvider.lastTenantSeen); - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, "iso-cat-b", "b", "k", READ); + CometS3CredentialDispatcher.getCredentialsForPath(hb, "b", "k", READ); + assertEquals("T-B", TestCometS3CredentialProvider.lastTenantSeen); + } + + /** + * Multi-tenant collision: two sessions sharing the same FQCN and dispatchKey but configured with + * different catalogProperties (e.g. different REST endpoints, rotated tokens) must get distinct + * instances. Without the catalogProperties component in the key, the second session would + * silently use the first's credentials. + */ + @Test + public void distinctCatalogPropertiesIsolateInstances() throws Exception { + Map propsA = new HashMap<>(); + propsA.put("tenant-id", "T-A"); + propsA.put("credentials.uri", "https://rest-a.example/credentials"); + Map propsB = new HashMap<>(); + propsB.put("tenant-id", "T-B"); + propsB.put("credentials.uri", "https://rest-b.example/credentials"); + + long ha = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "shared-cat", propsA); + long hb = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "shared-cat", propsB); + + assertNotEquals(ha, hb); + assertEquals(2, TestCometS3CredentialProvider.initCount.get()); + + CometS3CredentialDispatcher.getCredentialsForPath(ha, "b", "k", READ); + assertEquals("T-A", TestCometS3CredentialProvider.lastTenantSeen); + + CometS3CredentialDispatcher.getCredentialsForPath(hb, "b", "k", READ); assertEquals("T-B", TestCometS3CredentialProvider.lastTenantSeen); } @Test public void providerExceptionsPropagate() { - init(); + long handle = init(); IllegalStateException boom = new IllegalStateException("simulated STS failure"); TestCometS3CredentialProvider.throwOnNext = boom; Exception thrown = assertThrows( Exception.class, - () -> - CometS3CredentialDispatcher.getCredentialsForPath( - TEST_PROVIDER, DK, "b", "k", READ)); + () -> CometS3CredentialDispatcher.getCredentialsForPath(handle, "b", "k", READ)); assertSame(boom, thrown); } @Test public void nullSessionTokenAllowed() throws Exception { - init(); + long handle = init(); TestCometS3CredentialProvider.nextResult = new CometS3Credentials("AKIA", "sec", null, 0L); CometS3Credentials creds = - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b", "k", READ); + CometS3CredentialDispatcher.getCredentialsForPath(handle, "b", "k", READ); assertNull(creds.getSessionToken()); } @Test public void providerReceivesEachCallSeparately() throws Exception { - init(); - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b1", "k1", READ); - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b2", "k2", READ); - CometS3CredentialDispatcher.getCredentialsForPath(TEST_PROVIDER, DK, "b3", "k3", READ); + long handle = init(); + CometS3CredentialDispatcher.getCredentialsForPath(handle, "b1", "k1", READ); + CometS3CredentialDispatcher.getCredentialsForPath(handle, "b2", "k2", READ); + CometS3CredentialDispatcher.getCredentialsForPath(handle, "b3", "k3", READ); assertEquals(3, TestCometS3CredentialProvider.callCount.get()); assertEquals("b3", TestCometS3CredentialProvider.lastBucket); assertEquals("k3", TestCometS3CredentialProvider.lastPath); } + + /** + * The JVM shutdown hook calls {@link CometS3CredentialDispatcher#closeAll()} on every cached + * provider. We invoke it directly here since junit cannot exercise a real JVM shutdown. + */ + @Test + public void closeAllInvokesEveryProvider() { + Map propsA = new HashMap<>(); + propsA.put("tenant-id", "T-A"); + Map propsB = new HashMap<>(); + propsB.put("tenant-id", "T-B"); + + CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "close-a", propsA); + CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "close-b", propsB); + + CometS3CredentialDispatcher.closeAll(); + + assertEquals(2, TestCometS3CredentialProvider.closeCount.get()); + + // After closeAll, handles are no longer registered. Re-init creates a fresh instance. + long fresh = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "close-a", propsA); + assertEquals(3, TestCometS3CredentialProvider.initCount.get()); + assertNotNull(fresh); + } + + /** + * A failing vendor close() must not block other providers from being closed. The dispatcher + * swallows exceptions during shutdown so a slow or buggy provider cannot keep an executor JVM + * from exiting cleanly. + */ + @Test + public void closeAllSwallowsProviderExceptions() { + CometS3CredentialDispatcher.ensureInitialized( + TEST_PROVIDER, "close-throws-a", Collections.emptyMap()); + CometS3CredentialDispatcher.ensureInitialized( + TEST_PROVIDER, "close-throws-b", Collections.emptyMap()); + + TestCometS3CredentialProvider.throwOnClose = new IllegalStateException("simulated cleanup"); + + CometS3CredentialDispatcher.closeAll(); + + // Both close() invocations ran even though one (the first to be visited) threw. + assertEquals(2, TestCometS3CredentialProvider.closeCount.get()); + } } diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java index ed83cc914e..e5a14aa169 100644 --- a/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java +++ b/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java @@ -105,11 +105,10 @@ public void initialize(Map catalogProperties) { } @Override - public CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) { + public CometS3Credentials getCredentialsForPath(CometS3CredentialContext context) { CALL_COUNT.incrementAndGet(); - LAST_BUCKET.set(bucket); - LAST_PATH.set(path); + LAST_BUCKET.set(context.getBucket()); + LAST_PATH.set(context.getPath()); Credentials c = CREDS.get(); if (c == null) { throw new IllegalStateException( diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java index 606a482f7d..7f3d7e2dbc 100644 --- a/spark/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java +++ b/spark/src/test/java/org/apache/comet/cloud/s3/NoNoArgCtorProvider.java @@ -26,8 +26,7 @@ public class NoNoArgCtorProvider implements CometS3CredentialProvider { public NoNoArgCtorProvider(String required) {} @Override - public CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) { + public CometS3Credentials getCredentialsForPath(CometS3CredentialContext context) { throw new UnsupportedOperationException("never instantiable"); } } diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java index 53595a9cf7..77ce86c836 100644 --- a/spark/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java +++ b/spark/src/test/java/org/apache/comet/cloud/s3/TestCometS3CredentialProvider.java @@ -30,11 +30,13 @@ public class TestCometS3CredentialProvider implements CometS3CredentialProvider static final AtomicInteger initCount = new AtomicInteger(0); static final AtomicInteger callCount = new AtomicInteger(0); + static final AtomicInteger closeCount = new AtomicInteger(0); static volatile String lastBucket; static volatile String lastPath; static volatile CometS3AccessMode lastMode; static volatile String lastTenantSeen; static volatile RuntimeException throwOnNext; + static volatile Exception throwOnClose; static volatile CometS3Credentials nextResult = new CometS3Credentials("AKIATEST", "secret", "session-tok", 0L); @@ -43,11 +45,13 @@ public class TestCometS3CredentialProvider implements CometS3CredentialProvider static void reset() { initCount.set(0); callCount.set(0); + closeCount.set(0); lastBucket = null; lastPath = null; lastMode = null; lastTenantSeen = null; throwOnNext = null; + throwOnClose = null; nextResult = new CometS3Credentials("AKIATEST", "secret", "session-tok", 0L); } @@ -58,12 +62,11 @@ public void initialize(Map catalogProperties) { } @Override - public CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) { + public CometS3Credentials getCredentialsForPath(CometS3CredentialContext context) { callCount.incrementAndGet(); - lastBucket = bucket; - lastPath = path; - lastMode = mode; + lastBucket = context.getBucket(); + lastPath = context.getPath(); + lastMode = context.getMode(); lastTenantSeen = tenantId; RuntimeException toThrow = throwOnNext; if (toThrow != null) { @@ -72,4 +75,14 @@ public CometS3Credentials getCredentialsForPath( } return nextResult; } + + @Override + public void close() throws Exception { + closeCount.incrementAndGet(); + Exception toThrow = throwOnClose; + if (toThrow != null) { + throwOnClose = null; + throw toThrow; + } + } } diff --git a/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java index 8c8c7d6c0d..0d5cde70d0 100644 --- a/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java +++ b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3Provider.java @@ -54,8 +54,7 @@ public void initialize(Map catalogProperties) { } @Override - public CometS3Credentials getCredentialsForPath( - String bucket, String path, CometS3AccessMode mode) { + public CometS3Credentials getCredentialsForPath(CometS3CredentialContext context) { VendedCredentialsProvider p = provider; if (p == null) { throw new IllegalStateException( diff --git a/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java index 28b851fb18..b49ca74012 100644 --- a/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java +++ b/spark/src/test/spark-4.x/java/org/apache/comet/cloud/s3/IcebergRESTVendedS3ProviderTest.java @@ -54,7 +54,9 @@ public void initializeThenGetReturnsVendedCredentials() { IcebergRESTVendedS3Provider p = new IcebergRESTVendedS3Provider(); p.initialize(staticVendedProps()); - CometS3Credentials c = p.getCredentialsForPath("bucket", "/k", CometS3AccessMode.READ); + CometS3Credentials c = + p.getCredentialsForPath( + new CometS3CredentialContext("bucket", "/k", CometS3AccessMode.READ)); assertEquals("AKIA_TEST", c.getAccessKeyId()); assertEquals("secret_TEST", c.getSecretAccessKey()); @@ -68,7 +70,9 @@ public void getBeforeInitializeThrows() { IcebergRESTVendedS3Provider p = new IcebergRESTVendedS3Provider(); assertThrows( IllegalStateException.class, - () -> p.getCredentialsForPath("bucket", "/k", CometS3AccessMode.READ)); + () -> + p.getCredentialsForPath( + new CometS3CredentialContext("bucket", "/k", CometS3AccessMode.READ))); } @Test @@ -76,9 +80,12 @@ public void multipleCallsServedByCache() { IcebergRESTVendedS3Provider p = new IcebergRESTVendedS3Provider(); p.initialize(staticVendedProps()); - CometS3Credentials a = p.getCredentialsForPath("b", "/k", CometS3AccessMode.READ); - CometS3Credentials b = p.getCredentialsForPath("b", "/k2", CometS3AccessMode.READ); - CometS3Credentials c = p.getCredentialsForPath("b", "/k3", CometS3AccessMode.READ); + CometS3Credentials a = + p.getCredentialsForPath(new CometS3CredentialContext("b", "/k", CometS3AccessMode.READ)); + CometS3Credentials b = + p.getCredentialsForPath(new CometS3CredentialContext("b", "/k2", CometS3AccessMode.READ)); + CometS3Credentials c = + p.getCredentialsForPath(new CometS3CredentialContext("b", "/k3", CometS3AccessMode.READ)); // CachedSupplier hands out the same identity-equal credential until expiry; we assert by // value so the test does not depend on AWS SDK internal caching semantics. From e37d61ed304613d466b7afb30475e9b7b5212826 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 21 May 2026 19:30:19 -0400 Subject: [PATCH 23/24] Clean up docs trying to get line count on the diff down. --- .../latest/s3-credential-providers.md | 2 +- native/core/src/cloud/s3/credential_bridge.rs | 41 ++++--------------- .../src/execution/operators/iceberg_scan.rs | 23 ++++------- native/core/src/parquet/objectstore/s3.rs | 12 ++---- native/jni-bridge/src/lib.rs | 2 - .../cloud/s3/CometS3CredentialContext.java | 5 +-- .../cloud/s3/CometS3CredentialProvider.java | 40 +++++------------- .../comet/cloud/s3/CometS3Credentials.java | 9 ++-- .../org/apache/comet/util/ClassLoaders.java | 0 .../comet/iceberg/IcebergReflection.scala | 35 ++++------------ .../s3/CometS3CredentialDispatcherTest.java | 18 +------- .../s3/MinioCometS3CredentialProvider.java | 3 +- 12 files changed, 49 insertions(+), 141 deletions(-) rename {common => spark}/src/main/java/org/apache/comet/util/ClassLoaders.java (100%) diff --git a/docs/source/user-guide/latest/s3-credential-providers.md b/docs/source/user-guide/latest/s3-credential-providers.md index eeb9e4ffb7..7a98bf3c2a 100644 --- a/docs/source/user-guide/latest/s3-credential-providers.md +++ b/docs/source/user-guide/latest/s3-credential-providers.md @@ -256,7 +256,7 @@ Vendor implementations need the Comet SPI classes at compile time only. Use `pro ```xml org.apache.datafusion - comet-common-spark${spark.version.short}_${scala.binary.version} + comet-spark-spark${spark.version.short}_${scala.binary.version} ${comet.version} provided diff --git a/native/core/src/cloud/s3/credential_bridge.rs b/native/core/src/cloud/s3/credential_bridge.rs index 7de2f2c0d6..4f799a1ffd 100644 --- a/native/core/src/cloud/s3/credential_bridge.rs +++ b/native/core/src/cloud/s3/credential_bridge.rs @@ -17,23 +17,7 @@ //! JNI bridge to the JVM `CometS3CredentialDispatcher` SPI, exposed as //! `object_store::CredentialProvider` (raw Parquet path) and `reqsign_core::ProvideCredential` -//! (Iceberg via `opendal`). -//! -//! ```text -//! JVM Native (Rust) -//! --- ------------- -//! -//! spark.hadoop.fs.s3a.comet.credential parquet/objectstore/s3.rs (object_store) -//! .provider.class execution/operators/iceberg_scan.rs (opendal) -//! | | -//! v v -//! CometS3CredentialDispatcher CometS3CredentialBridge -//! ^ | -//! | ensureInitialized -> long handle | -//! +<--- getCredentialsForPath(handle, ...) ------+ -//! v -//! vendor CometS3CredentialProvider -> CometS3Credentials -//! ``` +//! (Iceberg via `opendal`). See `docs/source/contributor-guide/s3-credential-provider-design.md`. use crate::execution::operators::ExecutionError; use crate::jvm_bridge::{jni_new_global_ref, jni_static_call, JVMClasses}; @@ -61,27 +45,22 @@ use std::time::Duration; /// executor from holding a stale credential for the entire job lifetime. const DEFAULT_EXPIRY_WHEN_UNKNOWN: Duration = Duration::from_secs(300); -/// Once-per-process latch for the "missing expiry" warning; bridges are per-scan so a per-bridge -/// latch would log once per scan on the same misbehaving provider. +/// Once-per-process latch for the "missing expiry" warning. Bridges are per-scan, so a per-bridge +/// latch would re-log on every scan. static WARNED_MISSING_EXPIRY: OnceCell<()> = OnceCell::new(); /// Access intent forwarded to the Java SPI. Ordinal must match the JVM `CometS3AccessMode` enum. #[derive(Debug, Clone, Copy)] pub enum AccessMode { Read = 0, - /// No native write path yet; kept so the SPI contract is complete. #[allow(dead_code)] Write = 1, } -/// Per-scan credential provider that delegates to the JVM SPI via JNI. -/// -/// `handle` is the JVM-allocated identity for the `(provider_class, dispatch_key, -/// catalog_properties)` triple, returned by `ensureInitialized` at construction. Per-request -/// calls carry `(handle, bucket, path, mode)`, which lets the JVM disambiguate multi-tenant -/// providers without re-sending the property bag and saves one JNI string allocation on the hot -/// path. `bucket` and `path` are immutable for the bridge's lifetime so we cache them as JNI -/// global refs. +/// Per-scan credential provider that delegates to the JVM SPI via JNI. `handle` is the JVM-side +/// identity for the `(provider_class, dispatch_key, catalog_properties)` triple returned by +/// `ensureInitialized`. `bucket_jstr` / `path_jstr` are interned once at construction to avoid +/// per-call `new_string` allocations on the hot path. pub struct CometS3CredentialBridge { provider_class: String, dispatch_key: String, @@ -89,7 +68,6 @@ pub struct CometS3CredentialBridge { path: String, mode: AccessMode, handle: i64, - /// Cached JNI globals for the two constant String arguments to `getCredentialsForPath`. bucket_jstr: Arc>>, path_jstr: Arc>>, } @@ -108,9 +86,6 @@ impl fmt::Debug for CometS3CredentialBridge { } impl CometS3CredentialBridge { - /// Run `ensureInitialized` synchronously and stash the returned handle for the bridge's - /// lifetime. `dispatch_key` is the bucket on the Parquet path, the catalog name on the Iceberg - /// path. `catalog_properties` is forwarded into the vendor's `initialize(Map)`. pub fn new( provider_class: impl Into, dispatch_key: impl Into, @@ -232,7 +207,7 @@ fn ensure_initialized( } /// Construct a `java.util.HashMap` and populate it. Called once per bridge at -/// construction (per-scan), so the per-call HashMap/put cost is amortized away from the hot path. +/// construction, so per-call HashMap/put cost stays off the hot path. fn build_java_string_map<'a>( env: &mut jni::Env<'a>, map: &HashMap, diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 5bcd168af7..ce1388f78f 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -51,10 +51,8 @@ use datafusion_comet_spark_expr::EvalMode; use datafusion_physical_expr_adapter::{PhysicalExprAdapter, PhysicalExprAdapterFactory}; use iceberg::scan::FileScanTask; -/// Iceberg-namespaced activation knob for the `CometS3CredentialProvider` SPI, read from a Spark -/// catalog's `s3.*` property bag (`spark.sql.catalog..s3.comet.credential.provider.class`). -/// Mirrors the Hadoop-style `comet.credential.provider.class` used on the Parquet/object_store -/// path, but lives here because only the Iceberg path consumes it. +/// Activation key for the `CometS3CredentialProvider` SPI on the Iceberg path, read from a Spark +/// catalog's `s3.*` property bag. const ICEBERG_PROVIDER_CLASS_PROPERTY: &str = "s3.comet.credential.provider.class"; /// Iceberg table scan operator that uses iceberg-rust to read Iceberg tables. @@ -71,9 +69,8 @@ pub struct IcebergScanExec { /// may contain OAuth tokens, REST `credentials.uri`, and other secrets the credential bridge /// needs. Redacted in `Debug` so plan dumps and tracing do not leak credentials. catalog_properties: HashMap, - /// Spark V2 catalog name; forwarded as dispatchKey to the credential bridge so multiple - /// catalogs sharing one provider FQCN get isolated provider instances. Empty when the table - /// has no catalog identity. + /// Spark V2 catalog name; forwarded as dispatchKey to the credential bridge. Empty when the + /// table has no catalog identity. catalog_name: String, /// Pre-planned file scan tasks tasks: Vec, @@ -269,10 +266,8 @@ impl IcebergScanExec { const STORAGE_PROPERTY_PREFIXES: &[&str] = &["s3.", "gcs.", "adls.", "client."]; -/// Wires the configured Comet credential provider into opendal's S3 service for this scan, or -/// returns `None` so opendal falls back to its default credential chain. Iceberg passes its -/// per-catalog properties (`spark.sql.catalog..*` after Spark stripping), so the activation -/// key here is `s3.comet.credential.provider.class` to match Iceberg's `s3.*` namespace. +/// Wires the configured Comet credential provider into opendal's S3 service, or returns `None` +/// so opendal falls back to its default credential chain. fn build_s3_credential_loader( metadata_location: &str, catalog_properties: &HashMap, @@ -284,10 +279,8 @@ fn build_s3_credential_loader( .get(ICEBERG_PROVIDER_CLASS_PROPERTY) .map(|s| s.trim()) .filter(|s| !s.is_empty())?; - // Catalog name scopes provider instances on the JVM dispatcher so two catalogs sharing one - // provider class get isolated state. Falls back to the bucket when the table has no catalog - // identity (e.g. HadoopTables loaded by raw path), keeping the previous behavior in that - // case. + // Fall back to the bucket when the table has no catalog identity (e.g. HadoopTables loaded by + // raw path). let dispatch_key: &str = if catalog_name.is_empty() { bucket } else { diff --git a/native/core/src/parquet/objectstore/s3.rs b/native/core/src/parquet/objectstore/s3.rs index 16d60440fd..821bac8da0 100644 --- a/native/core/src/parquet/objectstore/s3.rs +++ b/native/core/src/parquet/objectstore/s3.rs @@ -79,9 +79,7 @@ pub fn create_store( source: "Missing bucket name in S3 URL".into(), })?; - // Parquet path: dispatch_key = bucket (matches the per-bucket override config namespace); - // catalog_properties is empty since vendors on the Parquet path read from Hadoop conf, not - // catalog props. + // Parquet path: catalog_properties is empty; vendors here read from Hadoop conf. let empty_props: HashMap = HashMap::new(); let bridge = match lookup_provider_class(configs, bucket) { Some(provider_class) => match CometS3CredentialBridge::new( @@ -322,14 +320,10 @@ pub(super) fn get_config_trimmed<'a>( get_config(configs, bucket, property).map(|s| s.trim()) } -/// Hadoop-style config key (without `fs.s3a.` prefix) naming the vendor `CometS3CredentialProvider` -/// FQCN. Looked up via [`get_config_trimmed`] so the per-bucket override -/// (`fs.s3a.bucket..comet.credential.provider.class`) is honored before the global -/// (`fs.s3a.comet.credential.provider.class`). +/// Activation key (without `fs.s3a.` prefix) naming the vendor `CometS3CredentialProvider` FQCN. +/// Per-bucket override is honored via [`get_config_trimmed`]. const PROVIDER_CLASS_PROPERTY: &str = "comet.credential.provider.class"; -/// Returns the configured `CometS3CredentialProvider` FQCN for the given bucket, or `None` when -/// no provider is registered. Trims surrounding whitespace and treats an empty string as unset. fn lookup_provider_class<'a>( configs: &'a HashMap, bucket: &str, diff --git a/native/jni-bridge/src/lib.rs b/native/jni-bridge/src/lib.rs index 5420cc8157..490e80d076 100644 --- a/native/jni-bridge/src/lib.rs +++ b/native/jni-bridge/src/lib.rs @@ -236,8 +236,6 @@ pub struct JVMClasses<'a> { /// `None` if the class is not on the classpath. pub comet_udf_bridge: Option>, /// JNI handles for the CometS3CredentialDispatcher SPI and the CometS3Credentials POJO. - /// Always present (the classes ship in `comet-common`); whether a vendor provider is actually - /// registered is a separate runtime check. pub comet_s3_credential_dispatcher: CometS3CredentialDispatcher<'a>, } diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialContext.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialContext.java index 4dfcb3adb7..105ac79ee7 100644 --- a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialContext.java +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialContext.java @@ -24,9 +24,8 @@ /** * Per-request context passed to {@link * CometS3CredentialProvider#getCredentialsForPath(CometS3CredentialContext)}. New fields can be - * added here without breaking vendors compiled against earlier Comet versions, so the SPI method - * signature does not change when Comet learns to forward additional per-request information (e.g. - * region, write-target ARN). + * added here without changing the SPI method signature, so vendors compiled against earlier + * versions stay binary-compatible. */ public final class CometS3CredentialContext { diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java index 62155bd760..a77b9f2f01 100644 --- a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java @@ -22,50 +22,32 @@ import java.util.Map; /** - * SPI for supplying AWS credentials to Comet's native S3 readers, which bypass Spark's Hadoop S3A - * code path. Vendors implement this when path-aware or vendor-managed credential mechanisms cannot - * be reached through the standard parameterless {@code AWSCredentialsProvider.getCredentials()} - * contract. + * SPI for supplying AWS credentials to Comet's native S3 readers, which bypass Hadoop S3A. See the + * user guide (operator setup, vendor contract) and the contributor-guide design notes for the + * rationale. * - *

Vendors register an implementation by setting {@code - * spark.hadoop.fs.s3a.comet.credential.provider.class} (or the per-bucket form {@code - * spark.hadoop.fs.s3a.bucket..comet.credential.provider.class}) for the Parquet path, or - * {@code spark.sql.catalog..s3.comet.credential.provider.class} for the Iceberg path. The - * class must have a public no-arg constructor. - * - *

{@link #initialize(Map)} runs once per Comet-cached instance before any {@link - * #getCredentialsForPath} call, must be cheap and non-blocking, and may receive secrets in its map. - * {@link #getCredentialsForPath} may be invoked concurrently from many native worker threads so - * implementations must be thread-safe; it returns credentials or throws (no fall-through). - * - *

See the user guide on S3 credential providers for caching, refresh, and multi-tenant isolation - * guidance. + *

{@link #getCredentialsForPath} may be invoked concurrently from many native worker threads; + * implementations must be thread-safe. It returns credentials or throws (no fall-through). */ public interface CometS3CredentialProvider extends AutoCloseable { /** - * Called once per {@code (FQCN, dispatchKey)} on each executor before any {@link - * #getCredentialsForPath} call. The {@code catalogProperties} map carries the full FileIO - * property bag for the Iceberg path (including {@code credentials.uri}, OAuth tokens, vendor keys - * like {@code tenant-id}) and is empty on the Parquet path. The default no-op keeps Parquet - * vendors source-compatible. + * Called once per Comet-cached instance before any {@link #getCredentialsForPath} call. Must be + * cheap and non-blocking. On the Iceberg path the map carries the unfiltered FileIO bag; on the + * Parquet path it is empty. * - * @param catalogProperties unfiltered FileIO/catalog properties; may contain secrets, do not log + * @param catalogProperties may contain secrets, do not log */ default void initialize(Map catalogProperties) {} /** - * @param context per-request context (bucket, path, access mode). Fields can be added to {@link - * CometS3CredentialContext} in future Comet releases without changing this method signature, - * so vendors compiled against today's API stay binary-compatible. * @return non-null credentials; {@code null} is a contract violation */ CometS3Credentials getCredentialsForPath(CometS3CredentialContext context) throws Exception; /** - * Releases vendor-owned resources (HTTP clients, refresh executors, STS connection pools). - * Invoked from a best-effort JVM shutdown hook installed by the dispatcher; default no-op suits - * stateless providers. The hook swallows exceptions thrown here. + * Invoked from the dispatcher's best-effort JVM shutdown hook. Default no-op suits stateless + * providers; override to release HTTP clients, refresh executors, etc. */ @Override default void close() throws Exception {} diff --git a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java index 6c443bde52..3342f28c24 100644 --- a/spark/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java +++ b/spark/src/main/java/org/apache/comet/cloud/s3/CometS3Credentials.java @@ -22,12 +22,9 @@ import java.util.Objects; /** - * Credentials returned by a {@link CometS3CredentialProvider}. Fields are read back over JNI by - * name, so the field names are part of the cross-language contract. - * - *

{@code sessionToken} is null for non-STS credentials. {@code expirationEpochMillis} of {@code - * 0} means "unknown"; the Iceberg path then caps opendal's cache at a short fallback to avoid - * serving stale credentials for the executor lifetime. + * Credentials returned by a {@link CometS3CredentialProvider}. Field names are read back over JNI + * by name and are part of the cross-language contract. {@code sessionToken} is null for non-STS + * credentials; {@code expirationEpochMillis} of {@code 0} means "unknown". */ public final class CometS3Credentials { diff --git a/common/src/main/java/org/apache/comet/util/ClassLoaders.java b/spark/src/main/java/org/apache/comet/util/ClassLoaders.java similarity index 100% rename from common/src/main/java/org/apache/comet/util/ClassLoaders.java rename to spark/src/main/java/org/apache/comet/util/ClassLoaders.java diff --git a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala index e1a468bf23..b4e18da6af 100644 --- a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala +++ b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala @@ -664,10 +664,8 @@ object IcebergReflection extends Logging { * @param catalogProperties * Catalog properties for FileIO (S3 credentials, regions, etc.) * @param catalogName - * Spark V2 catalog name that loaded this table, if it can be derived. Forwarded as - * `dispatchKey` to the native CometS3CredentialBridge so two catalogs sharing one provider FQCN - * get isolated provider instances. `None` when the table has no catalog identity (e.g. - * HadoopTables loaded by raw path). + * Spark V2 catalog name forwarded as `dispatchKey` to CometS3CredentialBridge. `None` when the + * table has no catalog identity (e.g. HadoopTables loaded by raw path). */ case class CometIcebergNativeScanMetadata( table: Any, @@ -740,20 +738,16 @@ object CometIcebergNativeScanMetadata extends Logging { } /** - * Best-effort extraction of the Spark V2 catalog name from an Iceberg `Table`. Iceberg's - * `Table.name()` returns `catalog.namespace.table` for tables loaded through a catalog. We - * intersect that name against the V2 catalogs Spark has registered so a value like `s3.foo` is - * not mistaken for a catalog `s3` when no such catalog exists. Falls back to the dotted-prefix - * split when the catalog manager is not reachable or the name does not match. Returns `None` - * when the table has no catalog identity (e.g. HadoopTables loaded by raw path) or when - * reflection fails. + * Extracts the Spark V2 catalog name from an Iceberg `Table`. `Table.name()` returns + * `catalog.namespace.table` for tables loaded through a catalog; we intersect against the + * registered V2 catalogs so a value like `s3.foo` is not mistaken for a catalog `s3`. Returns + * `None` for HadoopTables loaded by raw path or when reflection fails. */ private[iceberg] def deriveCatalogName(table: Any): Option[String] = deriveCatalogName(table, registeredCatalogNames _) /** - * Test seam for [[deriveCatalogName(table:Any)]]. The `knownCatalogNames` thunk lets tests - * inject a fixed catalog set without bootstrapping a SparkSession. + * Test seam that lets tests inject a fixed catalog set without bootstrapping a SparkSession. */ private[iceberg] def deriveCatalogName( table: Any, @@ -773,15 +767,6 @@ object CometIcebergNativeScanMetadata extends Logging { } } - /** - * Calls Iceberg's public `Table.name()` reflectively. Uses `getMethod` so the interface default - * is reachable when a concrete table class does not override it. Matches the pattern used for - * `Field.name()` / `Column.name()` elsewhere in this file. - * - * `name()` is a default method on `org.apache.iceberg.Table`. A thrown exception here means the - * classpath is wrong or the object is not actually an Iceberg Table, so log at `warn` to make - * it visible. A `null` return is legitimate (anonymous tables) and not noteworthy. - */ private def invokeTableName(table: Any): Option[String] = { try { table.getClass.getMethod("name").invoke(table) match { @@ -791,10 +776,8 @@ object CometIcebergNativeScanMetadata extends Logging { } } catch { case e: Exception => - logWarning( - s"Iceberg reflection: Table.name() not callable on ${table.getClass.getName}; " + - "native S3 credential dispatch will fall back to bucket-keyed isolation: " + - s"${e.getMessage}") + logWarning(s"Iceberg reflection: Table.name() not callable on ${table.getClass.getName}; " + + s"native S3 credential dispatch will fall back to bucket-keyed isolation: ${e.getMessage}") None } } diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java b/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java index d527b4866f..9601497f1b 100644 --- a/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java +++ b/spark/src/test/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcherTest.java @@ -174,12 +174,7 @@ public void distinctDispatchKeysIsolateInstances() throws Exception { assertEquals("T-B", TestCometS3CredentialProvider.lastTenantSeen); } - /** - * Multi-tenant collision: two sessions sharing the same FQCN and dispatchKey but configured with - * different catalogProperties (e.g. different REST endpoints, rotated tokens) must get distinct - * instances. Without the catalogProperties component in the key, the second session would - * silently use the first's credentials. - */ + /** Multi-tenant collision: same FQCN and dispatchKey, different catalogProperties; isolated. */ @Test public void distinctCatalogPropertiesIsolateInstances() throws Exception { Map propsA = new HashMap<>(); @@ -238,10 +233,6 @@ public void providerReceivesEachCallSeparately() throws Exception { assertEquals("k3", TestCometS3CredentialProvider.lastPath); } - /** - * The JVM shutdown hook calls {@link CometS3CredentialDispatcher#closeAll()} on every cached - * provider. We invoke it directly here since junit cannot exercise a real JVM shutdown. - */ @Test public void closeAllInvokesEveryProvider() { Map propsA = new HashMap<>(); @@ -256,17 +247,12 @@ public void closeAllInvokesEveryProvider() { assertEquals(2, TestCometS3CredentialProvider.closeCount.get()); - // After closeAll, handles are no longer registered. Re-init creates a fresh instance. long fresh = CometS3CredentialDispatcher.ensureInitialized(TEST_PROVIDER, "close-a", propsA); assertEquals(3, TestCometS3CredentialProvider.initCount.get()); assertNotNull(fresh); } - /** - * A failing vendor close() must not block other providers from being closed. The dispatcher - * swallows exceptions during shutdown so a slow or buggy provider cannot keep an executor JVM - * from exiting cleanly. - */ + /** A failing vendor close() must not block other providers from being closed. */ @Test public void closeAllSwallowsProviderExceptions() { CometS3CredentialDispatcher.ensureInitialized( diff --git a/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java b/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java index e5a14aa169..2b4f5e8c86 100644 --- a/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java +++ b/spark/src/test/java/org/apache/comet/cloud/s3/MinioCometS3CredentialProvider.java @@ -26,7 +26,8 @@ import java.util.concurrent.atomic.AtomicReference; /** - * Test-only {@link CometS3CredentialProvider} registered via {@code META-INF/services}. Returns + * Test-only {@link CometS3CredentialProvider} named via {@code + * spark.hadoop.fs.s3a.comet.credential.provider.class} (and the per-catalog Iceberg form). Returns * Minio's static credentials once {@link #installCredentials} has been called and counts every * invocation so suites can assert the bridge was actually used. */ From f08805e34d480d1df560db4a9dedbc4a21ed2477 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 21 May 2026 19:32:04 -0400 Subject: [PATCH 24/24] fix format. --- .../scala/org/apache/comet/iceberg/IcebergReflection.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala index b4e18da6af..2a39369c76 100644 --- a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala +++ b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala @@ -776,8 +776,10 @@ object CometIcebergNativeScanMetadata extends Logging { } } catch { case e: Exception => - logWarning(s"Iceberg reflection: Table.name() not callable on ${table.getClass.getName}; " + - s"native S3 credential dispatch will fall back to bucket-keyed isolation: ${e.getMessage}") + logWarning( + s"Iceberg reflection: Table.name() not callable on ${table.getClass.getName}. " + + "Native S3 credential dispatch will fall back to bucket-keyed isolation: " + + s"${e.getMessage}") None } }