Samrat002 commented on code in PR #27187:
URL: https://github.com/apache/flink/pull/27187#discussion_r2876568482
##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/S3ClientProvider.java:
##########
@@ -372,14 +382,129 @@ public S3ClientProvider build() {
}
private AwsCredentialsProvider buildBaseCredentialsProvider() {
- // The token provider returns null if credentials are not
available,
- // allowing the chain to proceed to the next provider (fallback).
+ if (credentialsProviderClasses != null
+ && !credentialsProviderClasses.trim().isEmpty()) {
+ return
buildCustomCredentialsProvider(credentialsProviderClasses);
+ }
+ // Default chain: delegation tokens -> static credentials (if
configured)
+ // -> DefaultCredentialsProvider.
return AwsCredentialsProviderChain.builder()
.credentialsProviders(
new DynamicTemporaryAWSCredentialsProvider(),
buildFallbackProvider())
.build();
}
+ private AwsCredentialsProvider buildCustomCredentialsProvider(String
classNames) {
+ String[] names = classNames.split(",");
+ List<AwsCredentialsProvider> providers = new ArrayList<>();
+ for (String name : names) {
+ String trimmed = name.trim();
+ if (trimmed.isEmpty()) {
+ continue;
+ }
+ providers.add(instantiateCredentialsProvider(trimmed));
+ }
+ if (providers.isEmpty()) {
+ throw new IllegalArgumentException(
+ "fs.s3.aws.credentials.provider is set but contains no
valid provider class names");
+ }
+ LOG.info(
+ "Using custom credentials provider chain with {}
provider(s): {}",
+ providers.size(),
+ classNames);
+ if (providers.size() == 1) {
+ return providers.get(0);
+ }
+ return
AwsCredentialsProviderChain.builder().credentialsProviders(providers).build();
+ }
+
+ /**
+ * Instantiates an {@link AwsCredentialsProvider} from a class name.
+ *
+ * <p>Supports:
+ *
+ * <ul>
+ * <li>Fully-qualified AWS SDK v2 class names (e.g. {@code
+ *
software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider})
+ * <li>Simple class names resolved from the {@code
+ * software.amazon.awssdk.auth.credentials} package (e.g. {@code
+ * AnonymousCredentialsProvider})
+ * <li>Well-known Hadoop / AWS SDK v1 names mapped to their SDK v2
equivalents
+ * </ul>
+ */
+ private AwsCredentialsProvider instantiateCredentialsProvider(String
className) {
+ String resolvedClassName = resolveProviderClassName(className);
+ try {
+ Class<?> clazz = Class.forName(resolvedClassName);
+ if (!AwsCredentialsProvider.class.isAssignableFrom(clazz)) {
+ throw new IllegalArgumentException(
+ "Class "
+ + resolvedClassName
+ + " does not implement
AwsCredentialsProvider");
+ }
+
+ // SDK v2 providers use a static create() factory method by
convention
+ try {
+ java.lang.reflect.Method createMethod =
clazz.getMethod("create");
+ if
(java.lang.reflect.Modifier.isStatic(createMethod.getModifiers())
+ && AwsCredentialsProvider.class.isAssignableFrom(
+ createMethod.getReturnType())) {
+ return (AwsCredentialsProvider)
createMethod.invoke(null);
+ }
+ } catch (NoSuchMethodException ignored) {
+ }
+
+ return (AwsCredentialsProvider)
clazz.getDeclaredConstructor().newInstance();
+ } catch (ClassNotFoundException e) {
+ throw new IllegalArgumentException(
+ "Credentials provider class not found: "
+ + resolvedClassName
+ + " (original: "
+ + className
+ + ")",
+ e);
+ } catch (IllegalArgumentException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new IllegalArgumentException(
+ "Failed to instantiate credentials provider: " +
resolvedClassName, e);
+ }
+ }
+
+ private static final java.util.Map<String, String>
HADOOP_TO_SDK_V2_PROVIDERS =
+ java.util.Map.of(
+ "com.amazonaws.auth.AnonymousAWSCredentialsProvider",
+
"software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider",
+
"org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider",
+
"software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider",
+
"com.amazonaws.auth.EnvironmentVariableCredentialsProvider",
+
"software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider",
+
"com.amazonaws.auth.InstanceProfileCredentialsProvider",
+
"software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider",
+
"com.amazonaws.auth.profile.ProfileCredentialsProvider",
+
"software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider",
+ "com.amazonaws.auth.ContainerCredentialsProvider",
+
"software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider");
Review Comment:
i am not sure about providing this support . Open for thoughts and
suggestion
@gaborgsomogyi @pnowojski @Izeren
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]