瀏覽代碼

HDFS-5898. Allow NFS gateway to login/relogin from its kerberos keytab. Contributed by Abin Shahab.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1574693 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 11 年之前
父節點
當前提交
6adf7a0ecb

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java

@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs.nfs.mount;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_USER_NAME_KEY;
 
 import java.io.IOException;
 import java.net.InetAddress;
@@ -46,6 +48,8 @@ import org.apache.hadoop.oncrpc.RpcResponse;
 import org.apache.hadoop.oncrpc.RpcUtil;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.security.VerifierNone;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.ChannelHandlerContext;
@@ -83,6 +87,9 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
         Nfs3Constant.EXPORT_POINT_DEFAULT));
     this.hostsMatcher = NfsExports.getInstance(config);
     this.mounts = Collections.synchronizedList(new ArrayList<MountEntry>());
+    UserGroupInformation.setConfiguration(config);
+    SecurityUtil.login(config, DFS_NFS_KEYTAB_FILE_KEY,
+            DFS_NFS_USER_NAME_KEY);
     this.dfsClient = new DFSClient(NameNode.getAddress(config), config);
   }
   

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.nfs.nfs3;
 
+import org.apache.commons.logging.LogFactory;
+
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -28,7 +30,6 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -192,6 +193,8 @@ class DFSClientCache {
           throws IOException {
     Preconditions.checkNotNull(effectiveUser);
     Preconditions.checkNotNull(realUser);
+    realUser.checkTGTAndReloginFromKeytab();
+
     UserGroupInformation ugi =
             UserGroupInformation.createProxyUser(effectiveUser, realUser);
     if (LOG.isDebugEnabled()){

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -121,6 +121,8 @@ import org.apache.hadoop.oncrpc.security.SysSecurityHandler;
 import org.apache.hadoop.oncrpc.security.Verifier;
 import org.apache.hadoop.oncrpc.security.VerifierNone;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.Channel;
@@ -128,6 +130,9 @@ import org.jboss.netty.channel.ChannelHandlerContext;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_USER_NAME_KEY;
+
 /**
  * RPC program corresponding to nfs daemon. See {@link Nfs3}.
  */
@@ -187,6 +192,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         Nfs3Constant.FILE_DUMP_DIR_DEFAULT);
     boolean enableDump = config.getBoolean(Nfs3Constant.ENABLE_FILE_DUMP_KEY,
         Nfs3Constant.ENABLE_FILE_DUMP_DEFAULT);
+    UserGroupInformation.setConfiguration(config);
+    SecurityUtil.login(config, DFS_NFS_KEYTAB_FILE_KEY,
+            DFS_NFS_USER_NAME_KEY);
+
     if (!enableDump) {
       writeDumpDir = null;
     } else {

+ 24 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.nfs.nfs3;
 
+import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -52,6 +53,29 @@ public class TestDFSClientCache {
     assertEquals(MAX_CACHE_SIZE - 1, cache.clientCache.size());
   }
 
+  @Test
+  public void testGetUserGroupInformationSecure() throws IOException {
+    String userName = "user1";
+    String currentUser = "test-user";
+
+
+    Configuration conf = new Configuration();
+    UserGroupInformation currentUserUgi
+            = UserGroupInformation.createRemoteUser(currentUser);
+    currentUserUgi.setAuthenticationMethod(KERBEROS);
+    UserGroupInformation.setLoginUser(currentUserUgi);
+
+    DFSClientCache cache = new DFSClientCache(conf);
+    UserGroupInformation ugiResult
+            = cache.getUserGroupInformation(userName, currentUserUgi);
+
+    assertThat(ugiResult.getUserName(), is(userName));
+    assertThat(ugiResult.getRealUser(), is(currentUserUgi));
+    assertThat(
+            ugiResult.getAuthenticationMethod(),
+            is(UserGroupInformation.AuthenticationMethod.PROXY));
+  }
+
   @Test
   public void testGetUserGroupInformation() throws IOException {
     String userName = "user1";
@@ -59,8 +83,6 @@ public class TestDFSClientCache {
 
     UserGroupInformation currentUserUgi = UserGroupInformation
             .createUserForTesting(currentUser, new String[0]);
-    currentUserUgi.setAuthenticationMethod(
-            UserGroupInformation.AuthenticationMethod.KERBEROS);
     Configuration conf = new Configuration();
     conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost");
     DFSClientCache cache = new DFSClientCache(conf);

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

@@ -686,6 +686,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5857. TestWebHDFS#testNamenodeRestart fails intermittently with NPE.
     (Mit Desai via wheat9)
 
+    HDFS-5898. Allow NFS gateway to login/relogin from its kerberos keytab.
+    (Abin Shahab via atm)
+
   BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS
 
     HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9)

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

@@ -613,4 +613,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE =
       "dfs.client.hedged.read.threadpool.size";
   public static final int DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE = 0;
+  public static final String DFS_NFS_KEYTAB_FILE_KEY = "dfs.nfs.keytab.file";
+  public static final String DFS_NFS_USER_NAME_KEY = "dfs.nfs.kerberos.principal";
 }

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -1264,6 +1264,26 @@
     non-zero integer.
   </description>
 </property>
+<property>
+  <name>dfs.nfs.keytab.file</name>
+  <value></value>
+  <description>
+    *Note*: Advanced property. Change with caution.
+    This is the path to the keytab file for the hdfs-nfs gateway.
+    This is required when the cluster is kerberized.
+  </description>
+</property>
+
+<property>
+  <name>dfs.nfs.kerberos.principal</name>
+  <value></value>
+  <description>
+    *Note*: Advanced property. Change with caution.
+    This is the name of the kerberos principal. This is required when
+    the cluster is kerberized.It must be of this format:
+    nfs-gateway-user/nfs-gateway-host@kerberos-realm
+  </description>
+</property>
 
 <property>
   <name>dfs.webhdfs.enabled</name>

+ 32 - 2
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm

@@ -46,8 +46,38 @@ HDFS NFS Gateway
 
 * {Configuration}
 
-   NFS gateway can work with its default settings in most cases. However, it's
-   strongly recommended for the users to update a few configuration properties based on their use
+   The user running the NFS-gateway must be able to proxy all the users using the NFS mounts. 
+   For instance, if user 'nfsserver' is running the gateway, and users belonging to the groups 'nfs-users1'
+   and 'nfs-users2' use the NFS mounts, then in core-site.xml of the namenode, the following must be set:
+----
+<property>
+  <name>hadoop.proxyuser.nfsserver.groups</name>
+  <value>nfs-users1,nfs-users2</value>
+  <description>
+         The 'nfsserver' user is allowed to proxy all members of the 'nfs-users1' and 'nfs-users2' groups. Set this to '*' to allow nfsserver user to proxy any group.
+  </description>
+</property>
+<property>
+  <name>hadoop.proxyuser.nfsserver.hosts</name>
+  <value>nfs-client-host1.com</value>
+  <description>
+         This is the host where the nfs gateway is running. Set this to '*' to allow requests from any hosts to be proxied.
+  </description>
+</property>
+----
+   The above are the only required configuration for the NFS gateway in non-secure mode. For Kerberized
+   hadoop clusters, the following configurations need to be added to hdfs-site.xml:
+----
+<property>
+  <name>dfs.nfsgateway.keytab.file</name>
+  <value>/etc/hadoop/conf/nfsserver.keytab</value> <!-- path to the nfs gateway keytab -->
+</property>
+<property>
+  <name>dfs.nfsgateway.kerberos.principal</name>
+  <value>nfsserver/_HOST@YOUR-REALM.COM</value>
+</property>
+----
+   It's strongly recommended for the users to update a few configuration properties based on their use
    cases. All the related configuration properties can be added or updated in hdfs-site.xml.
   
    * If the client mounts the export with access time update allowed, make sure the following