[ 
https://issues.apache.org/jira/browse/HADOOP-19733?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18032474#comment-18032474
 ] 

ASF GitHub Bot commented on HADOOP-19733:
-----------------------------------------

steveloughran commented on code in PR #8048:
URL: https://github.com/apache/hadoop/pull/8048#discussion_r2455036391


##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java:
##########
@@ -37,10 +46,33 @@
  */
 public class ITestS3AFileSystemIsolatedClassloader extends AbstractS3ATestBase 
{
 
+  private static String customClassName = "custom.class.name";
+
+  private static class CustomCredentialsProvider implements 
AwsCredentialsProvider {
+
+      public CustomCredentialsProvider() {
+      }
+
+      @Override
+      public AwsCredentials resolveCredentials() {
+          return null;
+      }
+
+  }
+
   private static class CustomClassLoader extends ClassLoader {
   }
 
-  private final ClassLoader customClassLoader = new CustomClassLoader();
+  private final ClassLoader customClassLoader = spy(new CustomClassLoader());
+  {
+    try {

Review Comment:
   this is a nice way to simulate classloader pain.
   



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java:
##########
@@ -28,6 +29,14 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.impl.InstantiationIOException;
+
+import software.amazon.awssdk.auth.credentials.AwsCredentials;

Review Comment:
   nit: put the amazon imports in the same group as the junit ones



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java:
##########
@@ -100,11 +122,26 @@ public void defaultIsolatedClassloader() throws 
IOException {
               .isEqualTo(fs.getClass().getClassLoader())
               .describedAs("the classloader that loaded the fs");
     });
+
+    Throwable thrown = Assertions.catchThrowable(() -> {

Review Comment:
   Use our `LambdaTestUtils.intercept()`; it's like the spark one and does the 
casting checks
   
   ```
   InstantiationIOException ex = intercept(InstantiationIOException.class, () 
-> { assert...})
   ```
   we have a `assertExceptionContains` to look at the inner stuff, but the 
assert of L136 is fine.
   
   



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java:
##########
@@ -115,11 +152,31 @@ public void isolatedClassloader() throws IOException {
               .isEqualTo(fs.getClass().getClassLoader())
               .describedAs("the classloader that loaded the fs");
     });
+
+    Throwable thrown = Assertions.catchThrowable(() -> {

Review Comment:
   again `intercept()` and cut the assert at L163



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java:
##########
@@ -77,19 +109,9 @@ private void assertInNewFilesystem(Map<String, String> 
confToSet, Consumer<FileS
     }
   }
 
-  private Map<String, String> mapOf() {
-    return new HashMap<>();
-  }
-
-  private Map<String, String> mapOf(String key, String value) {
-    HashMap<String, String> m = new HashMap<>();
-    m.put(key, value);
-    return m;
-  }

Review Comment:
   It's because we only switched to java17 yesterday! And in trunk only.
   
   If you want to see this change in Hadoop 3.4.3 it'll still need to be java8 
code, so this needs to be restored. Otherwise: trunk/3.5.0 only





> S3A: Credentials provider classes not found despite setting 
> `fs.s3a.classloader.isolation` to `false`
> -----------------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-19733
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19733
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>    Affects Versions: 3.4.2
>            Reporter: Brandon
>            Priority: Minor
>              Labels: pull-request-available
>
> HADOOP-18993 added the option `fs.s3a.classloader.isolation` to support, for 
> example, a Spark job using an AWS credentials provider class that is bundled 
> into the Spark job JAR. In testing this, the AWS credentials provider classes 
> are still not found.
> I think the cause is:
>  * `fs.s3a.classloader.isolation` is implemented by setting (or not setting) 
> a classloader on the `Configuration`
>  * However, code paths to load AWS credential provider call 
> `S3AUtils.getInstanceFromReflection`, which uses the classloader that loaded 
> the S3AUtils class. That's likely to be the built-in application classloader, 
> which won't be able to load classes in a Spark job JAR.
> And the fix seems small:
>  * Change `S3AUtils.getInstanceFromReflection` to load classes using the 
> `Configuration`'s classloader. Luckily we already have the Configuration in 
> this method.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to