HADOOP-18993. Add option fs.s3a.classloader.isolation (#6301)

The option fs.s3a.classloader.isolation (default: true) can be set to false to disable s3a classloader isolation;

This can assist in using custom credential providers and other extension points.

Contributed by Antonio Murgia
This commit is contained in:
Antonio Murgia 2024-02-05 18:59:36 +01:00 committed by GitHub
parent 4f0f5a546c
commit b11159d799
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 198 additions and 2 deletions

View File

@ -1607,4 +1607,21 @@ private Constants() {
*/
public static final boolean CHECKSUM_VALIDATION_DEFAULT = false;
/**
* Are extensions classes, such as {@code fs.s3a.aws.credentials.provider},
* going to be loaded from the same classloader that loaded
* the {@link S3AFileSystem}?
* It is useful to turn classloader isolation off for Apache Spark applications
* that might load {@link S3AFileSystem} from the Spark distribution (Launcher classloader)
* while users might want to provide custom extensions (loaded by Spark MutableClassloader).
* Value: {@value}.
*/
public static final String AWS_S3_CLASSLOADER_ISOLATION =
"fs.s3a.classloader.isolation";
/**
* Default value for {@link #AWS_S3_CLASSLOADER_ISOLATION}.
* Value: {@value}.
*/
public static final boolean DEFAULT_AWS_S3_CLASSLOADER_ISOLATION = true;
}

View File

@ -559,8 +559,8 @@ public void initialize(URI name, Configuration originalConf)
// fix up the classloader of the configuration to be whatever
// classloader loaded this filesystem.
// See: HADOOP-17372
conf.setClassLoader(this.getClass().getClassLoader());
// See: HADOOP-17372 and follow-up on HADOOP-18993
S3AUtils.maybeIsolateClassloader(conf, this.getClass().getClassLoader());
// patch the Hadoop security providers
patchSecurityCredentialProviders(conf);

View File

@ -1693,4 +1693,25 @@ public static Map<String, String> getTrimmedStringCollectionSplitByEquals(
.getTrimmedStringCollectionSplitByEquals(valueString);
}
/**
* If classloader isolation is {@code true}
* (through {@link Constants#AWS_S3_CLASSLOADER_ISOLATION}) or not
* explicitly set, then the classLoader of the input configuration object
* will be set to the input classloader, otherwise nothing will happen.
* @param conf configuration object.
* @param classLoader isolated classLoader.
*/
static void maybeIsolateClassloader(Configuration conf, ClassLoader classLoader) {
if (conf.getBoolean(Constants.AWS_S3_CLASSLOADER_ISOLATION,
Constants.DEFAULT_AWS_S3_CLASSLOADER_ISOLATION)) {
LOG.debug("Configuration classloader set to S3AFileSystem classloader: {}", classLoader);
conf.setClassLoader(classLoader);
} else {
LOG.debug("Configuration classloader not changed, support classes needed will be loaded " +
"from the classloader that instantiated the Configuration object: {}",
conf.getClassLoader());
}
}
}

View File

@ -590,6 +590,30 @@ obtain the credentials needed to access AWS services in the role the EC2 VM
was deployed as.
This AWS credential provider is enabled in S3A by default.
## Custom AWS Credential Providers and Apache Spark
Apache Spark employs two class loaders, one that loads "distribution" (Spark + Hadoop) classes and one that
loads custom user classes. If the user wants to load custom implementations of AWS credential providers,
custom signers, delegation token providers or any other dynamically loaded extension class
through user provided jars she will need to set the following configuration:
```xml
<property>
<name>fs.s3a.classloader.isolation</name>
<value>false</value>
</property>
<property>
<name>fs.s3a.aws.credentials.provider</name>
<value>CustomCredentialsProvider</value>
</property>
```
If the following property is not set or set to `true`, the following exception will be thrown:
```
java.io.IOException: From option fs.s3a.aws.credentials.provider java.lang.ClassNotFoundException: Class CustomCredentialsProvider not found
```
## <a name="hadoop_credential_providers"></a>Storing secrets with Hadoop Credential Providers

