HDFS-6666. Abort NameNode and DataNode startup if security is enabled but block access token is not enabled. Contributed by Vijay Bhat.
(cherry picked from commit d45aa7647b
)
Conflicts:
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java
This commit is contained in:
parent
a75679cdb3
commit
9779b79ed8
|
@ -173,6 +173,9 @@ Release 2.8.0 - UNRELEASED
|
|||
HDFS-7701. Support reporting per storage type quota and usage
|
||||
with hadoop/hdfs shell. (Peter Shi via Arpit Agarwal)
|
||||
|
||||
HDFS-6666. Abort NameNode and DataNode startup if security is enabled but
|
||||
block access token is not enabled. (Vijay Bhat via cnauroth)
|
||||
|
||||
Release 2.7.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -359,7 +359,7 @@ public class BlockManager {
|
|||
}
|
||||
|
||||
private static BlockTokenSecretManager createBlockTokenSecretManager(
|
||||
final Configuration conf) {
|
||||
final Configuration conf) throws IOException {
|
||||
final boolean isEnabled = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY,
|
||||
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT);
|
||||
|
@ -367,10 +367,11 @@ public class BlockManager {
|
|||
|
||||
if (!isEnabled) {
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
LOG.error("Security is enabled but block access tokens " +
|
||||
String errMessage = "Security is enabled but block access tokens " +
|
||||
"(via " + DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY + ") " +
|
||||
"aren't enabled. This may cause issues " +
|
||||
"when clients attempt to talk to a DataNode.");
|
||||
"when clients attempt to connect to a DataNode. Aborting NameNode";
|
||||
throw new IOException(errMessage);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -1190,6 +1190,20 @@ public class DataNode extends ReconfigurableBase
|
|||
if (!UserGroupInformation.isSecurityEnabled()) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Abort out of inconsistent state if Kerberos is enabled
|
||||
// but block access tokens are not enabled.
|
||||
boolean isEnabled = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY,
|
||||
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT);
|
||||
if (!isEnabled) {
|
||||
String errMessage = "Security is enabled but block access tokens " +
|
||||
"(via " + DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY + ") " +
|
||||
"aren't enabled. This may cause issues " +
|
||||
"when clients attempt to connect to a DataNode. Aborting DataNode";
|
||||
throw new RuntimeException(errMessage);
|
||||
}
|
||||
|
||||
SaslPropertiesResolver saslPropsResolver = dnConf.getSaslPropsResolver();
|
||||
if (resources != null && saslPropsResolver == null) {
|
||||
return;
|
||||
|
|
|
@ -1234,10 +1234,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
cacheManager.stopMonitorThread();
|
||||
cacheManager.clearDirectiveStats();
|
||||
}
|
||||
blockManager.getDatanodeManager().clearPendingCachingCommands();
|
||||
blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
|
||||
// Don't want to keep replication queues when not in Active.
|
||||
blockManager.clearQueues();
|
||||
if (blockManager != null) {
|
||||
blockManager.getDatanodeManager().clearPendingCachingCommands();
|
||||
blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
|
||||
// Don't want to keep replication queues when not in Active.
|
||||
blockManager.clearQueues();
|
||||
}
|
||||
initializedReplQueues = false;
|
||||
} finally {
|
||||
writeUnlock();
|
||||
|
|
|
@ -33,6 +33,7 @@ import static org.junit.Assert.*;
|
|||
import java.io.File;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.commons.lang.RandomStringUtils;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.http.HttpConfig;
|
||||
|
@ -48,10 +49,28 @@ public abstract class SaslDataTransferTestCase {
|
|||
|
||||
private static File baseDir;
|
||||
private static String hdfsPrincipal;
|
||||
private static String userPrincipal;
|
||||
private static MiniKdc kdc;
|
||||
private static String keytab;
|
||||
private static String hdfsKeytab;
|
||||
private static String userKeyTab;
|
||||
private static String spnegoPrincipal;
|
||||
|
||||
public static String getUserKeyTab() {
|
||||
return userKeyTab;
|
||||
}
|
||||
|
||||
public static String getUserPrincipal() {
|
||||
return userPrincipal;
|
||||
}
|
||||
|
||||
public static String getHdfsPrincipal() {
|
||||
return hdfsPrincipal;
|
||||
}
|
||||
|
||||
public static String getHdfsKeytab() {
|
||||
return hdfsKeytab;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void initKdc() throws Exception {
|
||||
baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
|
||||
|
@ -63,11 +82,17 @@ public abstract class SaslDataTransferTestCase {
|
|||
kdc = new MiniKdc(kdcConf, baseDir);
|
||||
kdc.start();
|
||||
|
||||
String userName = UserGroupInformation.getLoginUser().getShortUserName();
|
||||
File keytabFile = new File(baseDir, userName + ".keytab");
|
||||
keytab = keytabFile.getAbsolutePath();
|
||||
kdc.createPrincipal(keytabFile, userName + "/localhost", "HTTP/localhost");
|
||||
hdfsPrincipal = userName + "/localhost@" + kdc.getRealm();
|
||||
String userName = RandomStringUtils.randomAlphabetic(8);
|
||||
File userKeytabFile = new File(baseDir, userName + ".keytab");
|
||||
userKeyTab = userKeytabFile.getAbsolutePath();
|
||||
kdc.createPrincipal(userKeytabFile, userName + "/localhost");
|
||||
userPrincipal = userName + "/localhost@" + kdc.getRealm();
|
||||
|
||||
String superUserName = "hdfs";
|
||||
File hdfsKeytabFile = new File(baseDir, superUserName + ".keytab");
|
||||
hdfsKeytab = hdfsKeytabFile.getAbsolutePath();
|
||||
kdc.createPrincipal(hdfsKeytabFile, superUserName + "/localhost", "HTTP/localhost");
|
||||
hdfsPrincipal = superUserName + "/localhost@" + kdc.getRealm();
|
||||
spnegoPrincipal = "HTTP/localhost@" + kdc.getRealm();
|
||||
}
|
||||
|
||||
|
@ -91,9 +116,9 @@ public abstract class SaslDataTransferTestCase {
|
|||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
|
||||
conf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
|
||||
conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
|
||||
conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, hdfsKeytab);
|
||||
conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
|
||||
conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
|
||||
conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, hdfsKeytab);
|
||||
conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoPrincipal);
|
||||
conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
|
||||
conf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, dataTransferProtection);
|
||||
|
|
Loading…
Reference in New Issue