HDFS-6666. Abort NameNode and DataNode startup if security is enabled but block access token is not enabled. Contributed by Vijay Bhat.

This commit is contained in:
cnauroth 2015-04-14 09:59:01 -07:00
parent b5a0b24643
commit d45aa7647b
6 changed files with 115 additions and 50 deletions

View File

@ -494,6 +494,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

View File

@ -359,7 +359,7 @@ public BlockManager(final Namesystem namesystem, final Configuration conf)
}
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 @@ private static BlockTokenSecretManager createBlockTokenSecretManager(
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;
}

View File

@ -1183,6 +1183,20 @@ private static void checkSecureConfig(DNConf dnConf, Configuration conf,
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;

View File

@ -1242,10 +1242,12 @@ void stopActiveServices() {
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();

View File

@ -33,6 +33,7 @@
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 static void initKdc() throws Exception {
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 @@ protected HdfsConfiguration createSecureConfig(
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);

View File

@ -24,71 +24,63 @@
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.security.TestUGIWithSecurityOn;
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferTestCase;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.junit.Assume;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
public class TestSecureNameNode {
public class TestSecureNameNode extends SaslDataTransferTestCase {
final static private int NUM_OF_DATANODES = 0;
@Before
public void testKdcRunning() {
// Tests are skipped if KDC is not running
Assume.assumeTrue(TestUGIWithSecurityOn.isKdcRunning());
}
@Rule
public ExpectedException exception = ExpectedException.none();
@Test
public void testName() throws IOException, InterruptedException {
public void testName() throws Exception {
MiniDFSCluster cluster = null;
HdfsConfiguration conf = createSecureConfig(
"authentication,privacy");
try {
String keyTabDir = System.getProperty("kdc.resource.dir") + "/keytabs";
String nn1KeytabPath = keyTabDir + "/nn1.keytab";
String user1KeyTabPath = keyTabDir + "/user1.keytab";
Configuration conf = new HdfsConfiguration();
conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
"kerberos");
conf.set(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY,
"nn1/localhost@EXAMPLE.COM");
conf.set(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY, nn1KeytabPath);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_OF_DATANODES)
.build();
final MiniDFSCluster clusterRef = cluster;
cluster.waitActive();
FileSystem fsForCurrentUser = cluster.getFileSystem();
fsForCurrentUser.mkdirs(new Path("/tmp"));
fsForCurrentUser.setPermission(new Path("/tmp"), new FsPermission(
FileSystem fsForSuperUser = UserGroupInformation
.loginUserFromKeytabAndReturnUGI(getHdfsPrincipal(), getHdfsKeytab()).doAs(new PrivilegedExceptionAction<FileSystem>() {
@Override
public FileSystem run() throws Exception {
return clusterRef.getFileSystem();
}
});
fsForSuperUser.mkdirs(new Path("/tmp"));
fsForSuperUser.setPermission(new Path("/tmp"), new FsPermission(
(short) 511));
UserGroupInformation ugi = UserGroupInformation
.loginUserFromKeytabAndReturnUGI("user1@EXAMPLE.COM", user1KeyTabPath);
.loginUserFromKeytabAndReturnUGI(getUserPrincipal(), getUserKeyTab());
FileSystem fs = ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
@Override
public FileSystem run() throws Exception {
return clusterRef.getFileSystem();
}
});
try {
Path p = new Path("/users");
fs.mkdirs(p);
fail("user1 must not be allowed to write in /");
} catch (IOException expected) {
}
Path p = new Path("/tmp/alpha");
Path p = new Path("/mydir");
exception.expect(IOException.class);
fs.mkdirs(p);
assertNotNull(fs.listStatus(p));
Path tmp = new Path("/tmp/alpha");
fs.mkdirs(tmp);
assertNotNull(fs.listStatus(tmp));
assertEquals(AuthenticationMethod.KERBEROS,
ugi.getAuthenticationMethod());
} finally {
@ -97,4 +89,32 @@ public FileSystem run() throws Exception {
}
}
}
/**
* Verify the following scenario.
* 1. Kerberos is enabled.
* 2. HDFS block tokens are not enabled.
* 3. Start the NN.
* 4. NN should throw an IOException and abort
* @throws Exception
*/
@Test
public void testKerberosHdfsBlockTokenInconsistencyNNStartup() throws Exception {
MiniDFSCluster dfsCluster = null;
HdfsConfiguration conf = createSecureConfig(
"authentication,privacy");
try {
conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, false);
exception.expect(IOException.class);
exception.expectMessage("Security is enabled but block access tokens");
dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
dfsCluster.waitActive();
} finally {
if (dfsCluster != null) {
dfsCluster.shutdown();
}
}
return;
}
}