|
@@ -335,9 +335,7 @@ public class DataNode extends Configured
|
|
: new HttpServer("datanode", infoHost, tmpInfoPort, tmpInfoPort == 0,
|
|
: new HttpServer("datanode", infoHost, tmpInfoPort, tmpInfoPort == 0,
|
|
conf, new AccessControlList(conf.get(DFS_ADMIN, " ")),
|
|
conf, new AccessControlList(conf.get(DFS_ADMIN, " ")),
|
|
secureResources.getListener());
|
|
secureResources.getListener());
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Datanode listening on " + infoHost + ":" + tmpInfoPort);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.info("Opened info server at " + infoHost + ":" + tmpInfoPort);
|
|
if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
|
|
if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
|
|
boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
|
|
boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
|
|
DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT);
|
|
DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT);
|
|
@@ -403,7 +401,8 @@ public class DataNode extends Configured
|
|
.newReflectiveBlockingService(interDatanodeProtocolXlator);
|
|
.newReflectiveBlockingService(interDatanodeProtocolXlator);
|
|
DFSUtil.addPBProtocol(conf, InterDatanodeProtocolPB.class, service,
|
|
DFSUtil.addPBProtocol(conf, InterDatanodeProtocolPB.class, service,
|
|
ipcServer);
|
|
ipcServer);
|
|
-
|
|
|
|
|
|
+ LOG.info("Opened IPC server at " + ipcServer.getListenerAddress());
|
|
|
|
+
|
|
// set service-level authorization security policy
|
|
// set service-level authorization security policy
|
|
if (conf.getBoolean(
|
|
if (conf.getBoolean(
|
|
CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
|
|
CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
|
|
@@ -491,14 +490,14 @@ public class DataNode extends Configured
|
|
}
|
|
}
|
|
|
|
|
|
private void initDataXceiver(Configuration conf) throws IOException {
|
|
private void initDataXceiver(Configuration conf) throws IOException {
|
|
- InetSocketAddress socAddr = DataNode.getStreamingAddr(conf);
|
|
|
|
|
|
+ InetSocketAddress streamingAddr = DataNode.getStreamingAddr(conf);
|
|
|
|
|
|
// find free port or use privileged port provided
|
|
// find free port or use privileged port provided
|
|
ServerSocket ss;
|
|
ServerSocket ss;
|
|
if(secureResources == null) {
|
|
if(secureResources == null) {
|
|
ss = (dnConf.socketWriteTimeout > 0) ?
|
|
ss = (dnConf.socketWriteTimeout > 0) ?
|
|
ServerSocketChannel.open().socket() : new ServerSocket();
|
|
ServerSocketChannel.open().socket() : new ServerSocket();
|
|
- Server.bind(ss, socAddr, 0);
|
|
|
|
|
|
+ Server.bind(ss, streamingAddr, 0);
|
|
} else {
|
|
} else {
|
|
ss = secureResources.getStreamingSocket();
|
|
ss = secureResources.getStreamingSocket();
|
|
}
|
|
}
|
|
@@ -507,8 +506,7 @@ public class DataNode extends Configured
|
|
int tmpPort = ss.getLocalPort();
|
|
int tmpPort = ss.getLocalPort();
|
|
selfAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
|
|
selfAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
|
|
tmpPort);
|
|
tmpPort);
|
|
- LOG.info("Opened info server at " + tmpPort);
|
|
|
|
-
|
|
|
|
|
|
+ LOG.info("Opened streaming server at " + selfAddr);
|
|
this.threadGroup = new ThreadGroup("dataXceiverServer");
|
|
this.threadGroup = new ThreadGroup("dataXceiverServer");
|
|
this.dataXceiverServer = new Daemon(threadGroup,
|
|
this.dataXceiverServer = new Daemon(threadGroup,
|
|
new DataXceiverServer(ss, conf, this));
|
|
new DataXceiverServer(ss, conf, this));
|