Skip to content

Commit d9596de

Browse files
committed
cleanup to get ready for review
1 parent 0cd8a36 commit d9596de

26 files changed

Lines changed: 694 additions & 1125 deletions

common/src/main/java/org/apache/comet/cloud/CometCloudCredentialDispatcher.java

Lines changed: 0 additions & 176 deletions
This file was deleted.

common/src/main/java/org/apache/comet/cloud/CometCloudCredentialProvider.java

Lines changed: 0 additions & 56 deletions
This file was deleted.

common/src/main/java/org/apache/comet/cloud/CometAccessMode.java renamed to common/src/main/java/org/apache/comet/cloud/s3/CometS3AccessMode.java

Lines changed: 5 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -17,20 +17,12 @@
1717
* under the License.
1818
*/
1919

20-
package org.apache.comet.cloud;
20+
package org.apache.comet.cloud.s3;
2121

22-
/**
23-
* Access intent for a credential request issued by Comet's native code, passed to {@link
24-
* CometCloudCredentialProvider#getCredentialsForPath}.
25-
*
26-
* <p>Granularity is intentionally binary. Vendors that issue WRITE-scoped credentials are expected
27-
* to include READ permissions when their workflows require it (multipart-completion HEAD, Iceberg
28-
* manifest reads on the write path, etc.) — the SPI does not promise that a WRITE credential is
29-
* also read-capable; the vendor's IAM policy does.
30-
*/
31-
public enum CometAccessMode {
32-
/** GET / HEAD / LIST and equivalent. All Comet native scan paths request this today. */
22+
/** Access intent passed to {@link CometS3CredentialProvider#getCredentialsForPath}. */
23+
public enum CometS3AccessMode {
24+
/** GET / HEAD / LIST. All Comet native scan paths request this today. */
3325
READ,
34-
/** PUT / POST / DELETE / multipart and equivalent. Reserved for future native write paths. */
26+
/** PUT / POST / DELETE / multipart. Reserved for future native write paths. */
3527
WRITE
3628
}
Lines changed: 90 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,90 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
20+
package org.apache.comet.cloud.s3;
21+
22+
import java.util.ArrayList;
23+
import java.util.List;
24+
import java.util.ServiceLoader;
25+
import java.util.stream.Collectors;
26+
27+
import org.slf4j.Logger;
28+
import org.slf4j.LoggerFactory;
29+
30+
/**
31+
* JNI entry point invoked from native code to resolve {@link CometS3CredentialProvider}.
32+
*
33+
* <p>The provider is resolved once via {@link ServiceLoader} and cached in a {@code static final}
34+
* field. A query falling back from Comet to Spark mid-execution therefore sees identical
35+
* credentials, since both paths resolve from the same executor classpath.
36+
*
37+
* <p>Multiple registered impls fail fast at class-load; chaining is a vendor-side concern.
38+
*/
39+
public final class CometS3CredentialDispatcher {
40+
41+
private static final Logger LOG = LoggerFactory.getLogger(CometS3CredentialDispatcher.class);
42+
43+
private static final CometS3CredentialProvider PROVIDER = resolve();
44+
private static final CometS3AccessMode[] MODES = CometS3AccessMode.values();
45+
46+
private CometS3CredentialDispatcher() {}
47+
48+
public static boolean isProviderRegistered() {
49+
return PROVIDER != null;
50+
}
51+
52+
/** Invoked by native code. {@code mode} is the {@link CometS3AccessMode} ordinal. */
53+
public static CometS3Credentials getCredentialsForPath(String bucket, String path, int mode)
54+
throws Exception {
55+
if (PROVIDER == null) {
56+
throw new IllegalStateException(
57+
"No CometS3CredentialProvider registered; check META-INF/services on the classpath");
58+
}
59+
if (mode < 0 || mode >= MODES.length) {
60+
throw new IllegalArgumentException("Invalid CometS3AccessMode ordinal: " + mode);
61+
}
62+
CometS3AccessMode accessMode = MODES[mode];
63+
if (LOG.isDebugEnabled()) {
64+
LOG.debug("Fetching credentials for bucket={} path={} mode={}", bucket, path, accessMode);
65+
}
66+
return PROVIDER.getCredentialsForPath(bucket, path, accessMode);
67+
}
68+
69+
private static CometS3CredentialProvider resolve() {
70+
List<CometS3CredentialProvider> impls = new ArrayList<>();
71+
for (CometS3CredentialProvider impl : ServiceLoader.load(CometS3CredentialProvider.class)) {
72+
impls.add(impl);
73+
}
74+
if (impls.isEmpty()) {
75+
LOG.info(
76+
"No CometS3CredentialProvider registered; native S3 readers will use the default "
77+
+ "AWS credential chain");
78+
return null;
79+
}
80+
if (impls.size() > 1) {
81+
List<String> names =
82+
impls.stream().map(p -> p.getClass().getName()).collect(Collectors.toList());
83+
throw new IllegalStateException(
84+
"Multiple CometS3CredentialProvider impls on classpath: " + names);
85+
}
86+
CometS3CredentialProvider provider = impls.get(0);
87+
LOG.info("Registered CometS3CredentialProvider: {}", provider.getClass().getName());
88+
return provider;
89+
}
90+
}

0 commit comments

Comments
 (0)