HDFS-9804. Allow long-running Balancer to login with keytab. Contributed by Xiao Chen.

Change-Id: I54b82e0b11929082057b4d21bfcf285222c94cdb
This commit is contained in:
Zhe Zhang 2016-02-25 09:24:32 -08:00
parent 6979cbfc1f
commit ccff6035f5
6 changed files with 194 additions and 10 deletions

View File

@ -1238,7 +1238,7 @@ private boolean hasSufficientTimeElapsed(long now) {
if (now - user.getLastLogin() < kerberosMinSecondsBeforeRelogin ) {
LOG.warn("Not attempting to re-login since the last re-login was " +
"attempted less than " + (kerberosMinSecondsBeforeRelogin/1000) +
" seconds before.");
" seconds before. Last Login=" + user.getLastLogin());
return false;
}
return true;

View File

@ -58,6 +58,9 @@ Trunk (Unreleased)
HDFS-9525. hadoop utilities need to support provided delegation
tokens (HeeSoo Kim via aw)
HDFS-9804. Allow long-running Balancer to login with keytab.
(Xiao Chen via zhz)
IMPROVEMENTS
HDFS-9653. Added blocks pending deletion report to dfsadmin.

View File

@ -447,6 +447,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final long DFS_BALANCER_GETBLOCKS_SIZE_DEFAULT = 2L*1024*1024*1024; // 2GB
public static final String DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY = "dfs.balancer.getBlocks.min-block-size";
public static final long DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_DEFAULT = 10L*1024*1024; // 10MB
public static final String DFS_BALANCER_KEYTAB_ENABLED_KEY = "dfs.balancer.keytab.enabled";
public static final boolean DFS_BALANCER_KEYTAB_ENABLED_DEFAULT = false;
public static final String DFS_BALANCER_ADDRESS_KEY = "dfs.balancer.address";
public static final String DFS_BALANCER_ADDRESS_DEFAULT= "0.0.0.0:0";
public static final String DFS_BALANCER_KEYTAB_FILE_KEY = "dfs.balancer.keytab.file";
public static final String DFS_BALANCER_KERBEROS_PRINCIPAL_KEY = "dfs.balancer.kerberos.principal";
public static final String DFS_MOVER_MOVEDWINWIDTH_KEY = "dfs.mover.movedWinWidth";

View File

