|
@@ -25,9 +25,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
|
|
import org.apache.hadoop.http.HttpConfig;
|
|
import org.apache.hadoop.http.HttpConfig;
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
-import org.apache.hadoop.security.UserGroupInformation;
|
|
|
|
|
|
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.ServerSocket;
|
|
import java.net.ServerSocket;
|
|
@@ -43,11 +43,21 @@ public class SecureDataNodeStarter implements Daemon {
|
|
* Stash necessary resources needed for datanode operation in a secure env.
|
|
* Stash necessary resources needed for datanode operation in a secure env.
|
|
*/
|
|
*/
|
|
public static class SecureResources {
|
|
public static class SecureResources {
|
|
|
|
+ private final boolean isSaslEnabled;
|
|
|
|
+ private final boolean isRpcPortPrivileged;
|
|
|
|
+ private final boolean isHttpPortPrivileged;
|
|
|
|
+
|
|
private final ServerSocket streamingSocket;
|
|
private final ServerSocket streamingSocket;
|
|
private final ServerSocketChannel httpServerSocket;
|
|
private final ServerSocketChannel httpServerSocket;
|
|
- public SecureResources(ServerSocket streamingSocket, ServerSocketChannel httpServerSocket) {
|
|
|
|
|
|
+
|
|
|
|
+ public SecureResources(ServerSocket streamingSocket, ServerSocketChannel
|
|
|
|
+ httpServerSocket, boolean saslEnabled, boolean rpcPortPrivileged,
|
|
|
|
+ boolean httpPortPrivileged) {
|
|
this.streamingSocket = streamingSocket;
|
|
this.streamingSocket = streamingSocket;
|
|
this.httpServerSocket = httpServerSocket;
|
|
this.httpServerSocket = httpServerSocket;
|
|
|
|
+ this.isSaslEnabled = saslEnabled;
|
|
|
|
+ this.isRpcPortPrivileged = rpcPortPrivileged;
|
|
|
|
+ this.isHttpPortPrivileged = httpPortPrivileged;
|
|
}
|
|
}
|
|
|
|
|
|
public ServerSocket getStreamingSocket() { return streamingSocket; }
|
|
public ServerSocket getStreamingSocket() { return streamingSocket; }
|
|
@@ -55,6 +65,18 @@ public class SecureDataNodeStarter implements Daemon {
|
|
public ServerSocketChannel getHttpServerChannel() {
|
|
public ServerSocketChannel getHttpServerChannel() {
|
|
return httpServerSocket;
|
|
return httpServerSocket;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ public boolean isSaslEnabled() {
|
|
|
|
+ return isSaslEnabled;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public boolean isRpcPortPrivileged() {
|
|
|
|
+ return isRpcPortPrivileged;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public boolean isHttpPortPrivileged() {
|
|
|
|
+ return isHttpPortPrivileged;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
private String [] args;
|
|
private String [] args;
|
|
@@ -90,8 +112,12 @@ public class SecureDataNodeStarter implements Daemon {
|
|
public static SecureResources getSecureResources(Configuration conf)
|
|
public static SecureResources getSecureResources(Configuration conf)
|
|
throws Exception {
|
|
throws Exception {
|
|
HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
|
|
HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
|
|
- boolean isSecure = UserGroupInformation.isSecurityEnabled();
|
|
|
|
|
|
+ boolean isSaslEnabled =
|
|
|
|
+ DataTransferSaslUtil.getSaslPropertiesResolver(conf) != null;
|
|
|
|
+ boolean isRpcPrivileged;
|
|
|
|
+ boolean isHttpPrivileged = false;
|
|
|
|
|
|
|
|
+ System.err.println("isSaslEnabled:" + isSaslEnabled);
|
|
// Obtain secure port for data streaming to datanode
|
|
// Obtain secure port for data streaming to datanode
|
|
InetSocketAddress streamingAddr = DataNode.getStreamingAddr(conf);
|
|
InetSocketAddress streamingAddr = DataNode.getStreamingAddr(conf);
|
|
int socketWriteTimeout = conf.getInt(
|
|
int socketWriteTimeout = conf.getInt(
|
|
@@ -118,12 +144,7 @@ public class SecureDataNodeStarter implements Daemon {
|
|
+ "context. Needed " + streamingAddr.getPort() + ", got "
|
|
+ "context. Needed " + streamingAddr.getPort() + ", got "
|
|
+ ss.getLocalPort());
|
|
+ ss.getLocalPort());
|
|
}
|
|
}
|
|
-
|
|
|
|
- if (!SecurityUtil.isPrivilegedPort(ss.getLocalPort()) && isSecure) {
|
|
|
|
- throw new RuntimeException(
|
|
|
|
- "Cannot start secure datanode with unprivileged RPC ports");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ isRpcPrivileged = SecurityUtil.isPrivilegedPort(ss.getLocalPort());
|
|
System.err.println("Opened streaming server at " + streamingAddr);
|
|
System.err.println("Opened streaming server at " + streamingAddr);
|
|
|
|
|
|
// Bind a port for the web server. The code intends to bind HTTP server to
|
|
// Bind a port for the web server. The code intends to bind HTTP server to
|
|
@@ -151,16 +172,14 @@ public class SecureDataNodeStarter implements Daemon {
|
|
System.err.println("Successfully obtained privileged resources (streaming port = "
|
|
System.err.println("Successfully obtained privileged resources (streaming port = "
|
|
+ ss + " ) (http listener port = " + localAddr.getPort() +")");
|
|
+ ss + " ) (http listener port = " + localAddr.getPort() +")");
|
|
|
|
|
|
- if (localAddr.getPort() > 1023 && isSecure) {
|
|
|
|
- throw new RuntimeException(
|
|
|
|
- "Cannot start secure datanode with unprivileged HTTP ports");
|
|
|
|
- }
|
|
|
|
|
|
+ isHttpPrivileged = SecurityUtil.isPrivilegedPort(localAddr.getPort());
|
|
System.err.println("Opened info server at " + infoSocAddr);
|
|
System.err.println("Opened info server at " + infoSocAddr);
|
|
} else {
|
|
} else {
|
|
httpChannel = null;
|
|
httpChannel = null;
|
|
}
|
|
}
|
|
|
|
|
|
- return new SecureResources(ss, httpChannel);
|
|
|
|
|
|
+ return new SecureResources(ss, httpChannel, isSaslEnabled,
|
|
|
|
+ isRpcPrivileged, isHttpPrivileged);
|
|
}
|
|
}
|
|
|
|
|
|
private static BindException appendMessageToBindException(BindException e,
|
|
private static BindException appendMessageToBindException(BindException e,
|