|
| 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