HADOOP-16972. Ignore AuthenticationFilterInitializer for KMSWebServer. (#1961)

This commit is contained in:
Masatake Iwasaki 2020-04-18 06:38:25 +09:00 committed by GitHub
parent 3601054822
commit ac40daece1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 62 additions and 0 deletions

View File

@ -22,12 +22,16 @@ import java.net.InetSocketAddress;
import java.net.MalformedURLException;
import java.net.URI;
import java.net.URL;
import java.util.LinkedHashSet;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.http.HttpServer2;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.source.JvmMetrics;
import org.apache.hadoop.security.AuthenticationFilterInitializer;
import org.apache.hadoop.security.authentication.server.ProxyUserAuthenticationFilterInitializer;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.ssl.SSLFactory;
import org.apache.hadoop.util.JvmPauseMonitor;
@ -94,6 +98,22 @@ public class KMSWebServer {
KMSConfiguration.HTTP_PORT_DEFAULT);
URI endpoint = new URI(scheme, null, host, port, null, null, null);
String configuredInitializers =
conf.get(HttpServer2.FILTER_INITIALIZER_PROPERTY);
if (configuredInitializers != null) {
Set<String> target = new LinkedHashSet<String>();
String[] initializers = configuredInitializers.split(",");
for (String init : initializers) {
if (!init.equals(AuthenticationFilterInitializer.class.getName()) &&
!init.equals(
ProxyUserAuthenticationFilterInitializer.class.getName())) {
target.add(init);
}
}
String actualInitializers = StringUtils.join(",", target);
conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY, actualInitializers);
}
httpServer = new HttpServer2.Builder()
.setName(NAME)
.setConf(conf)

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.AuthenticationFilterInitializer;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
@ -3079,4 +3080,45 @@ public class TestKMS {
}
});
}
@Test
public void testFilterInitializer() throws Exception {
Configuration conf = new Configuration();
File testDir = getTestDir();
conf = createBaseKMSConf(testDir, conf);
conf.set("hadoop.security.authentication", "kerberos");
conf.set("hadoop.kms.authentication.token.validity", "1");
conf.set("hadoop.kms.authentication.type", "kerberos");
conf.set("hadoop.kms.authentication.kerberos.keytab",
keytab.getAbsolutePath());
conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
conf.set("hadoop.http.filter.initializers",
AuthenticationFilterInitializer.class.getName());
conf.set("hadoop.http.authentication.type", "kerberos");
conf.set("hadoop.http.authentication.kerberos.principal", "HTTP/localhost");
conf.set("hadoop.http.authentication.kerberos.keytab",
keytab.getAbsolutePath());
writeConf(testDir, conf);
runServer(null, null, testDir, new KMSCallable<Void>() {
@Override
public Void call() throws Exception {
final Configuration conf = new Configuration();
URL url = getKMSUrl();
final URI uri = createKMSUri(getKMSUrl());
doAs("client", new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
final KeyProvider kp = createProvider(uri, conf);
Assert.assertTrue(kp.getKeys().isEmpty());
return null;
}
});
return null;
}
});
}
}