HDFS-9887. WebHdfs socket timeouts should be configurable. Contributed by Austin Donnelly and Chris Douglas.
This commit is contained in:
parent
27e0681f28
commit
5abf051249
|
@ -38,6 +38,11 @@ public interface HdfsClientConfigKeys {
|
||||||
String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
|
String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
|
||||||
"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
|
"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
|
||||||
|
|
||||||
|
String DFS_WEBHDFS_SOCKET_CONNECT_TIMEOUT_KEY =
|
||||||
|
"dfs.webhdfs.socket.connect-timeout";
|
||||||
|
String DFS_WEBHDFS_SOCKET_READ_TIMEOUT_KEY =
|
||||||
|
"dfs.webhdfs.socket.read-timeout";
|
||||||
|
|
||||||
String DFS_WEBHDFS_OAUTH_ENABLED_KEY = "dfs.webhdfs.oauth2.enabled";
|
String DFS_WEBHDFS_OAUTH_ENABLED_KEY = "dfs.webhdfs.oauth2.enabled";
|
||||||
boolean DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT = false;
|
boolean DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT = false;
|
||||||
|
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.net.HttpURLConnection;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.net.URLConnection;
|
import java.net.URLConnection;
|
||||||
import java.security.GeneralSecurityException;
|
import java.security.GeneralSecurityException;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
import javax.net.ssl.HostnameVerifier;
|
import javax.net.ssl.HostnameVerifier;
|
||||||
import javax.net.ssl.HttpsURLConnection;
|
import javax.net.ssl.HttpsURLConnection;
|
||||||
|
@ -31,6 +32,7 @@ import javax.net.ssl.SSLSocketFactory;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.web.oauth2.OAuth2ConnectionConfigurator;
|
import org.apache.hadoop.hdfs.web.oauth2.OAuth2ConnectionConfigurator;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
||||||
|
@ -62,7 +64,9 @@ public class URLConnectionFactory {
|
||||||
@Override
|
@Override
|
||||||
public HttpURLConnection configure(HttpURLConnection conn)
|
public HttpURLConnection configure(HttpURLConnection conn)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
URLConnectionFactory.setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
|
URLConnectionFactory.setTimeouts(conn,
|
||||||
|
DEFAULT_SOCKET_TIMEOUT,
|
||||||
|
DEFAULT_SOCKET_TIMEOUT);
|
||||||
return conn;
|
return conn;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -128,17 +132,29 @@ public class URLConnectionFactory {
|
||||||
* Create a new ConnectionConfigurator for SSL connections
|
* Create a new ConnectionConfigurator for SSL connections
|
||||||
*/
|
*/
|
||||||
private static ConnectionConfigurator newSslConnConfigurator(
|
private static ConnectionConfigurator newSslConnConfigurator(
|
||||||
final int timeout, Configuration conf)
|
final int defaultTimeout, Configuration conf)
|
||||||
throws IOException, GeneralSecurityException {
|
throws IOException, GeneralSecurityException {
|
||||||
final SSLFactory factory;
|
final SSLFactory factory;
|
||||||
final SSLSocketFactory sf;
|
final SSLSocketFactory sf;
|
||||||
final HostnameVerifier hv;
|
final HostnameVerifier hv;
|
||||||
|
final int connectTimeout;
|
||||||
|
final int readTimeout;
|
||||||
|
|
||||||
factory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
|
factory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
|
||||||
factory.init();
|
factory.init();
|
||||||
sf = factory.createSSLSocketFactory();
|
sf = factory.createSSLSocketFactory();
|
||||||
hv = factory.getHostnameVerifier();
|
hv = factory.getHostnameVerifier();
|
||||||
|
|
||||||
|
connectTimeout = (int) conf.getTimeDuration(
|
||||||
|
HdfsClientConfigKeys.DFS_WEBHDFS_SOCKET_CONNECT_TIMEOUT_KEY,
|
||||||
|
defaultTimeout,
|
||||||
|
TimeUnit.MILLISECONDS);
|
||||||
|
|
||||||
|
readTimeout = (int) conf.getTimeDuration(
|
||||||
|
HdfsClientConfigKeys.DFS_WEBHDFS_SOCKET_READ_TIMEOUT_KEY,
|
||||||
|
defaultTimeout,
|
||||||
|
TimeUnit.MILLISECONDS);
|
||||||
|
|
||||||
return new ConnectionConfigurator() {
|
return new ConnectionConfigurator() {
|
||||||
@Override
|
@Override
|
||||||
public HttpURLConnection configure(HttpURLConnection conn)
|
public HttpURLConnection configure(HttpURLConnection conn)
|
||||||
|
@ -148,7 +164,7 @@ public class URLConnectionFactory {
|
||||||
c.setSSLSocketFactory(sf);
|
c.setSSLSocketFactory(sf);
|
||||||
c.setHostnameVerifier(hv);
|
c.setHostnameVerifier(hv);
|
||||||
}
|
}
|
||||||
URLConnectionFactory.setTimeouts(conn, timeout);
|
URLConnectionFactory.setTimeouts(conn, connectTimeout, readTimeout);
|
||||||
return conn;
|
return conn;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -208,8 +224,10 @@ public class URLConnectionFactory {
|
||||||
* @param socketTimeout
|
* @param socketTimeout
|
||||||
* the connection and read timeout of the connection.
|
* the connection and read timeout of the connection.
|
||||||
*/
|
*/
|
||||||
private static void setTimeouts(URLConnection connection, int socketTimeout) {
|
private static void setTimeouts(URLConnection connection,
|
||||||
connection.setConnectTimeout(socketTimeout);
|
int connectTimeout,
|
||||||
connection.setReadTimeout(socketTimeout);
|
int readTimeout) {
|
||||||
|
connection.setConnectTimeout(connectTimeout);
|
||||||
|
connection.setReadTimeout(readTimeout);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1993,9 +1993,12 @@ Release 2.8.0 - UNRELEASED
|
||||||
HDFS-9843. Document distcp options required for copying between encrypted
|
HDFS-9843. Document distcp options required for copying between encrypted
|
||||||
locations. (Xiaoyu Yao via cnauroth)
|
locations. (Xiaoyu Yao via cnauroth)
|
||||||
|
|
||||||
HDFS-9831.Document webhdfs retry configuration keys introduced by
|
HDFS-9831. Document webhdfs retry configuration keys introduced by
|
||||||
HDFS-5219/HDFS-5122. (Xiaobing Zhou via xyao)
|
HDFS-5219/HDFS-5122. (Xiaobing Zhou via xyao)
|
||||||
|
|
||||||
|
HDFS-9887. WebHdfs socket timeouts should be configurable.
|
||||||
|
(Austin Donnelly and Chris Douglas via xyao)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
|
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
|
||||||
|
|
|
@ -2317,6 +2317,33 @@
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>dfs.webhdfs.socket.connect-timeout</name>
|
||||||
|
<value>60s</value>
|
||||||
|
<description>
|
||||||
|
Socket timeout for connecting to WebHDFS servers. This prevents a
|
||||||
|
WebHDFS client from hanging if the server hostname is
|
||||||
|
misconfigured, or the server does not response before the timeout
|
||||||
|
expires. Value is followed by a unit specifier: ns, us, ms, s, m,
|
||||||
|
h, d for nanoseconds, microseconds, milliseconds, seconds,
|
||||||
|
minutes, hours, days respectively. Values should provide units,
|
||||||
|
but milliseconds are assumed.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>dfs.webhdfs.socket.read-timeout</name>
|
||||||
|
<value>60s</value>
|
||||||
|
<description>
|
||||||
|
Socket timeout for reading data from WebHDFS servers. This
|
||||||
|
prevents a WebHDFS client from hanging if the server stops sending
|
||||||
|
data. Value is followed by a unit specifier: ns, us, ms, s, m, h,
|
||||||
|
d for nanoseconds, microseconds, milliseconds, seconds, minutes,
|
||||||
|
hours, days respectively. Values should provide units,
|
||||||
|
but milliseconds are assumed.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>dfs.client.context</name>
|
<name>dfs.client.context</name>
|
||||||
<value>default</value>
|
<value>default</value>
|
||||||
|
|
|
@ -199,6 +199,8 @@ Below are the HDFS configuration options for WebHDFS.
|
||||||
|:---- |:---- |
|
|:---- |:---- |
|
||||||
| `dfs.web.authentication.kerberos.principal` | The HTTP Kerberos principal used by Hadoop-Auth in the HTTP endpoint. The HTTP Kerberos principal MUST start with 'HTTP/' per Kerberos HTTP SPNEGO specification. A value of "\*" will use all HTTP principals found in the keytab. |
|
| `dfs.web.authentication.kerberos.principal` | The HTTP Kerberos principal used by Hadoop-Auth in the HTTP endpoint. The HTTP Kerberos principal MUST start with 'HTTP/' per Kerberos HTTP SPNEGO specification. A value of "\*" will use all HTTP principals found in the keytab. |
|
||||||
| `dfs.web.authentication.kerberos.keytab ` | The Kerberos keytab file with the credentials for the HTTP Kerberos principal used by Hadoop-Auth in the HTTP endpoint. |
|
| `dfs.web.authentication.kerberos.keytab ` | The Kerberos keytab file with the credentials for the HTTP Kerberos principal used by Hadoop-Auth in the HTTP endpoint. |
|
||||||
|
| `dfs.webhdfs.socket.connect-timeout` | How long to wait for a connection to be established before failing. Specified as a time duration, ie numerical value followed by a units symbol, eg 2m for two minutes. Defaults to 60s. |
|
||||||
|
| `dfs.webhdfs.socket.read-timeout` | How long to wait for data to arrive before failing. Defaults to 60s. |
|
||||||
|
|
||||||
Authentication
|
Authentication
|
||||||
--------------
|
--------------
|
||||||
|
|
|
@ -33,18 +33,25 @@ import java.net.Socket;
|
||||||
import java.net.SocketTimeoutException;
|
import java.net.SocketTimeoutException;
|
||||||
import java.nio.channels.SocketChannel;
|
import java.nio.channels.SocketChannel;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
|
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
import org.junit.runners.Parameterized.Parameter;
|
||||||
|
import org.junit.runners.Parameterized.Parameters;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -54,6 +61,7 @@ import org.junit.Test;
|
||||||
* bogus server on the namenode HTTP port, which is rigged to not accept new
|
* bogus server on the namenode HTTP port, which is rigged to not accept new
|
||||||
* connections or to accept connections but not send responses.
|
* connections or to accept connections but not send responses.
|
||||||
*/
|
*/
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
public class TestWebHdfsTimeouts {
|
public class TestWebHdfsTimeouts {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(TestWebHdfsTimeouts.class);
|
private static final Log LOG = LogFactory.getLog(TestWebHdfsTimeouts.class);
|
||||||
|
@ -77,14 +85,41 @@ public class TestWebHdfsTimeouts {
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
public enum TimeoutSource { ConnectionFactory, Configuration };
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run all tests twice: once with the timeouts set by the
|
||||||
|
* connection factory, and again with the timeouts set by
|
||||||
|
* configuration options.
|
||||||
|
*/
|
||||||
|
@Parameters(name = "timeoutSource={0}")
|
||||||
|
public static Collection<Object[]> data() {
|
||||||
|
return Arrays.asList(new Object[][] {
|
||||||
|
{ TimeoutSource.ConnectionFactory },
|
||||||
|
{ TimeoutSource.Configuration }
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Parameter
|
||||||
|
public TimeoutSource timeoutSource;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
Configuration conf = WebHdfsTestUtil.createConf();
|
Configuration conf = WebHdfsTestUtil.createConf();
|
||||||
serverSocket = new ServerSocket(0, CONNECTION_BACKLOG);
|
serverSocket = new ServerSocket(0, CONNECTION_BACKLOG);
|
||||||
nnHttpAddress = new InetSocketAddress("localhost", serverSocket.getLocalPort());
|
nnHttpAddress = new InetSocketAddress("localhost", serverSocket.getLocalPort());
|
||||||
conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "localhost:" + serverSocket.getLocalPort());
|
conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "localhost:" + serverSocket.getLocalPort());
|
||||||
|
if (timeoutSource == TimeoutSource.Configuration) {
|
||||||
|
String v = Integer.toString(SHORT_SOCKET_TIMEOUT) + "ms";
|
||||||
|
conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_SOCKET_CONNECT_TIMEOUT_KEY, v);
|
||||||
|
conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_SOCKET_READ_TIMEOUT_KEY, v);
|
||||||
|
}
|
||||||
|
|
||||||
fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME);
|
fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME);
|
||||||
fs.connectionFactory = connectionFactory;
|
if (timeoutSource == TimeoutSource.ConnectionFactory) {
|
||||||
|
fs.connectionFactory = connectionFactory;
|
||||||
|
}
|
||||||
|
|
||||||
clients = new ArrayList<SocketChannel>();
|
clients = new ArrayList<SocketChannel>();
|
||||||
serverThread = null;
|
serverThread = null;
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue