HADOOP-10695. KMSClientProvider should respect a configurable timeout. (yoderme via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1606930 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alejandro Abdelnur 2014-06-30 20:54:56 +00:00
parent 0ca41a8f35
commit 950ae82571
3 changed files with 93 additions and 2 deletions

View File

@ -165,6 +165,9 @@ Trunk (Unreleased)
HADOOP-10696. Add optional attributes to KeyProvider Options and Metadata.
(tucu)
HADOOP-10695. KMSClientProvider should respect a configurable timeout.
(yoderme via tucu)
BUG FIXES
HADOOP-9451. Fault single-layer config if node group topology is enabled.

View File

@ -17,7 +17,6 @@
*/
package org.apache.hadoop.crypto.key.kms;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -27,6 +26,7 @@
import org.apache.hadoop.security.ProviderUtils;
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.authentication.client.PseudoAuthenticator;
import org.apache.hadoop.security.ssl.SSLFactory;
import org.apache.http.client.utils.URIBuilder;
@ -71,6 +71,13 @@ public class KMSClientProvider extends KeyProvider {
private static final String HTTP_PUT = "PUT";
private static final String HTTP_DELETE = "DELETE";
private static final String CONFIG_PREFIX = "hadoop.security.kms.client.";
/* It's possible to specify a timeout, in seconds, in the config file */
public static final String TIMEOUT_ATTR = CONFIG_PREFIX + "timeout";
public static final int DEFAULT_TIMEOUT = 60;
private static KeyVersion parseJSONKeyVersion(Map valueMap) {
KeyVersion keyVersion = null;
if (!valueMap.isEmpty()) {
@ -141,6 +148,7 @@ public static String checkNotEmpty(String s, String name)
private String kmsUrl;
private SSLFactory sslFactory;
private ConnectionConfigurator configurator;
@Override
public String toString() {
@ -149,6 +157,42 @@ public String toString() {
return sb.toString();
}
/**
* This small class exists to set the timeout values for a connection
*/
private static class TimeoutConnConfigurator
implements ConnectionConfigurator {
private ConnectionConfigurator cc;
private int timeout;
/**
* Sets the timeout and wraps another connection configurator
* @param timeout - will set both connect and read timeouts - in seconds
* @param cc - another configurator to wrap - may be null
*/
public TimeoutConnConfigurator(int timeout, ConnectionConfigurator cc) {
this.timeout = timeout;
this.cc = cc;
}
/**
* Calls the wrapped configure() method, then sets timeouts
* @param conn the {@link HttpURLConnection} instance to configure.
* @return the connection
* @throws IOException
*/
@Override
public HttpURLConnection configure(HttpURLConnection conn)
throws IOException {
if (cc != null) {
conn = cc.configure(conn);
}
conn.setConnectTimeout(timeout * 1000); // conversion to milliseconds
conn.setReadTimeout(timeout * 1000);
return conn;
}
}
public KMSClientProvider(URI uri, Configuration conf) throws IOException {
Path path = ProviderUtils.unnestUri(uri);
URL url = path.toUri().toURL();
@ -161,6 +205,8 @@ public KMSClientProvider(URI uri, Configuration conf) throws IOException {
throw new IOException(ex);
}
}
int timeout = conf.getInt(TIMEOUT_ATTR, DEFAULT_TIMEOUT);
configurator = new TimeoutConnConfigurator(timeout, sslFactory);
}
private String createServiceURL(URL url) throws IOException {
@ -222,7 +268,7 @@ private HttpURLConnection createConnection(URL url, String method)
HttpURLConnection conn;
try {
AuthenticatedURL authUrl = new AuthenticatedURL(new PseudoAuthenticator(),
sslFactory);
configurator);
conn = authUrl.openConnection(url, new AuthenticatedURL.Token());
} catch (AuthenticationException ex) {
throw new IOException(ex);

View File

@ -38,10 +38,12 @@
import javax.security.auth.login.LoginContext;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.io.Writer;
import java.net.InetAddress;
import java.net.MalformedURLException;
import java.net.ServerSocket;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.net.URL;
import java.security.Principal;
@ -851,4 +853,44 @@ public Void run() throws Exception {
});
}
/**
* Test the configurable timeout in the KMSClientProvider. Open up a
* socket, but don't accept connections for it. This leads to a timeout
* when the KMS client attempts to connect.
* @throws Exception
*/
@Test
public void testKMSTimeout() throws Exception {
File confDir = getTestDir();
Configuration conf = createBaseKMSConf(confDir);
conf.setInt(KMSClientProvider.TIMEOUT_ATTR, 1);
writeConf(confDir, conf);
ServerSocket sock;
int port;
try {
sock = new ServerSocket(0, 50, InetAddress.getByName("localhost"));
port = sock.getLocalPort();
} catch ( Exception e ) {
/* Problem creating socket? Just bail. */
return;
}
URL url = new URL("http://localhost:" + port + "/kms");
URI uri = createKMSUri(url);
boolean caughtTimeout = false;
try {
KeyProvider kp = new KMSClientProvider(uri, conf);
kp.getKeys();
} catch (SocketTimeoutException e) {
caughtTimeout = true;
} catch (IOException e) {
Assert.assertTrue("Caught unexpected exception" + e.toString(), false);
}
Assert.assertTrue(caughtTimeout);
sock.close();
}
}