|
@@ -60,6 +60,7 @@ import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.io.BytesWritable;
|
|
@@ -193,6 +194,7 @@ public abstract class Server {
|
|
|
protected RpcDetailedMetrics rpcDetailedMetrics;
|
|
|
|
|
|
private Configuration conf;
|
|
|
+ private String portRangeConfig = null;
|
|
|
private SecretManager<TokenIdentifier> secretManager;
|
|
|
private ServiceAuthorizationManager serviceAuthorizationManager = new ServiceAuthorizationManager();
|
|
|
|
|
@@ -225,8 +227,33 @@ public abstract class Server {
|
|
|
*/
|
|
|
public static void bind(ServerSocket socket, InetSocketAddress address,
|
|
|
int backlog) throws IOException {
|
|
|
+ bind(socket, address, backlog, null, null);
|
|
|
+ }
|
|
|
+
|
|
|
+ public static void bind(ServerSocket socket, InetSocketAddress address,
|
|
|
+ int backlog, Configuration conf, String rangeConf) throws IOException {
|
|
|
try {
|
|
|
- socket.bind(address, backlog);
|
|
|
+ IntegerRanges range = null;
|
|
|
+ if (rangeConf != null) {
|
|
|
+ range = conf.getRange(rangeConf, "");
|
|
|
+ }
|
|
|
+ if (range == null || range.isEmpty() || (address.getPort() != 0)) {
|
|
|
+ socket.bind(address, backlog);
|
|
|
+ } else {
|
|
|
+ for (Integer port : range) {
|
|
|
+ if (socket.isBound()) break;
|
|
|
+ try {
|
|
|
+ InetSocketAddress temp = new InetSocketAddress(address.getAddress(),
|
|
|
+ port);
|
|
|
+ socket.bind(temp, backlog);
|
|
|
+ } catch(BindException e) {
|
|
|
+ //Ignored
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (!socket.isBound()) {
|
|
|
+ throw new BindException("Could not find a free port in "+range);
|
|
|
+ }
|
|
|
+ }
|
|
|
} catch (SocketException e) {
|
|
|
throw NetUtils.wrapException(null,
|
|
|
0,
|
|
@@ -310,7 +337,7 @@ public abstract class Server {
|
|
|
acceptChannel.configureBlocking(false);
|
|
|
|
|
|
// Bind the server socket to the local host and port
|
|
|
- bind(acceptChannel.socket(), address, backlogLength);
|
|
|
+ bind(acceptChannel.socket(), address, backlogLength, conf, portRangeConfig);
|
|
|
port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
|
|
|
// create a selector;
|
|
|
selector= Selector.open();
|
|
@@ -1543,9 +1570,18 @@ public abstract class Server {
|
|
|
Configuration conf)
|
|
|
throws IOException
|
|
|
{
|
|
|
- this(bindAddress, port, paramClass, handlerCount, -1, -1, conf, Integer.toString(port), null);
|
|
|
+ this(bindAddress, port, paramClass, handlerCount, -1, -1, conf, Integer.toString(port), null, null);
|
|
|
}
|
|
|
|
|
|
+ protected Server(String bindAddress, int port,
|
|
|
+ Class<? extends Writable> rpcRequestClass, int handlerCount,
|
|
|
+ int numReaders, int queueSizePerHandler, Configuration conf,
|
|
|
+ String serverName, SecretManager<? extends TokenIdentifier> secretManager)
|
|
|
+ throws IOException {
|
|
|
+ this(bindAddress, port, rpcRequestClass, handlerCount, numReaders,
|
|
|
+ queueSizePerHandler, conf, serverName, secretManager, null);
|
|
|
+ }
|
|
|
+
|
|
|
/** Constructs a server listening on the named port and address. Parameters passed must
|
|
|
* be of the named class. The <code>handlerCount</handlerCount> determines
|
|
|
* the number of handler threads that will be used to process calls.
|
|
@@ -1554,11 +1590,13 @@ public abstract class Server {
|
|
|
*/
|
|
|
@SuppressWarnings("unchecked")
|
|
|
protected Server(String bindAddress, int port,
|
|
|
- Class<? extends Writable> paramClass, int handlerCount, int numReaders, int queueSizePerHandler,
|
|
|
- Configuration conf, String serverName, SecretManager<? extends TokenIdentifier> secretManager)
|
|
|
- throws IOException {
|
|
|
+ Class<? extends Writable> paramClass, int handlerCount, int numReaders, int queueSizePerHandler,
|
|
|
+ Configuration conf, String serverName, SecretManager<? extends TokenIdentifier> secretManager,
|
|
|
+ String portRangeConfig)
|
|
|
+ throws IOException {
|
|
|
this.bindAddress = bindAddress;
|
|
|
this.conf = conf;
|
|
|
+ this.portRangeConfig = portRangeConfig;
|
|
|
this.port = port;
|
|
|
this.paramClass = paramClass;
|
|
|
this.handlerCount = handlerCount;
|