Skip to content

feat: vendor-pluggable S3 credentials for native scans#4309

Open
mbutrovich wants to merge 34 commits into
apache:mainfrom
mbutrovich:credential_provider
Open

feat: vendor-pluggable S3 credentials for native scans#4309
mbutrovich wants to merge 34 commits into
apache:mainfrom
mbutrovich:credential_provider

Conversation

@mbutrovich
Copy link
Copy Markdown
Contributor

@mbutrovich mbutrovich commented May 13, 2026

Which issue does this PR close?

Closes #4332.

Rationale for this change

Comet's native scan paths (object_store for raw Parquet, opendal via iceberg-rust for Iceberg) bypass Spark's Hadoop S3A credential infrastructure. Vendors with per-path STS, REST-vended creds, or other custom mechanisms cannot reach Comet through any existing SPI. AWSCredentialsProvider.getCredentials() is parameterless, Hadoop S3A custom signers never return credentials outside the signing pipeline, and Spark's CloudCredentialsProvider yields one JWT per service name with no path argument.

This PR adds a narrow, S3-specific SPI plus JNI plumbing to call it from native code. Activation is config-driven and modeled on parquet.crypto.factory.class (PME KMS, #2447). The user names one vendor class in a Spark or Hadoop config and the vendor dispatches across backends inside it.

Design rationale (keying, lifecycle, returns-or-throws, no Comet-side cache, property-bag handling, error-fidelity caveats) lives in the contributor guide page s3-credential-provider-design.md. Operator setup and vendor contract live in the user guide page s3-credential-providers.md.

What is in this PR

  • Java SPI under org.apache.comet.cloud.s3 (in the spark module, since refactor: Move most of comet-common module into comet-spark #4325 collapsed common to a minimal bootstrap): CometS3CredentialProvider (AutoCloseable, default initialize(Map)), CometS3Credentials, CometS3AccessMode, CometS3CredentialContext, and CometS3CredentialDispatcher keyed by (FQCN, dispatchKey, catalogProperties) with ensureInitialized(...) returning a long handle, hot-path getCredentialsForPath(handle, ...), and a JVM shutdown hook that closes every cached provider.
  • Shared org.apache.comet.util.ClassLoaders.loadClass prefers the thread context ClassLoader. Both the dispatcher and IcebergReflection.loadClass delegate to it.
  • Rust CometS3CredentialBridge (under native/core/src/cloud/s3/) implementing object_store::CredentialProvider and reqsign_core::ProvideCredential, plus a JNI handle in native/jni-bridge.
  • Activation keys: fs.s3a.comet.credential.provider.class (with per-bucket override) for Parquet, and s3.comet.credential.provider.class on the Spark catalog property for Iceberg. dispatchKey is the bucket on the Parquet path and the V2 catalog name on the Iceberg path.
  • The unfiltered FileIO property bag crosses JNI as catalog_properties. The storage-prefix filter (s3., gcs., adls., client.) moves native-side to iceberg_scan.rs::load_file_io.
  • IcebergScanExec gets a manual redacting Debug so plan dumps do not leak the property bag.
  • iceberg-rust pin bumped to 83b4595 (for reqsign-core 3.0 and CustomAwsCredentialLoader). testcontainers bumped to 1.21.4 and docker-java to 3.7.1 for modern Docker daemons.
  • Reference IcebergRESTVendedS3Provider (test scope, Spark 4.x build only) wrapping Iceberg's VendedCredentialsProvider. Test scope keeps iceberg-aws and AWS SDK v2 off Comet's runtime classpath.
  • New user guide and contributor guide pages (linked above).

How are these changes tested?

  • JUnit CometS3CredentialDispatcherTest: handle round-trip, ensureInitialized idempotence, distinct dispatchKey and catalogProperties isolation, closeAll swallows provider exceptions, missing-class / wrong-interface / no-arg-ctor / empty-FQCN failure modes, get-without-init guard.
  • JUnit IcebergRESTVendedS3ProviderTest (Spark 4.x).
  • End-to-end CometS3CredentialBridgeSuite (Minio): Parquet on S3, Iceberg on S3, REST plus SPI integration with a sentinel non-storage-prefix key reaching initialize(Map), multi-catalog isolation across two catalogs sharing one FQCN. Added to dev/ci/check-suites.py ignore list (manual, like other Docker-dependent S3 suites).
  • Confirmed end-to-end with a downstream custom credential provider.

@mbutrovich mbutrovich changed the title Credential provider feat: Spark custom credential providers for native scans May 13, 2026
@mbutrovich mbutrovich force-pushed the credential_provider branch from b549155 to 0cd8a36 Compare May 14, 2026 14:33
@mbutrovich mbutrovich moved this from Todo to In progress in Comet Development May 14, 2026
Comment thread common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java Outdated
@mbutrovich mbutrovich changed the title feat: Spark custom credential providers for native scans feat: vendor-pluggable S3 credentials for Comet native scans May 14, 2026
@mbutrovich mbutrovich changed the title feat: vendor-pluggable S3 credentials for Comet native scans feat: vendor-pluggable S3 credentials for native scans May 14, 2026
@mbutrovich mbutrovich marked this pull request as ready for review May 14, 2026 16:18
… 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.
@karuppayya
Copy link
Copy Markdown
Contributor

karuppayya commented May 18, 2026

edit:
deleted this comment since i posted in incorrect PR

@mbutrovich
Copy link
Copy Markdown
Contributor Author

CC @snmvaughan

Copy link
Copy Markdown
Contributor

@karuppayya karuppayya left a comment

Choose a reason for hiding this comment

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

Left some comments. Will do another pass later today

Comment thread spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java Outdated
Comment thread spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java Outdated
Comment thread spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialProvider.java Outdated
Comment thread spark/src/main/java/org/apache/comet/cloud/s3/CometS3CredentialDispatcher.java Outdated
Comment thread native/core/src/cloud/s3/credential_bridge.rs
InstanceKey key = new InstanceKey(providerClassName, dispatchKey == null ? "" : dispatchKey);
Map<String, String> props =
catalogProperties == null ? Collections.emptyMap() : catalogProperties;
INSTANCES.computeIfAbsent(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

A vendor whose initialize throws gets re-attempted on every get_credential call from object_store. Should we cache error per key and backoff. May be a followup

@mbutrovich
Copy link
Copy Markdown
Contributor Author

mbutrovich commented May 21, 2026

Thanks for the feedback @karuppayya! I think I addressed everything but:

A vendor whose initialize throws gets re-attempted on every get_credential call from object_store. Should we cache error per key and backoff. May be a followup

I will update my internal credential provider to align with these SPI changes and test again.

@mbutrovich
Copy link
Copy Markdown
Contributor Author

Updated my internal implementation to match the latest SPI changes, and things are working well!

@mbutrovich mbutrovich requested a review from karuppayya May 21, 2026 23:16
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

Status: In progress

Development

Successfully merging this pull request may close these issues.

Credential Provider Support

3 participants