소스 검색

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

Change-Id: I54b82e0b11929082057b4d21bfcf285222c94cdb
Zhe Zhang 9 년 전
부모
커밋
ccff6035f5

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -1238,7 +1238,7 @@ public class UserGroupInformation {
     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;

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -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.  

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -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";

+ 21 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 
 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.namenode.UnsupportedActionException;
 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 @@ public class Balancer {
     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 @@ public class Balancer {
     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;

+ 43 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -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>

+ 120 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java

@@ -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.io.PrintWriter;
 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.HadoopIllegalArgumentException;
 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.blockmanagement.DatanodeManager;
 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 @@ public class TestBalancer {
     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)
@@ -947,6 +1030,13 @@ public class TestBalancer {
     initConf(conf);
     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.
@@ -955,10 +1045,8 @@ public class TestBalancer {
    * 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 @@ public class TestBalancer {
     }
   }
 
+  /**
+   * 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
    */
@@ -1853,4 +1968,4 @@ public class TestBalancer {
     balancerTest.testBalancer1();
     balancerTest.testBalancer2();
   }
-}
+}