HDFS-14037. Fix SSLFactory truststore reloader thread leak in URLConnectionFactory.

(cherry picked from commit 55fb3c32fb)
This commit is contained in:
Takanobu Asanuma 2019-03-27 03:27:02 +09:00
parent dbc02bcda7
commit 1fc74178eb
6 changed files with 139 additions and 35 deletions

View File

@ -0,0 +1,72 @@
/*
* 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.hdfs.web;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
import org.apache.hadoop.security.ssl.SSLFactory;
import javax.net.ssl.HostnameVerifier;
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLSocketFactory;
import java.io.IOException;
import java.net.HttpURLConnection;
import java.security.GeneralSecurityException;
/**
* Configure a connection to use SSL authentication.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class SSLConnectionConfigurator implements ConnectionConfigurator {
private final SSLFactory factory;
private final SSLSocketFactory sf;
private final HostnameVerifier hv;
private final int connectTimeout;
private final int readTimeout;
SSLConnectionConfigurator(int connectTimeout, int readTimeout,
Configuration conf) throws IOException, GeneralSecurityException {
factory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
factory.init();
sf = factory.createSSLSocketFactory();
hv = factory.getHostnameVerifier();
this.connectTimeout = connectTimeout;
this.readTimeout = readTimeout;
}
@Override
public HttpURLConnection configure(HttpURLConnection conn) {
if (conn instanceof HttpsURLConnection) {
HttpsURLConnection c = (HttpsURLConnection) conn;
c.setSSLSocketFactory(sf);
c.setHostnameVerifier(hv);
}
conn.setConnectTimeout(connectTimeout);
conn.setReadTimeout(readTimeout);
return conn;
}
void destroy() {
factory.destroy();
}
}

View File

@ -22,11 +22,6 @@ import java.io.IOException;
import java.net.HttpURLConnection;
import java.net.URL;
import java.net.URLConnection;
import java.security.GeneralSecurityException;
import javax.net.ssl.HostnameVerifier;
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLSocketFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -36,7 +31,6 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
import org.apache.hadoop.security.authentication.client.AuthenticationException;
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
import org.apache.hadoop.security.ssl.SSLFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -103,7 +97,7 @@ public class URLConnectionFactory {
final int connectTimeout, final int readTimeout, Configuration conf) {
ConnectionConfigurator conn;
try {
conn = newSslConnConfigurator(connectTimeout, readTimeout, conf);
conn = new SSLConnectionConfigurator(connectTimeout, readTimeout, conf);
} catch (Exception e) {
LOG.warn(
"Cannot load customized ssl related configuration. Fallback to" +
@ -139,7 +133,7 @@ public class URLConnectionFactory {
ConnectionConfigurator conn;
try {
ConnectionConfigurator sslConnConfigurator
= newSslConnConfigurator(connectTimeout, readTimeout, conf);
= new SSLConnectionConfigurator(connectTimeout, readTimeout, conf);
conn = new OAuth2ConnectionConfigurator(conf, sslConnConfigurator);
} catch (Exception e) {
@ -153,33 +147,6 @@ public class URLConnectionFactory {
this.connConfigurator = connConfigurator;
}
private static ConnectionConfigurator newSslConnConfigurator(
final int connectTimeout, final int readTimeout, Configuration conf)
throws IOException, GeneralSecurityException {
final SSLFactory factory;
final SSLSocketFactory sf;
final HostnameVerifier hv;
factory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
factory.init();
sf = factory.createSSLSocketFactory();
hv = factory.getHostnameVerifier();
return new ConnectionConfigurator() {
@Override
public HttpURLConnection configure(HttpURLConnection conn)
throws IOException {
if (conn instanceof HttpsURLConnection) {
HttpsURLConnection c = (HttpsURLConnection) conn;
c.setSSLSocketFactory(sf);
c.setHostnameVerifier(hv);
}
URLConnectionFactory.setTimeouts(conn, connectTimeout, readTimeout);
return conn;
}
};
}
/**
* Opens a url with read and connect timeouts
*
@ -242,4 +209,10 @@ public class URLConnectionFactory {
connection.setConnectTimeout(connectTimeout);
connection.setReadTimeout(readTimeout);
}
public void destroy() {
if (connConfigurator instanceof SSLConnectionConfigurator) {
((SSLConnectionConfigurator) connConfigurator).destroy();
}
}
}

View File

@ -1518,6 +1518,9 @@ public class WebHdfsFileSystem extends FileSystem
} catch (IOException ioe) {
LOG.debug("Token cancel failed: ", ioe);
} finally {
if (connectionFactory != null) {
connectionFactory.destroy();
}
super.close();
}
}

View File

@ -17,13 +17,17 @@
*/
package org.apache.hadoop.hdfs.web;
import java.io.File;
import java.io.IOException;
import java.net.HttpURLConnection;
import java.net.URL;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.apache.hadoop.security.ssl.SSLFactory;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Assert;
@ -64,4 +68,53 @@ public final class TestURLConnectionFactory {
logs.getOutput().contains(
"Cannot load customized ssl related configuration"));
}
@Test
public void testSSLFactoryCleanup() throws Exception {
String baseDir = GenericTestUtils.getTempPath(
TestURLConnectionFactory.class.getSimpleName());
File base = new File(baseDir);
FileUtil.fullyDelete(base);
base.mkdirs();
String keystoreDir = new File(baseDir).getAbsolutePath();
String sslConfDir = KeyStoreTestUtil.getClasspathDir(
TestURLConnectionFactory.class);
Configuration conf = new Configuration();
KeyStoreTestUtil.setupSSLConfig(keystoreDir, sslConfDir, conf, false,
true);
Configuration sslConf = KeyStoreTestUtil.getSslConfig();
sslConf.set("fs.defaultFS", "swebhdfs://localhost");
FileSystem fs = FileSystem.get(sslConf);
ThreadGroup threadGroup = Thread.currentThread().getThreadGroup();
while (threadGroup.getParent() != null) {
threadGroup = threadGroup.getParent();
}
Thread[] threads = new Thread[threadGroup.activeCount()];
threadGroup.enumerate(threads);
Thread reloaderThread = null;
for (Thread thread : threads) {
if ((thread.getName() != null)
&& (thread.getName().contains("Truststore reloader thread"))) {
reloaderThread = thread;
}
}
Assert.assertTrue("Reloader is not alive", reloaderThread.isAlive());
fs.close();
boolean reloaderStillAlive = true;
for (int i = 0; i < 10; i++) {
reloaderStillAlive = reloaderThread.isAlive();
if (!reloaderStillAlive) {
break;
}
Thread.sleep(1000);
}
Assert.assertFalse("Reloader is still alive", reloaderStillAlive);
}
}

View File

@ -472,11 +472,13 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
(HttpURLConnection)connectionFactory.openConnection(url);
conn.setRequestMethod(method);
connectionFactory.destroy();
return conn;
} catch (Exception e) {
LOG.error("Cannot redirect request to {}", namenode, e);
}
}
connectionFactory.destroy();
return null;
}

View File

@ -467,6 +467,7 @@ public class QuorumJournalManager implements JournalManager {
@Override
public void close() throws IOException {
loggers.close();
connectionFactory.destroy();
}
public void selectInputStreams(Collection<EditLogInputStream> streams,