HDFS-11508. Fix bind failure in SimpleTCPServer & Portmap where bind fails because socket is in TIME_WAIT state. Contributed by Mukul Kumar Singh.
This commit is contained in:
parent
959940b0ab
commit
1f9848dfe1
|
@ -81,6 +81,7 @@ public class SimpleTcpServer {
|
|||
});
|
||||
server.setOption("child.tcpNoDelay", true);
|
||||
server.setOption("child.keepAlive", true);
|
||||
server.setOption("reuseAddress", true);
|
||||
|
||||
// Listen to TCP port
|
||||
ch = server.bind(new InetSocketAddress(port));
|
||||
|
|
|
@ -109,12 +109,14 @@ final class Portmap {
|
|||
RpcUtil.STAGE_RPC_TCP_RESPONSE);
|
||||
}
|
||||
});
|
||||
tcpServer.setOption("reuseAddress", true);
|
||||
|
||||
udpServer = new ConnectionlessBootstrap(new NioDatagramChannelFactory(
|
||||
Executors.newCachedThreadPool()));
|
||||
|
||||
udpServer.setPipeline(Channels.pipeline(RpcUtil.STAGE_RPC_MESSAGE_PARSER,
|
||||
handler, RpcUtil.STAGE_RPC_UDP_RESPONSE));
|
||||
udpServer.setOption("reuseAddress", true);
|
||||
|
||||
tcpChannel = tcpServer.bind(tcpAddress);
|
||||
udpChannel = udpServer.bind(udpAddress);
|
||||
|
|
Loading…
Reference in New Issue