View File

@ -0,0 +1,134 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Consumer;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
/**
* Checks that classloader isolation for loading extension classes is applied
* correctly. Both default, true and false tests performed.
* See {@link Constants#AWS_S3_CLASSLOADER_ISOLATION} property and
* HADOOP-17372 and follow-up on HADOOP-18993 for more info.
*/
public class ITestS3AFileSystemIsolatedClassloader extends AbstractS3ATestBase {
private static class CustomClassLoader extends ClassLoader {
}
private final ClassLoader customClassLoader = new CustomClassLoader();
private S3AFileSystem createNewTestFs(Configuration conf) throws IOException {
S3AFileSystem fs = new S3AFileSystem();
fs.initialize(getFileSystem().getUri(), conf);
return fs;
}
/**
* Asserts that the given assertions are valid in a new filesystem context.
* The filesystem is created after setting the context classloader to
* {@link ITestS3AFileSystemIsolatedClassloader#customClassLoader} in this way we are
* able to check if the classloader is reset during the initialization or not.
*
* @param confToSet The configuration settings to be applied to the new filesystem.
* @param asserts The assertions to be performed on the new filesystem.
* @throws IOException If an I/O error occurs.
*/
private void assertInNewFilesystem(Map<String, String> confToSet, Consumer<FileSystem> asserts)
throws IOException {
ClassLoader previousClassloader = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(customClassLoader);
Configuration conf = new Configuration();
Assertions.assertThat(conf.getClassLoader()).isEqualTo(customClassLoader);
S3ATestUtils.prepareTestConfiguration(conf);
for (Map.Entry<String, String> e : confToSet.entrySet()) {
conf.set(e.getKey(), e.getValue());
}
try (S3AFileSystem fs = createNewTestFs(conf)) {
asserts.accept(fs);
}
} finally {
Thread.currentThread().setContextClassLoader(previousClassloader);
}
}
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;
}
@Test
public void defaultIsolatedClassloader() throws IOException {
assertInNewFilesystem(mapOf(), (fs) -> {
Assertions.assertThat(fs.getConf().getClassLoader())
.describedAs("The classloader used to load s3a fs extensions")
.isNotEqualTo(Thread.currentThread().getContextClassLoader())
.describedAs("the current classloader");
Assertions.assertThat(fs.getConf().getClassLoader())
.describedAs("The classloader used to load s3a fs extensions")
.isEqualTo(fs.getClass().getClassLoader())
.describedAs("the classloader that loaded the fs");
});
}
@Test
public void isolatedClassloader() throws IOException {
assertInNewFilesystem(mapOf(Constants.AWS_S3_CLASSLOADER_ISOLATION, "true"), (fs) -> {
Assertions.assertThat(fs.getConf().getClassLoader())
.describedAs("The classloader used to load s3a fs extensions")
.isNotEqualTo(Thread.currentThread().getContextClassLoader())
.describedAs("the current context classloader");
Assertions.assertThat(fs.getConf().getClassLoader())
.describedAs("The classloader used to load s3a fs extensions")
.isEqualTo(fs.getClass().getClassLoader())
.describedAs("the classloader that loaded the fs");
});
}
@Test
public void notIsolatedClassloader() throws IOException {
assertInNewFilesystem(mapOf(Constants.AWS_S3_CLASSLOADER_ISOLATION, "false"), (fs) -> {
Assertions.assertThat(fs.getConf().getClassLoader())
.describedAs("The classloader used to load s3a fs extensions")
.isEqualTo(Thread.currentThread().getContextClassLoader())
.describedAs("the current context classloader");
Assertions.assertThat(fs.getConf().getClassLoader())
.describedAs("The classloader used to load s3a fs extensions")
.isNotEqualTo(fs.getClass().getClassLoader())
.describedAs("the classloader that loaded the fs");
});
}
}