HADOOP-19733. S3A: load credentials providers using configured classloader#8048
Conversation
…lassloader Follow-up to HADOOP-17372 and HADOOP-18993. The issue described in HADOOP-18993 still existed, because the code path to load credentials providers goes through `S3AUtils#getInstanceFromReflection` and always uses the classloader that loaded the `S3AUtils` class. With this change, credentials providers are loaded using the Configuration's classloader so they respect the user's `fs.s3a.classloader.isolation` setting
...hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java
Show resolved
Hide resolved
This comment was marked as resolved.
This comment was marked as resolved.
This comment was marked as resolved.
This comment was marked as resolved.
steveloughran
left a comment
There was a problem hiding this comment.
looks good and it's nice to have that spark test done.
Here's the full policy on testing though:
https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/testing.html#Policy_for_submitting_patches_which_affect_the_hadoop-aws_module.
you are going to have to run all the integration test suites against an s3 store and say which one (S3 london) as well as the maven parameters for the hadoop-aws module, such as -Dparallel-tests -DtestsThreadCount=8 -Panalytics
regarding mapof() versus Map.of(); it's best to leave as is for backporting
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; | ||
|
|
||
| import software.amazon.awssdk.auth.credentials.AwsCredentials; |
There was a problem hiding this comment.
nit: put the amazon imports in the same group as the junit ones
...hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java
Show resolved
Hide resolved
...hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java
Outdated
Show resolved
Hide resolved
...hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemIsolatedClassloader.java
Outdated
Show resolved
Hide resolved
| private final ClassLoader customClassLoader = new CustomClassLoader(); | ||
| private final ClassLoader customClassLoader = spy(new CustomClassLoader()); | ||
| { | ||
| try { |
There was a problem hiding this comment.
this is a nice way to simulate classloader pain.
|
what is the status of this...got lost in the pool of open jiras/prs |
|
@steveloughran, There's a bit more implementation work to ensure the custom signer loading uses the right classloader. I've started that work, but not quite finished. I've also been trying to run the S3A test suite and have had trouble with that. It's pretty close. I'll try to find some time to work on this soon. |
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java
Outdated
Show resolved
Hide resolved
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
@steveloughran I did another pass over this. I tested using S3 in us-west-2. |
|
🎊 +1 overall
This message was automatically generated. |
|
ok, you've got yetus nice and quiet, and done the cloud storage tests, just a couple of checkstyles remaining +1 pending you addressing those nits |
|
💔 -1 overall
This message was automatically generated. |
|
@steveloughran thanks for looking! I've resolved the checkstyle issues. Yetus says this PR can't be applied cleanly to |
…oader (#8048) Follow-up to HADOOP-17372 and HADOOP-18993. Contributed by Brandon Vincent
Description of PR
HADOOP-19733: With
fs.s3a.classloader.isolationset tofalsein a Spark application, it was still impossible to load a credentials provider class from the Spark application jar.fs.s3a.classloader.isolationworks by saving a reference to the intended classloader in theConfiguration.However, loading credentials providers goes through
S3AUtils#getInstanceFromReflection, which always used the classloader that loadedS3AUtils. It should useConfiguration's classloader.How was this patch tested?
Unit tests in
org.apache.hadoop.fs.s3a.ITestS3AFileSystemIsolatedClassloader.Manual testing in a Spark application.
For code changes:
LICENSE,LICENSE-binary,NOTICE-binaryfiles?