@ -21,6 +21,7 @@
import java.io.IOException;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.net.URI;
import java.text.DateFormat;
import java.util.Arrays;
@ -55,6 +56,9 @@
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.HostsFileReader;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
@ -650,7 +654,7 @@ static int run(Collection<URI> namenodes, final BalancerParameters p,
LOG.info("included nodes = " + p.getIncludedNodes());
LOG.info("excluded nodes = " + p.getExcludedNodes());
LOG.info("source nodes = " + p.getSourceNodes());
checkKeytabAndInit(conf);
System.out.println("Time Stamp Iteration# Bytes Already Moved Bytes Left To Move Bytes Being Moved");
List<NameNodeConnector> connectors = Collections.emptyList();
@ -694,6 +698,22 @@ static int run(Collection<URI> namenodes, final BalancerParameters p,
return ExitStatus.SUCCESS.getExitCode();
}
private static void checkKeytabAndInit(Configuration conf)
throws IOException {
if (conf.getBoolean(DFSConfigKeys.DFS_BALANCER_KEYTAB_ENABLED_KEY,
DFSConfigKeys.DFS_BALANCER_KEYTAB_ENABLED_DEFAULT)) {
LOG.info("Keytab is configured, will login using keytab.");
UserGroupInformation.setConfiguration(conf);
String addr = conf.get(DFSConfigKeys.DFS_BALANCER_ADDRESS_KEY,
DFSConfigKeys.DFS_BALANCER_ADDRESS_DEFAULT);
InetSocketAddress socAddr = NetUtils.createSocketAddr(addr, 0,
DFSConfigKeys.DFS_BALANCER_ADDRESS_KEY);
SecurityUtil.login(conf, DFSConfigKeys.DFS_BALANCER_KEYTAB_FILE_KEY,
DFSConfigKeys.DFS_BALANCER_KERBEROS_PRINCIPAL_KEY,
socAddr.getHostName());
}
}
/* Given elaspedTime in ms, return a printable string */
private static String time2Str(long elapsedTime) {
String unit;

View File

@ -1504,7 +1504,7 @@
<value></value>
<description>
The NameNode service principal. This is typically set to
nn/_HOST@REALM.TLD. Each NameNode will subsitute _HOST with its
nn/_HOST@REALM.TLD. Each NameNode will substitute _HOST with its
own fully qualified hostname at startup. The _HOST placeholder
allows using the same configuration setting on both NameNodes
in an HA setup.
@ -1526,7 +1526,7 @@
<value></value>
<description>
The DataNode service principal. This is typically set to
dn/_HOST@REALM.TLD. Each DataNode will subsitute _HOST with its
dn/_HOST@REALM.TLD. Each DataNode will substitute _HOST with its
own fully qualified hostname at startup. The _HOST placeholder
allows using the same configuration setting on all DataNodes.
</description>
@ -1547,7 +1547,7 @@
<value></value>
<description>
The JournalNode service principal. This is typically set to
jn/_HOST@REALM.TLD. Each JournalNode will subsitute _HOST with its
jn/_HOST@REALM.TLD. Each JournalNode will substitute _HOST with its
own fully qualified hostname at startup. The _HOST placeholder
allows using the same configuration setting on all JournalNodes.
</description>
@ -2830,4 +2830,44 @@
prevention header.
</description>
</property>
<property>
<name>dfs.balancer.keytab.enabled</name>
<value>false</value>
<description>
Set to true to enable login using a keytab for Kerberized Hadoop.
</description>
</property>
<property>
<name>dfs.balancer.address</name>
<value>0.0.0.0:0</value>
<description>
The hostname used for a keytab based Kerberos login. Keytab based login
can be enabled with dfs.balancer.keytab.enabled.
</description>
</property>
<property>
<name>dfs.balancer.keytab.file</name>
<value></value>
<description>
The keytab file used by the Balancer to login as its
service principal. The principal name is configured with
dfs.balancer.kerberos.principal. Keytab based login can be
enabled with dfs.balancer.keytab.enabled.
</description>
</property>
<property>
<name>dfs.balancer.kerberos.principal</name>
<value></value>
<description>
The Balancer principal. This is typically set to
balancer/_HOST@REALM.TLD. The Balancer will substitute _HOST with its
own fully qualified hostname at startup. The _HOST placeholder
allows using the same configuration setting on different servers.
Keytab based login can be enabled with dfs.balancer.keytab.enabled.
</description>
</property>
</configuration>

View File

@ -17,15 +17,32 @@
*/
package org.apache.hadoop.hdfs.server.balancer;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
import static org.apache.hadoop.fs.StorageType.DEFAULT;
import static org.apache.hadoop.fs.StorageType.RAM_DISK;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BALANCER_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BALANCER_KERBEROS_PRINCIPAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BALANCER_KEYTAB_ENABLED_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BALANCER_KEYTAB_FILE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BLOCK_PINNING_ENABLED;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -38,12 +55,14 @@
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Properties;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.TimeoutException;
@ -55,6 +74,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.fs.permission.FsPermission;
@ -85,7 +105,12 @@
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
import org.apache.hadoop.http.HttpConfig;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool;
@ -109,6 +134,11 @@ public class TestBalancer {
final static String RACK2 = "/rack2";
final private static String fileName = "/tmp.txt";
final static Path filePath = new Path(fileName);
final static private String username = "balancer";
private static String principal;
private static File baseDir;
private static MiniKdc kdc;
private static File keytabFile;
private MiniDFSCluster cluster;
@After
@ -179,6 +209,59 @@ static void initConfWithStripe(Configuration conf) {
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
}
static void initSecureConf(Configuration conf) throws Exception {
baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
TestBalancer.class.getSimpleName());
FileUtil.fullyDelete(baseDir);
assertTrue(baseDir.mkdirs());
Properties kdcConf = MiniKdc.createConf();
kdc = new MiniKdc(kdcConf, baseDir);
kdc.start();
SecurityUtil.setAuthenticationMethod(
UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
UserGroupInformation.setConfiguration(conf);
assertTrue("Expected configuration to enable security",
UserGroupInformation.isSecurityEnabled());
keytabFile = new File(baseDir, username + ".keytab");
String keytab = keytabFile.getAbsolutePath();
// Windows will not reverse name lookup "127.0.0.1" to "localhost".
String krbInstance = Path.WINDOWS ? "127.0.0.1" : "localhost";
principal = username + "/" + krbInstance + "@" + kdc.getRealm();
String spnegoPrincipal = "HTTP/" + krbInstance + "@" + kdc.getRealm();
kdc.createPrincipal(keytabFile, username, username + "/" + krbInstance,
"HTTP/" + krbInstance);
conf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, principal);
conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, principal);
conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
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, "authentication");
conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
conf.set(DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
conf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY, 10);
conf.setBoolean(DFS_BALANCER_KEYTAB_ENABLED_KEY, true);
conf.set(DFS_BALANCER_ADDRESS_KEY, "localhost:0");
conf.set(DFS_BALANCER_KEYTAB_FILE_KEY, keytab);
conf.set(DFS_BALANCER_KERBEROS_PRINCIPAL_KEY, principal);
String keystoresDir = baseDir.getAbsolutePath();
String sslConfDir = KeyStoreTestUtil.getClasspathDir(TestBalancer.class);
KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
conf.set(DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY,
KeyStoreTestUtil.getClientSSLConfigFileName());
conf.set(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
KeyStoreTestUtil.getServerSSLConfigFileName());
initConf(conf);
}
/* create a file with a length of <code>fileLen</code> */
static void createFile(MiniDFSCluster cluster, Path filePath, long fileLen,
short replicationFactor, int nnIndex)
@ -948,6 +1031,13 @@ public void integrationTest(Configuration conf) throws Exception {
oneNodeTest(conf, false);
}
@Test(timeout = 100000)
public void testUnknownDatanodeSimple() throws Exception {
Configuration conf = new HdfsConfiguration();
initConf(conf);
testUnknownDatanode(conf);
}
/* we first start a cluster and fill the cluster up to a certain size.
* then redistribute blocks according the required distribution.
* Then we start an empty datanode.
@ -955,10 +1045,8 @@ public void integrationTest(Configuration conf) throws Exception {
* A partially filled datanode is excluded during balancing.
* This triggers a situation where one of the block's location is unknown.
*/
@Test(timeout=100000)
public void testUnknownDatanode() throws Exception {
Configuration conf = new HdfsConfiguration();
initConf(conf);
private void testUnknownDatanode(Configuration conf)
throws IOException, InterruptedException, TimeoutException {
long distribution[] = new long[] {50*CAPACITY/100, 70*CAPACITY/100, 0*CAPACITY/100};
long capacities[] = new long[]{CAPACITY, CAPACITY, CAPACITY};
String racks[] = new String[] {RACK0, RACK1, RACK1};
@ -1844,6 +1932,33 @@ private void doTestBalancerWithStripedFile(Configuration conf) throws Exception
}
}
/**
* Test Balancer runs fine when logging in with a keytab in kerberized env.
* Reusing testUnknownDatanode here for basic functionality testing.
*/
@Test(timeout = 300000)
public void testBalancerWithKeytabs() throws Exception {
final Configuration conf = new HdfsConfiguration();
initSecureConf(conf);
final UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
principal, keytabFile.getAbsolutePath());
try {
ugi.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
// verify that balancer runs Ok.
testUnknownDatanode(conf);
// verify that UGI was logged in using keytab.
assertTrue(UserGroupInformation.isLoginKeytabBased());
return null;
}
});
} finally {
// Reset UGI so that other tests are not affected.
UserGroupInformation.setConfiguration(new Configuration());
}
}
/**
* @param args
*/