瀏覽代碼

HDFS-5085. Merge change r1521607 from branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1521609 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 11 年之前
父節點
當前提交
430970f1a7
共有 30 個文件被更改,包括 738 次插入254 次删除
  1. 2 2
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java
  2. 1 1
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/AccessPrivilege.java
  3. 1 1
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java
  4. 2 2
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java
  5. 3 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java
  6. 60 44
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java
  7. 4 2
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java
  8. 12 9
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java
  9. 1 1
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java
  10. 1 1
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
  11. 53 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Credentials.java
  12. 15 25
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsGSS.java
  13. 43 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsNone.java
  14. 114 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsSys.java
  15. 10 17
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/RpcAuthInfo.java
  16. 63 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SecurityHandler.java
  17. 59 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java
  18. 49 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Verifier.java
  19. 41 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierGSS.java
  20. 41 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierNone.java
  21. 10 4
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapRequest.java
  22. 3 1
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java
  23. 3 2
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java
  24. 7 3
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java
  25. 19 18
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java
  26. 3 13
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java
  27. 2 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
  28. 112 105
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java
  30. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

+ 2 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java

@@ -19,10 +19,10 @@ package org.apache.hadoop.mount;
 
 
 import java.util.List;
 import java.util.List;
 
 
-import org.apache.hadoop.nfs.security.NfsExports;
+import org.apache.hadoop.nfs.NfsExports;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.XDR;
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
 
 
 /**
 /**
  * Helper class for sending MountResponse
  * Helper class for sending MountResponse

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/AccessPrivilege.java → hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/AccessPrivilege.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.nfs.security;
+package org.apache.hadoop.nfs;
 
 
 public enum AccessPrivilege {
 public enum AccessPrivilege {
   READ_ONLY,
   READ_ONLY,

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/NfsExports.java → hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.nfs.security;
+package org.apache.hadoop.nfs;
 
 
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.ArrayList;

+ 2 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java

@@ -147,7 +147,7 @@ public class IdUserGroup {
 
 
   synchronized public String getUserName(int uid, String unknown) {
   synchronized public String getUserName(int uid, String unknown) {
     checkAndUpdateMaps();
     checkAndUpdateMaps();
-    String uname = uidNameMap.get(Integer.valueOf(uid));
+    String uname = uidNameMap.get(uid);
     if (uname == null) {
     if (uname == null) {
       uname = unknown;
       uname = unknown;
     }
     }
@@ -156,7 +156,7 @@ public class IdUserGroup {
 
 
   synchronized public String getGroupName(int gid, String unknown) {
   synchronized public String getGroupName(int gid, String unknown) {
     checkAndUpdateMaps();
     checkAndUpdateMaps();
-    String gname = gidNameMap.get(Integer.valueOf(gid));
+    String gname = gidNameMap.get(gid);
     if (gname == null) {
     if (gname == null) {
       gname = unknown;
       gname = unknown;
     }
     }

+ 3 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java

@@ -205,4 +205,7 @@ public class Nfs3Constant {
   public static final String FILE_DUMP_DIR_DEFAULT = "/tmp/.hdfs-nfs";
   public static final String FILE_DUMP_DIR_DEFAULT = "/tmp/.hdfs-nfs";
   public static final String ENABLE_FILE_DUMP_KEY = "dfs.nfs3.enableDump";
   public static final String ENABLE_FILE_DUMP_KEY = "dfs.nfs3.enableDump";
   public static final boolean ENABLE_FILE_DUMP_DEFAULT = true;
   public static final boolean ENABLE_FILE_DUMP_DEFAULT = true;
+  
+  public final static String UNKNOWN_USER = "nobody";
+  public final static String UNKNOWN_GROUP = "nobody";
 }
 }

+ 60 - 44
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java

@@ -20,67 +20,83 @@ package org.apache.hadoop.nfs.nfs3;
 import java.net.InetAddress;
 import java.net.InetAddress;
 
 
 import org.apache.hadoop.nfs.nfs3.response.NFS3Response;
 import org.apache.hadoop.nfs.nfs3.response.NFS3Response;
-import org.apache.hadoop.oncrpc.RpcAuthSys;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.oncrpc.security.SecurityHandler;
 import org.jboss.netty.channel.Channel;
 import org.jboss.netty.channel.Channel;
 
 
 /**
 /**
  * RPC procedures as defined in RFC 1813.
  * RPC procedures as defined in RFC 1813.
  */
  */
 public interface Nfs3Interface {
 public interface Nfs3Interface {
-  
+
   /** NULL: Do nothing */
   /** NULL: Do nothing */
   public NFS3Response nullProcedure();
   public NFS3Response nullProcedure();
-  
+
   /** GETATTR: Get file attributes */
   /** GETATTR: Get file attributes */
-  public NFS3Response getattr(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response getattr(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** SETATTR: Set file attributes */
   /** SETATTR: Set file attributes */
-  public NFS3Response setattr(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response setattr(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** LOOKUP: Lookup filename */
   /** LOOKUP: Lookup filename */
-  public NFS3Response lookup(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
-  /** ACCESS: Check access permission  */
-  public NFS3Response access(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response lookup(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
+  /** ACCESS: Check access permission */
+  public NFS3Response access(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** READ: Read from file */
   /** READ: Read from file */
-  public NFS3Response read(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response read(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** WRITE: Write to file */
   /** WRITE: Write to file */
   public NFS3Response write(XDR xdr, Channel channel, int xid,
   public NFS3Response write(XDR xdr, Channel channel, int xid,
-      RpcAuthSys authSys, InetAddress client);
-  
-  /** CREATE: Create a file  */
-  public NFS3Response create(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
-  /** MKDIR: Create a directory  */
-  public NFS3Response mkdir(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
-  /** REMOVE: Remove a file  */
-  public NFS3Response remove(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
-  /** RMDIR: Remove a directory  */
-  public NFS3Response rmdir(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+      SecurityHandler securityHandler, InetAddress client);
+
+  /** CREATE: Create a file */
+  public NFS3Response create(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
+  /** MKDIR: Create a directory */
+  public NFS3Response mkdir(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
+  /** REMOVE: Remove a file */
+  public NFS3Response remove(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
+  /** RMDIR: Remove a directory */
+  public NFS3Response rmdir(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** RENAME: Rename a file or directory */
   /** RENAME: Rename a file or directory */
-  public NFS3Response rename(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
-  /** SYMLINK: Create a symbolic link  */
-  public NFS3Response symlink(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response rename(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
+  /** SYMLINK: Create a symbolic link */
+  public NFS3Response symlink(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** READDIR: Read From directory */
   /** READDIR: Read From directory */
-  public NFS3Response readdir(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
-  /** FSSTAT: Get dynamic file system information  */
-  public NFS3Response fsstat(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response readdir(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
+  /** FSSTAT: Get dynamic file system information */
+  public NFS3Response fsstat(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** FSINFO: Get static file system information */
   /** FSINFO: Get static file system information */
-  public NFS3Response fsinfo(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response fsinfo(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** PATHCONF: Retrieve POSIX information */
   /** PATHCONF: Retrieve POSIX information */
-  public NFS3Response pathconf(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
-  /** COMMIT: Commit cached data on a server to stable storage  */
-  public NFS3Response commit(XDR xdr, RpcAuthSys authSys, InetAddress client);
+  public NFS3Response pathconf(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
+  /** COMMIT: Commit cached data on a server to stable storage */
+  public NFS3Response commit(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
 }
 }

+ 4 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java

@@ -17,7 +17,9 @@
  */
  */
 package org.apache.hadoop.oncrpc;
 package org.apache.hadoop.oncrpc;
 
 
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
+import org.apache.hadoop.oncrpc.security.Verifier;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
 
 
 /** 
 /** 
  * Represents RPC message MSG_ACCEPTED reply body. See RFC 1831 for details.
  * Represents RPC message MSG_ACCEPTED reply body. See RFC 1831 for details.
@@ -54,7 +56,7 @@ public class RpcAcceptedReply extends RpcReply {
 
 
   public static RpcAcceptedReply read(int xid, RpcMessage.Type messageType,
   public static RpcAcceptedReply read(int xid, RpcMessage.Type messageType,
       ReplyState replyState, XDR xdr) {
       ReplyState replyState, XDR xdr) {
-    RpcAuthInfo verifier = RpcAuthInfo.read(xdr);
+    Verifier verifier = Verifier.readFlavorAndVerifier(xdr);
     AcceptState acceptState = AcceptState.fromValue(xdr.readInt());
     AcceptState acceptState = AcceptState.fromValue(xdr.readInt());
     return new RpcAcceptedReply(xid, messageType, replyState, verifier,
     return new RpcAcceptedReply(xid, messageType, replyState, verifier,
         acceptState);
         acceptState);

+ 12 - 9
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java

@@ -19,6 +19,8 @@ package org.apache.hadoop.oncrpc;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.oncrpc.security.Credentials;
+import org.apache.hadoop.oncrpc.security.Verifier;
 
 
 /**
 /**
  * Represents an RPC message of type RPC call as defined in RFC 1831
  * Represents an RPC message of type RPC call as defined in RFC 1831
@@ -30,11 +32,12 @@ public class RpcCall extends RpcMessage {
   private final int program;
   private final int program;
   private final int version;
   private final int version;
   private final int procedure;
   private final int procedure;
-  private final RpcAuthInfo credential;
-  private final RpcAuthInfo verifier;
+  private final Credentials credential;
+  private final Verifier verifier;
 
 
-  protected RpcCall(int xid, RpcMessage.Type messageType, int rpcVersion, int program,
-      int version, int procedure, RpcAuthInfo credential, RpcAuthInfo verifier) {
+  protected RpcCall(int xid, RpcMessage.Type messageType, int rpcVersion,
+      int program, int version, int procedure, Credentials credential,
+      Verifier verifier) {
     super(xid, messageType);
     super(xid, messageType);
     this.rpcVersion = rpcVersion;
     this.rpcVersion = rpcVersion;
     this.program = program;
     this.program = program;
@@ -79,19 +82,19 @@ public class RpcCall extends RpcMessage {
     return procedure;
     return procedure;
   }
   }
   
   
-  public RpcAuthInfo getCredential() {
+  public Credentials getCredential() {
     return credential;
     return credential;
   }
   }
 
 
-  public RpcAuthInfo getVerifier() {
+  public Verifier getVerifier() {
     return verifier;
     return verifier;
   }
   }
   
   
   public static RpcCall read(XDR xdr) {
   public static RpcCall read(XDR xdr) {
     return new RpcCall(xdr.readInt(), RpcMessage.Type.fromValue(xdr.readInt()),
     return new RpcCall(xdr.readInt(), RpcMessage.Type.fromValue(xdr.readInt()),
-        xdr.readInt(), xdr.readInt(),
-        xdr.readInt(), xdr.readInt(), RpcAuthInfo.read(xdr),
-        RpcAuthInfo.read(xdr));
+        xdr.readInt(), xdr.readInt(), xdr.readInt(), xdr.readInt(), 
+        Credentials.readFlavorAndCredentials(xdr),
+        Verifier.readFlavorAndVerifier(xdr));
   }
   }
   
   
   public static void write(XDR out, int xid, int program, int progVersion,
   public static void write(XDR out, int xid, int program, int progVersion,

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java

@@ -17,7 +17,7 @@
  */
  */
 package org.apache.hadoop.oncrpc;
 package org.apache.hadoop.oncrpc;
 
 
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
 
 
 /** 
 /** 
  * Represents RPC message MSG_DENIED reply body. See RFC 1831 for details.
  * Represents RPC message MSG_DENIED reply body. See RFC 1831 for details.

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java

@@ -280,7 +280,7 @@ public class XDR {
 
 
   public byte[] readVariableOpaque() {
   public byte[] readVariableOpaque() {
     int size = this.readInt();
     int size = this.readInt();
-    return size != 0 ? this.readFixedOpaque(size) : null;
+    return size != 0 ? this.readFixedOpaque(size) : new byte[0];
   }
   }
 
 
   public void skipVariableOpaque() {
   public void skipVariableOpaque() {

+ 53 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Credentials.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.oncrpc.security;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * Base class for all credentials. Currently we only support 3 different types
+ * of auth flavors: AUTH_NONE, AUTH_SYS, and RPCSEC_GSS.
+ */
+public abstract class Credentials extends RpcAuthInfo {
+  public static final Log LOG = LogFactory.getLog(Credentials.class);
+
+  public static Credentials readFlavorAndCredentials(XDR xdr) {
+    AuthFlavor flavor = AuthFlavor.fromValue(xdr.readInt());
+    final Credentials credentials;
+    if(flavor == AuthFlavor.AUTH_NONE) {
+      credentials = new CredentialsNone();
+    } else if(flavor == AuthFlavor.AUTH_SYS) {
+      credentials = new CredentialsSys();
+    } else if(flavor == AuthFlavor.RPCSEC_GSS) {
+      credentials = new CredentialsGSS();
+    } else {
+      throw new UnsupportedOperationException("Unsupported Credentials Flavor "
+          + flavor);
+    }
+    credentials.read(xdr);
+    return credentials;
+  }
+  
+  protected int mCredentialsLength;
+  
+  protected Credentials(AuthFlavor flavor) {
+    super(flavor);
+  }
+}

+ 15 - 25
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAuthSys.java → hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsGSS.java

@@ -15,37 +15,27 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.oncrpc;
+package org.apache.hadoop.oncrpc.security;
 
 
-/**
- * AUTH_SYS as defined in RFC 1831
- */
-public class RpcAuthSys {
-  private final int uid;
-  private final int gid;
+import org.apache.hadoop.oncrpc.XDR;
 
 
-  public RpcAuthSys(int uid, int gid) {
-    this.uid = uid;
-    this.gid = gid;
-  }
-  
-  public static RpcAuthSys from(byte[] credentials) {
-    XDR sys = new XDR(credentials);
-    sys.skip(4); // Stamp
-    sys.skipVariableOpaque(); // Machine name
-    return new RpcAuthSys(sys.readInt(), sys.readInt());
-  }
-  
-  public int getUid() {
-    return uid;
+/** Credential used by RPCSEC_GSS */
+public class CredentialsGSS extends Credentials {
+
+  public CredentialsGSS() {
+    super(AuthFlavor.RPCSEC_GSS);
   }
   }
 
 
-  public int getGid() {
-    return gid;
+  @Override
+  public void read(XDR xdr) {
+    // TODO Auto-generated method stub
+    
   }
   }
 
 
   @Override
   @Override
-  public String toString() {
-    return "(AuthSys: uid=" + uid + " gid=" + gid + ")";
+  public void write(XDR xdr) {
+    // TODO Auto-generated method stub
+    
   }
   }
+
 }
 }

+ 43 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsNone.java

@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.oncrpc.security;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+import com.google.common.base.Preconditions;
+
+/** Credential used by AUTH_NONE */
+public class CredentialsNone extends Credentials {
+
+  public CredentialsNone() {
+    super(AuthFlavor.AUTH_NONE);
+    mCredentialsLength = 0;
+  }
+
+  @Override
+  public void read(XDR xdr) {
+    mCredentialsLength = xdr.readInt();
+    Preconditions.checkState(mCredentialsLength == 0);
+  }
+
+  @Override
+  public void write(XDR xdr) {
+    Preconditions.checkState(mCredentialsLength == 0);
+    xdr.writeInt(mCredentialsLength);
+  }
+}

+ 114 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsSys.java

@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.oncrpc.security;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/** Credential used by AUTH_SYS */
+public class CredentialsSys extends Credentials {
+ 
+  private static final String HOSTNAME;
+  static {
+    try {
+      String s = InetAddress.getLocalHost().getHostName();
+      HOSTNAME = s;
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("HOSTNAME = " + HOSTNAME);
+      }
+    } catch (UnknownHostException e) {
+      LOG.error("Error setting HOSTNAME", e);
+      throw new RuntimeException(e);
+    }
+  }
+  
+  protected int mUID, mGID;
+  protected int[] mAuxGIDs;
+  protected String mHostName;
+  protected int mStamp;
+
+  public CredentialsSys() {
+    super(AuthFlavor.AUTH_SYS);
+    this.mCredentialsLength = 0;
+    this.mHostName = HOSTNAME;
+  }
+  
+  public int getGID() {
+    return mGID;
+  }
+
+  public int getUID() {
+    return mUID;
+  }
+
+  public void setGID(int gid) {
+    this.mGID = gid;
+  }
+
+  public void setUID(int uid) {
+    this.mUID = uid;
+  }
+  
+  public void setStamp(int stamp) {
+    this.mStamp = stamp;
+  }
+
+  @Override
+  public void read(XDR xdr) {
+    mCredentialsLength = xdr.readInt();
+
+    mStamp = xdr.readInt();
+    mHostName = xdr.readString();
+    mUID = xdr.readInt();
+    mGID = xdr.readInt();
+
+    int length = xdr.readInt();
+    mAuxGIDs = new int[length];
+    for (int i = 0; i < length; i++) {
+      mAuxGIDs[i] = xdr.readInt();
+    }
+  }
+
+  @Override
+  public void write(XDR xdr) {
+    // mStamp + mHostName.length + mHostName + mUID + mGID + mAuxGIDs.count
+    mCredentialsLength = 20 + mHostName.getBytes().length;
+    // mAuxGIDs
+    if (mAuxGIDs != null && mAuxGIDs.length > 0) {
+      mCredentialsLength += mAuxGIDs.length * 4;
+    }
+    xdr.writeInt(mCredentialsLength);
+    
+    xdr.writeInt(mStamp);
+    xdr.writeString(mHostName);
+    xdr.writeInt(mUID);
+    xdr.writeInt(mGID);
+    
+    if((mAuxGIDs == null) || (mAuxGIDs.length == 0)) {
+      xdr.writeInt(0);
+    } else {
+      xdr.writeInt(mAuxGIDs.length);
+      for (int i = 0; i < mAuxGIDs.length; i++) {
+        xdr.writeInt(mAuxGIDs[i]);
+      }
+    }
+  }
+
+}

+ 10 - 17
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAuthInfo.java → hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/RpcAuthInfo.java

@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.oncrpc;
+package org.apache.hadoop.oncrpc.security;
 
 
-import java.util.Arrays;
+import org.apache.hadoop.oncrpc.XDR;
 
 
 /**
 /**
- *  Authentication Info as defined in RFC 1831
+ *  Authentication Info. Base class of Verifier and Credential.
  */
  */
-public class RpcAuthInfo {
+public abstract class RpcAuthInfo {
   /** Different types of authentication as defined in RFC 1831 */
   /** Different types of authentication as defined in RFC 1831 */
   public enum AuthFlavor {
   public enum AuthFlavor {
     AUTH_NONE(0),
     AUTH_NONE(0),
@@ -52,27 +52,20 @@ public class RpcAuthInfo {
   }
   }
   
   
   private final AuthFlavor flavor;
   private final AuthFlavor flavor;
-  private final byte[] body;
   
   
-  protected RpcAuthInfo(AuthFlavor flavor, byte[] body) {
+  protected RpcAuthInfo(AuthFlavor flavor) {
     this.flavor = flavor;
     this.flavor = flavor;
-    this.body = body;
   }
   }
   
   
-  public static RpcAuthInfo read(XDR xdr) {
-    int type = xdr.readInt();
-    AuthFlavor flavor = AuthFlavor.fromValue(type);
-    byte[] body = xdr.readVariableOpaque();
-    return new RpcAuthInfo(flavor, body);
-  }
+  /** Load auth info */
+  public abstract void read(XDR xdr);
+  
+  /** Write auth info */
+  public abstract void write(XDR xdr);
   
   
   public AuthFlavor getFlavor() {
   public AuthFlavor getFlavor() {
     return flavor;
     return flavor;
   }
   }
-
-  public byte[] getBody() {
-    return Arrays.copyOf(body, body.length);
-  }
   
   
   @Override
   @Override
   public String toString() {
   public String toString() {

+ 63 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SecurityHandler.java

@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.oncrpc.security;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.oncrpc.RpcCall;
+import org.apache.hadoop.oncrpc.XDR;
+
+public abstract class SecurityHandler {
+  public static final Log LOG = LogFactory.getLog(SecurityHandler.class);
+  
+  public abstract String getUser();
+
+  public abstract boolean shouldSilentlyDrop(RpcCall request);
+
+  public abstract Verifier getVerifer(RpcCall request) throws IOException;
+
+  public boolean isUnwrapRequired() {
+    return false;
+  }
+
+  public boolean isWrapRequired() {
+    return false;
+  }
+
+  /** Used by GSS */
+  public XDR unwrap(RpcCall request, byte[] data ) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /** Used by GSS */
+  public byte[] wrap(RpcCall request, XDR response) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /** Used by AUTH_SYS */
+  public int getUid() {
+    throw new UnsupportedOperationException();
+  }
+  
+  /** Used by AUTH_SYS */
+  public int getGid() {
+    throw new UnsupportedOperationException();
+  }
+}

+ 59 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java

@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.oncrpc.security;
+
+import org.apache.hadoop.nfs.nfs3.IdUserGroup;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.oncrpc.RpcCall;
+
+public class SysSecurityHandler extends SecurityHandler {
+  
+  private final IdUserGroup iug;
+  private final CredentialsSys mCredentialsSys;
+  
+  public SysSecurityHandler(CredentialsSys credentialsSys,
+      IdUserGroup iug) {
+    this.mCredentialsSys = credentialsSys;
+    this.iug = iug;
+  }
+  
+  @Override
+  public String getUser() {
+    return iug.getUserName(mCredentialsSys.getUID(), Nfs3Constant.UNKNOWN_USER);
+  }
+
+  @Override
+  public boolean shouldSilentlyDrop(RpcCall request) {
+    return false;
+  }
+
+  @Override
+  public VerifierNone getVerifer(RpcCall request) {
+    return new VerifierNone();
+  }
+  
+  @Override
+  public int getUid() {
+    return mCredentialsSys.getUID();
+  }
+  
+  @Override
+  public int getGid() {
+    return mCredentialsSys.getGID();
+  }
+}

+ 49 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Verifier.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.oncrpc.security;
+
+import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
+
+/** 
+ * Base class for verifier. Currently we only support 3 types of auth flavors: 
+ * {@link AuthFlavor#AUTH_NONE}, {@link AuthFlavor#AUTH_SYS}, 
+ * and {@link AuthFlavor#RPCSEC_GSS}.
+ */
+public abstract class Verifier extends RpcAuthInfo {
+
+  protected Verifier(AuthFlavor flavor) {
+    super(flavor);
+  }
+
+  public static Verifier readFlavorAndVerifier(XDR xdr) {
+    AuthFlavor flavor = AuthFlavor.fromValue(xdr.readInt());
+    final Verifier verifer;
+    if(flavor == AuthFlavor.AUTH_NONE) {
+      verifer = new VerifierNone();
+    } else if(flavor == AuthFlavor.RPCSEC_GSS) {
+      verifer = new VerifierGSS();
+    } else {
+      throw new UnsupportedOperationException("Unsupported verifier flavor"
+          + flavor);
+    }
+    verifer.read(xdr);
+    return verifer;
+  }
+  
+}

+ 41 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierGSS.java

@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.oncrpc.security;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/** Verifier mapped to RPCSEC_GSS. */
+public class VerifierGSS extends Verifier {
+
+  public VerifierGSS() {
+    super(AuthFlavor.RPCSEC_GSS);
+  }
+
+  @Override
+  public void read(XDR xdr) {
+    // TODO Auto-generated method stub
+    
+  }
+
+  @Override
+  public void write(XDR xdr) {
+    // TODO Auto-generated method stub
+    
+  }
+
+}

+ 41 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierNone.java

@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.oncrpc.security;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+import com.google.common.base.Preconditions;
+
+/** Verifier used by AUTH_NONE. */
+public class VerifierNone extends Verifier {
+
+  public VerifierNone() {
+    super(AuthFlavor.AUTH_NONE);
+  }
+
+  @Override
+  public void read(XDR xdr) {
+    int length = xdr.readInt();
+    Preconditions.checkState(length == 0);
+  }
+
+  @Override
+  public void write(XDR xdr) {
+    xdr.writeInt(0);
+  }
+}

+ 10 - 4
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapRequest.java

@@ -17,10 +17,14 @@
  */
  */
 package org.apache.hadoop.portmap;
 package org.apache.hadoop.portmap;
 
 
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
 import org.apache.hadoop.oncrpc.RpcCall;
 import org.apache.hadoop.oncrpc.RpcCall;
 import org.apache.hadoop.oncrpc.RpcUtil;
 import org.apache.hadoop.oncrpc.RpcUtil;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.oncrpc.security.CredentialsNone;
+import org.apache.hadoop.oncrpc.security.Credentials;
+import org.apache.hadoop.oncrpc.security.Verifier;
+import org.apache.hadoop.oncrpc.security.VerifierNone;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
 import org.apache.hadoop.portmap.PortmapInterface.Procedure;
 import org.apache.hadoop.portmap.PortmapInterface.Procedure;
 
 
 /**
 /**
@@ -38,9 +42,11 @@ public class PortmapRequest {
         RpcProgramPortmap.PROGRAM, RpcProgramPortmap.VERSION,
         RpcProgramPortmap.PROGRAM, RpcProgramPortmap.VERSION,
         Procedure.PMAPPROC_SET.getValue());
         Procedure.PMAPPROC_SET.getValue());
     request.writeInt(AuthFlavor.AUTH_NONE.getValue());
     request.writeInt(AuthFlavor.AUTH_NONE.getValue());
-    request.writeInt(0);
-    request.writeInt(0);
-    request.writeInt(0);
+    Credentials credential = new CredentialsNone();
+    credential.write(request);
+    request.writeInt(AuthFlavor.AUTH_NONE.getValue());
+    Verifier verifier = new VerifierNone();
+    verifier.write(request);
     return mapping.serialize(request);
     return mapping.serialize(request);
   }
   }
 }
 }

+ 3 - 1
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/security/TestNfsExports.java → hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java

@@ -15,10 +15,12 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.nfs.security;
+package org.apache.hadoop.nfs;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
+import org.apache.hadoop.nfs.AccessPrivilege;
+import org.apache.hadoop.nfs.NfsExports;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
 import org.junit.Test;
 import org.junit.Test;
 
 

+ 3 - 2
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java

@@ -20,8 +20,9 @@ package org.apache.hadoop.oncrpc;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 
 
 import org.apache.hadoop.oncrpc.RpcAcceptedReply.AcceptState;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply.AcceptState;
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
 import org.apache.hadoop.oncrpc.RpcReply.ReplyState;
 import org.apache.hadoop.oncrpc.RpcReply.ReplyState;
+import org.apache.hadoop.oncrpc.security.Verifier;
+import org.apache.hadoop.oncrpc.security.VerifierNone;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /**
 /**
@@ -45,7 +46,7 @@ public class TestRpcAcceptedReply {
   
   
   @Test
   @Test
   public void testConstructor() {
   public void testConstructor() {
-    RpcAuthInfo verifier = new RpcAuthInfo(AuthFlavor.AUTH_NONE, new byte[0]);
+    Verifier verifier = new VerifierNone();
     RpcAcceptedReply reply = new RpcAcceptedReply(0, RpcMessage.Type.RPC_REPLY,
     RpcAcceptedReply reply = new RpcAcceptedReply(0, RpcMessage.Type.RPC_REPLY,
         ReplyState.MSG_ACCEPTED, verifier, AcceptState.SUCCESS);
         ReplyState.MSG_ACCEPTED, verifier, AcceptState.SUCCESS);
     assertEquals(0, reply.getXid());
     assertEquals(0, reply.getXid());

+ 7 - 3
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java

@@ -17,8 +17,12 @@
  */
  */
 package org.apache.hadoop.oncrpc;
 package org.apache.hadoop.oncrpc;
 
 
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.oncrpc.security.CredentialsNone;
+import org.apache.hadoop.oncrpc.security.Credentials;
+import org.apache.hadoop.oncrpc.security.Verifier;
+import org.apache.hadoop.oncrpc.security.VerifierNone;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /**
 /**
@@ -28,8 +32,8 @@ public class TestRpcCall {
   
   
   @Test
   @Test
   public void testConstructor() {
   public void testConstructor() {
-    RpcAuthInfo credential = new RpcAuthInfo(AuthFlavor.AUTH_NONE, new byte[0]);
-    RpcAuthInfo verifier = new RpcAuthInfo(AuthFlavor.AUTH_NONE, new byte[0]);
+    Credentials credential = new CredentialsNone();
+    Verifier verifier = new VerifierNone();
     int rpcVersion = RpcCall.RPC_VERSION;
     int rpcVersion = RpcCall.RPC_VERSION;
     int program = 2;
     int program = 2;
     int version = 3;
     int version = 3;

+ 19 - 18
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAuthSys.java → hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java

@@ -15,31 +15,32 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.oncrpc;
+package org.apache.hadoop.oncrpc.security;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 
 
+import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.oncrpc.security.CredentialsSys;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /**
 /**
- * Test for {@link RpcAuthSys}
+ * Test for {@link CredentialsSys}
  */
  */
-public class TestRpcAuthSys {
-  @Test
-  public void testConstructor() {
-    RpcAuthSys auth = new RpcAuthSys(0, 1);
-    assertEquals(0, auth.getUid());
-    assertEquals(1, auth.getGid());
-  }
-  
+public class TestCredentialsSys {
+
   @Test
   @Test
-  public void testRead() {
-    byte[] bytes = {0, 1, 2, 3}; // 4 bytes Stamp
-    bytes = XDR.append(bytes, XDR.getVariableOpque(new byte[0]));
-    bytes = XDR.append(bytes, XDR.toBytes(0)); // gid
-    bytes = XDR.append(bytes, XDR.toBytes(1)); // uid
-    RpcAuthSys auth = RpcAuthSys.from(bytes);
-    assertEquals(0, auth.getUid());
-    assertEquals(1, auth.getGid());
+  public void testReadWrite() {
+    CredentialsSys credential = new CredentialsSys();
+    credential.setUID(0);
+    credential.setGID(1);
+    
+    XDR xdr = new XDR();
+    credential.write(xdr);
+    
+    CredentialsSys newCredential = new CredentialsSys();
+    newCredential.read(xdr);
+    
+    assertEquals(0, newCredential.getUID());
+    assertEquals(1, newCredential.getGID());
   }
   }
 }
 }

+ 3 - 13
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAuthInfo.java → hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java

@@ -15,14 +15,12 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.oncrpc;
+package org.apache.hadoop.oncrpc.security;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 
-import java.util.Arrays;
-
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /**
 /**
@@ -42,12 +40,4 @@ public class TestRpcAuthInfo {
   public void testInvalidAuthFlavor() {
   public void testInvalidAuthFlavor() {
     assertEquals(AuthFlavor.AUTH_NONE, AuthFlavor.fromValue(4));
     assertEquals(AuthFlavor.AUTH_NONE, AuthFlavor.fromValue(4));
   }
   }
-  
-  @Test
-  public void testConsturctor() {
-    byte[] body = new byte[0];
-    RpcAuthInfo auth = new RpcAuthInfo(AuthFlavor.AUTH_NONE, body);
-    assertEquals(AuthFlavor.AUTH_NONE, auth.getFlavor());
-    assertTrue(Arrays.equals(body, auth.getBody()));
-  }
 }
 }

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

@@ -32,10 +32,10 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.mount.MountEntry;
 import org.apache.hadoop.mount.MountEntry;
 import org.apache.hadoop.mount.MountInterface;
 import org.apache.hadoop.mount.MountInterface;
 import org.apache.hadoop.mount.MountResponse;
 import org.apache.hadoop.mount.MountResponse;
+import org.apache.hadoop.nfs.AccessPrivilege;
+import org.apache.hadoop.nfs.NfsExports;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.Nfs3Status;
 import org.apache.hadoop.nfs.nfs3.Nfs3Status;
-import org.apache.hadoop.nfs.security.AccessPrivilege;
-import org.apache.hadoop.nfs.security.NfsExports;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply;
 import org.apache.hadoop.oncrpc.RpcCall;
 import org.apache.hadoop.oncrpc.RpcCall;
 import org.apache.hadoop.oncrpc.RpcProgram;
 import org.apache.hadoop.oncrpc.RpcProgram;

+ 112 - 105
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -42,6 +42,8 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.nfs.AccessPrivilege;
+import org.apache.hadoop.nfs.NfsExports;
 import org.apache.hadoop.nfs.NfsTime;
 import org.apache.hadoop.nfs.NfsTime;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.IdUserGroup;
 import org.apache.hadoop.nfs.nfs3.IdUserGroup;
@@ -96,16 +98,18 @@ import org.apache.hadoop.nfs.nfs3.response.VoidResponse;
 import org.apache.hadoop.nfs.nfs3.response.WRITE3Response;
 import org.apache.hadoop.nfs.nfs3.response.WRITE3Response;
 import org.apache.hadoop.nfs.nfs3.response.WccAttr;
 import org.apache.hadoop.nfs.nfs3.response.WccAttr;
 import org.apache.hadoop.nfs.nfs3.response.WccData;
 import org.apache.hadoop.nfs.nfs3.response.WccData;
-import org.apache.hadoop.nfs.security.AccessPrivilege;
-import org.apache.hadoop.nfs.security.NfsExports;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply;
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
-import org.apache.hadoop.oncrpc.RpcAuthSys;
 import org.apache.hadoop.oncrpc.RpcCall;
 import org.apache.hadoop.oncrpc.RpcCall;
 import org.apache.hadoop.oncrpc.RpcDeniedReply;
 import org.apache.hadoop.oncrpc.RpcDeniedReply;
 import org.apache.hadoop.oncrpc.RpcProgram;
 import org.apache.hadoop.oncrpc.RpcProgram;
 import org.apache.hadoop.oncrpc.RpcReply;
 import org.apache.hadoop.oncrpc.RpcReply;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.oncrpc.security.CredentialsSys;
+import org.apache.hadoop.oncrpc.security.Credentials;
+import org.apache.hadoop.oncrpc.security.Verifier;
+import org.apache.hadoop.oncrpc.security.SecurityHandler;
+import org.apache.hadoop.oncrpc.security.SysSecurityHandler;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 import org.jboss.netty.channel.Channel;
 import org.jboss.netty.channel.Channel;
 
 
@@ -205,8 +209,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public GETATTR3Response getattr(XDR xdr, RpcAuthSys authSys,
-      InetAddress client) {
+  public GETATTR3Response getattr(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     GETATTR3Response response = new GETATTR3Response(Nfs3Status.NFS3_OK);
     GETATTR3Response response = new GETATTR3Response(Nfs3Status.NFS3_OK);
     
     
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
@@ -214,8 +218,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
       return response;
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -268,9 +271,9 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     if (updateFields.contains(SetAttrField.UID)
     if (updateFields.contains(SetAttrField.UID)
         || updateFields.contains(SetAttrField.GID)) {
         || updateFields.contains(SetAttrField.GID)) {
       String uname = updateFields.contains(SetAttrField.UID) ? iug.getUserName(
       String uname = updateFields.contains(SetAttrField.UID) ? iug.getUserName(
-          newAttr.getUid(), UNKNOWN_USER) : null;
+          newAttr.getUid(), Nfs3Constant.UNKNOWN_USER) : null;
       String gname = updateFields.contains(SetAttrField.GID) ? iug
       String gname = updateFields.contains(SetAttrField.GID) ? iug
-          .getGroupName(newAttr.getGid(), UNKNOWN_GROUP) : null;
+          .getGroupName(newAttr.getGid(), Nfs3Constant.UNKNOWN_GROUP) : null;
       dfsClient.setOwner(fileIdPath, uname, gname);
       dfsClient.setOwner(fileIdPath, uname, gname);
     }
     }
 
 
@@ -287,11 +290,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public SETATTR3Response setattr(XDR xdr, RpcAuthSys authSys,
-      InetAddress client) {
+  public SETATTR3Response setattr(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     SETATTR3Response response = new SETATTR3Response(Nfs3Status.NFS3_OK);
     SETATTR3Response response = new SETATTR3Response(Nfs3Status.NFS3_OK);
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -364,7 +366,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public LOOKUP3Response lookup(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public LOOKUP3Response lookup(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     LOOKUP3Response response = new LOOKUP3Response(Nfs3Status.NFS3_OK);
     LOOKUP3Response response = new LOOKUP3Response(Nfs3Status.NFS3_OK);
     
     
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
@@ -372,8 +375,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
       return response;
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -426,7 +428,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
   
   
   @Override
   @Override
-  public ACCESS3Response access(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public ACCESS3Response access(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     ACCESS3Response response = new ACCESS3Response(Nfs3Status.NFS3_OK);
     ACCESS3Response response = new ACCESS3Response(Nfs3Status.NFS3_OK);
     
     
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
@@ -434,8 +437,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
       return response;
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -464,8 +466,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.error("Can't get path for fileId:" + handle.getFileId());
         LOG.error("Can't get path for fileId:" + handle.getFileId());
         return new ACCESS3Response(Nfs3Status.NFS3ERR_STALE);
         return new ACCESS3Response(Nfs3Status.NFS3ERR_STALE);
       }
       }
-      int access = Nfs3Utils.getAccessRightsForUserGroup(authSys.getUid(),
-          authSys.getGid(), attrs);
+      int access = Nfs3Utils.getAccessRightsForUserGroup(
+          securityHandler.getUid(), securityHandler.getGid(), attrs);
       
       
       return new ACCESS3Response(Nfs3Status.NFS3_OK, attrs, access);
       return new ACCESS3Response(Nfs3Status.NFS3_OK, attrs, access);
     } catch (IOException e) {
     } catch (IOException e) {
@@ -474,13 +476,14 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
     }
   }
   }
 
 
-  public READLINK3Response readlink(XDR xdr, RpcAuthSys authSys,
-      InetAddress client) {
+  public READLINK3Response readlink(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     return new READLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
     return new READLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
   }
   }
 
 
   @Override
   @Override
-  public READ3Response read(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public READ3Response read(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client) {
     READ3Response response = new READ3Response(Nfs3Status.NFS3_OK);
     READ3Response response = new READ3Response(Nfs3Status.NFS3_OK);
     
     
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
@@ -488,8 +491,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
       return response;
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -534,8 +536,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         }
         }
         return new READ3Response(Nfs3Status.NFS3ERR_NOENT);
         return new READ3Response(Nfs3Status.NFS3ERR_NOENT);
       }
       }
-      int access = Nfs3Utils.getAccessRightsForUserGroup(authSys.getUid(),
-          authSys.getGid(), attrs);
+      int access = Nfs3Utils.getAccessRightsForUserGroup(
+          securityHandler.getUid(), securityHandler.getGid(), attrs);
       if ((access & Nfs3Constant.ACCESS3_READ) != 0) {
       if ((access & Nfs3Constant.ACCESS3_READ) != 0) {
         eof = offset < attrs.getSize() ? false : true;
         eof = offset < attrs.getSize() ? false : true;
         return new READ3Response(Nfs3Status.NFS3_OK, attrs, 0, eof,
         return new READ3Response(Nfs3Status.NFS3_OK, attrs, 0, eof,
@@ -578,10 +580,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
 
 
   @Override
   @Override
   public WRITE3Response write(XDR xdr, Channel channel, int xid,
   public WRITE3Response write(XDR xdr, Channel channel, int xid,
-      RpcAuthSys authSys, InetAddress client) {
+      SecurityHandler securityHandler, InetAddress client) {
     WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK);
     WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK);
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -653,10 +655,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public CREATE3Response create(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public CREATE3Response create(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     CREATE3Response response = new CREATE3Response(Nfs3Status.NFS3_OK);
     CREATE3Response response = new CREATE3Response(Nfs3Status.NFS3_OK);
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -725,7 +727,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         // Set group if it's not specified in the request.
         // Set group if it's not specified in the request.
         if (!setAttr3.getUpdateFields().contains(SetAttrField.GID)) {
         if (!setAttr3.getUpdateFields().contains(SetAttrField.GID)) {
           setAttr3.getUpdateFields().add(SetAttrField.GID);
           setAttr3.getUpdateFields().add(SetAttrField.GID);
-          setAttr3.setGid(authSys.getGid());
+          setAttr3.setGid(securityHandler.getGid());
         }
         }
         setattrInternal(dfsClient, fileIdPath, setAttr3, false);
         setattrInternal(dfsClient, fileIdPath, setAttr3, false);
       }
       }
@@ -776,10 +778,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public MKDIR3Response mkdir(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public MKDIR3Response mkdir(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client) {
     MKDIR3Response response = new MKDIR3Response(Nfs3Status.NFS3_OK);
     MKDIR3Response response = new MKDIR3Response(Nfs3Status.NFS3_OK);
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -834,7 +836,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       // Set group if it's not specified in the request.
       // Set group if it's not specified in the request.
       if (!setAttr3.getUpdateFields().contains(SetAttrField.GID)) {
       if (!setAttr3.getUpdateFields().contains(SetAttrField.GID)) {
         setAttr3.getUpdateFields().add(SetAttrField.GID);
         setAttr3.getUpdateFields().add(SetAttrField.GID);
-        setAttr3.setGid(authSys.getGid());
+        setAttr3.setGid(securityHandler.getGid());
       }
       }
       setattrInternal(dfsClient, fileIdPath, setAttr3, false);
       setattrInternal(dfsClient, fileIdPath, setAttr3, false);
       
       
@@ -866,15 +868,16 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
     }
   }
   }
 
 
-  public READDIR3Response mknod(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public READDIR3Response mknod(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     return new READDIR3Response(Nfs3Status.NFS3ERR_NOTSUPP);
     return new READDIR3Response(Nfs3Status.NFS3ERR_NOTSUPP);
   }
   }
   
   
   @Override
   @Override
-  public REMOVE3Response remove(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public REMOVE3Response remove(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     REMOVE3Response response = new REMOVE3Response(Nfs3Status.NFS3_OK);
     REMOVE3Response response = new REMOVE3Response(Nfs3Status.NFS3_OK);
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -947,10 +950,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public RMDIR3Response rmdir(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public RMDIR3Response rmdir(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client) {
     RMDIR3Response response = new RMDIR3Response(Nfs3Status.NFS3_OK);
     RMDIR3Response response = new RMDIR3Response(Nfs3Status.NFS3_OK);
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -1030,10 +1033,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public RENAME3Response rename(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public RENAME3Response rename(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     RENAME3Response response = new RENAME3Response(Nfs3Status.NFS3_OK);
     RENAME3Response response = new RENAME3Response(Nfs3Status.NFS3_OK);
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -1118,18 +1121,18 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public SYMLINK3Response symlink(XDR xdr, RpcAuthSys authSys,
-      InetAddress client) {
+  public SYMLINK3Response symlink(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     return new SYMLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
     return new SYMLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
   }
   }
 
 
-  public READDIR3Response link(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public READDIR3Response link(XDR xdr, SecurityHandler securityHandler, InetAddress client) {
     return new READDIR3Response(Nfs3Status.NFS3ERR_NOTSUPP);
     return new READDIR3Response(Nfs3Status.NFS3ERR_NOTSUPP);
   }
   }
 
 
   @Override
   @Override
-  public READDIR3Response readdir(XDR xdr, RpcAuthSys authSys,
-      InetAddress client) {
+  public READDIR3Response readdir(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     READDIR3Response response = new READDIR3Response(Nfs3Status.NFS3_OK);
     READDIR3Response response = new READDIR3Response(Nfs3Status.NFS3_OK);
     
     
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
@@ -1137,8 +1140,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
       return response;
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -1269,14 +1271,13 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         dirStatus.getModificationTime(), dirList);
         dirStatus.getModificationTime(), dirList);
   }
   }
 
 
-  public READDIRPLUS3Response readdirplus(XDR xdr, RpcAuthSys authSys,
-      InetAddress client) {
+  public READDIRPLUS3Response readdirplus(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
       return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_ACCES);
       return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_ACCES);
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
       return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
     }
     }
@@ -1420,7 +1421,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
   
   
   @Override
   @Override
-  public FSSTAT3Response fsstat(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public FSSTAT3Response fsstat(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     FSSTAT3Response response = new FSSTAT3Response(Nfs3Status.NFS3_OK);
     FSSTAT3Response response = new FSSTAT3Response(Nfs3Status.NFS3_OK);
     
     
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
@@ -1428,8 +1430,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
       return response;
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -1478,7 +1479,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public FSINFO3Response fsinfo(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public FSINFO3Response fsinfo(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     FSINFO3Response response = new FSINFO3Response(Nfs3Status.NFS3_OK);
     FSINFO3Response response = new FSINFO3Response(Nfs3Status.NFS3_OK);
     
     
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
@@ -1486,8 +1488,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
       return response;
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -1530,8 +1531,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public PATHCONF3Response pathconf(XDR xdr, RpcAuthSys authSys,
-      InetAddress client) {
+  public PATHCONF3Response pathconf(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     PATHCONF3Response response = new PATHCONF3Response(Nfs3Status.NFS3_OK);
     PATHCONF3Response response = new PATHCONF3Response(Nfs3Status.NFS3_OK);
     
     
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
     if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
@@ -1539,8 +1540,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
       return response;
     }
     }
     
     
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -1578,10 +1578,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public COMMIT3Response commit(XDR xdr, RpcAuthSys authSys, InetAddress client) {
+  public COMMIT3Response commit(XDR xdr,
+      SecurityHandler securityHandler, InetAddress client) {
     COMMIT3Response response = new COMMIT3Response(Nfs3Status.NFS3_OK);
     COMMIT3Response response = new COMMIT3Response(Nfs3Status.NFS3_OK);
-    String uname = authSysCheck(authSys);
-    DFSClient dfsClient = clientCache.get(uname);
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
     if (dfsClient == null) {
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
       return response;
       return response;
@@ -1645,12 +1645,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
           Nfs3Constant.WRITE_COMMIT_VERF);
           Nfs3Constant.WRITE_COMMIT_VERF);
     }
     }
   }
   }
-  
-  private final static String UNKNOWN_USER = "nobody";
-  private final static String UNKNOWN_GROUP = "nobody";
 
 
-  private String authSysCheck(RpcAuthSys authSys) {
-    return iug.getUserName(authSys.getUid(), UNKNOWN_USER);
+  private SecurityHandler getSecurityHandler(Credentials credentials,
+      Verifier verifier) {
+    if (credentials instanceof CredentialsSys) {
+      return new SysSecurityHandler((CredentialsSys) credentials, iug);
+    } else {
+      // TODO: support GSS and handle other cases
+      return null;
+    }
   }
   }
   
   
   @Override
   @Override
@@ -1658,67 +1661,71 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       InetAddress client, Channel channel) {
       InetAddress client, Channel channel) {
     final NFSPROC3 nfsproc3 = NFSPROC3.fromValue(rpcCall.getProcedure());
     final NFSPROC3 nfsproc3 = NFSPROC3.fromValue(rpcCall.getProcedure());
     int xid = rpcCall.getXid();
     int xid = rpcCall.getXid();
-    RpcAuthSys authSys = null;
-    
+
+    Credentials credentials = rpcCall.getCredential();
     // Ignore auth only for NFSPROC3_NULL, especially for Linux clients.
     // Ignore auth only for NFSPROC3_NULL, especially for Linux clients.
     if (nfsproc3 != NFSPROC3.NULL) {
     if (nfsproc3 != NFSPROC3.NULL) {
-      if (rpcCall.getCredential().getFlavor() != AuthFlavor.AUTH_SYS) {
+      if (rpcCall.getCredential().getFlavor() != AuthFlavor.AUTH_SYS
+          && rpcCall.getCredential().getFlavor() != AuthFlavor.RPCSEC_GSS) {
         LOG.info("Wrong RPC AUTH flavor, "
         LOG.info("Wrong RPC AUTH flavor, "
-            + rpcCall.getCredential().getFlavor() + " is not AUTH_SYS.");
+            + rpcCall.getCredential().getFlavor()
+            + " is not AUTH_SYS or RPCSEC_GSS.");
         XDR reply = new XDR();
         XDR reply = new XDR();
         reply = RpcDeniedReply.voidReply(reply, xid,
         reply = RpcDeniedReply.voidReply(reply, xid,
             RpcReply.ReplyState.MSG_ACCEPTED,
             RpcReply.ReplyState.MSG_ACCEPTED,
             RpcDeniedReply.RejectState.AUTH_ERROR);
             RpcDeniedReply.RejectState.AUTH_ERROR);
         return reply;
         return reply;
       }
       }
-      authSys = RpcAuthSys.from(rpcCall.getCredential().getBody());
     }
     }
     
     
+    SecurityHandler securityHandler = getSecurityHandler(credentials,
+        rpcCall.getVerifier());
+    
     NFS3Response response = null;
     NFS3Response response = null;
     if (nfsproc3 == NFSPROC3.NULL) {
     if (nfsproc3 == NFSPROC3.NULL) {
       response = nullProcedure();
       response = nullProcedure();
     } else if (nfsproc3 == NFSPROC3.GETATTR) {
     } else if (nfsproc3 == NFSPROC3.GETATTR) {
-      response = getattr(xdr, authSys, client);
+      response = getattr(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.SETATTR) {
     } else if (nfsproc3 == NFSPROC3.SETATTR) {
-      response = setattr(xdr, authSys, client);
+      response = setattr(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.LOOKUP) {
     } else if (nfsproc3 == NFSPROC3.LOOKUP) {
-      response = lookup(xdr, authSys, client);
+      response = lookup(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.ACCESS) {
     } else if (nfsproc3 == NFSPROC3.ACCESS) {
-      response = access(xdr, authSys, client);
+      response = access(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.READLINK) {
     } else if (nfsproc3 == NFSPROC3.READLINK) {
-      response = readlink(xdr, authSys, client);
+      response = readlink(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.READ) {
     } else if (nfsproc3 == NFSPROC3.READ) {
-      response = read(xdr, authSys, client);
+      response = read(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.WRITE) {
     } else if (nfsproc3 == NFSPROC3.WRITE) {
-      response = write(xdr, channel, xid, authSys, client);
+      response = write(xdr, channel, xid, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.CREATE) {
     } else if (nfsproc3 == NFSPROC3.CREATE) {
-      response = create(xdr, authSys, client);
+      response = create(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.MKDIR) {      
     } else if (nfsproc3 == NFSPROC3.MKDIR) {      
-      response = mkdir(xdr, authSys, client);
+      response = mkdir(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.SYMLINK) {
     } else if (nfsproc3 == NFSPROC3.SYMLINK) {
-      response = symlink(xdr, authSys, client);
+      response = symlink(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.MKNOD) {
     } else if (nfsproc3 == NFSPROC3.MKNOD) {
-      response = mknod(xdr, authSys, client);
+      response = mknod(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.REMOVE) {
     } else if (nfsproc3 == NFSPROC3.REMOVE) {
-      response = remove(xdr, authSys, client);
+      response = remove(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.RMDIR) {
     } else if (nfsproc3 == NFSPROC3.RMDIR) {
-      response = rmdir(xdr, authSys, client);
+      response = rmdir(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.RENAME) {
     } else if (nfsproc3 == NFSPROC3.RENAME) {
-      response = rename(xdr, authSys, client);
+      response = rename(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.LINK) {
     } else if (nfsproc3 == NFSPROC3.LINK) {
-      response = link(xdr, authSys, client);
+      response = link(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.READDIR) {
     } else if (nfsproc3 == NFSPROC3.READDIR) {
-      response = readdir(xdr, authSys, client);
+      response = readdir(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.READDIRPLUS) {
     } else if (nfsproc3 == NFSPROC3.READDIRPLUS) {
-      response = readdirplus(xdr, authSys, client);
+      response = readdirplus(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.FSSTAT) {
     } else if (nfsproc3 == NFSPROC3.FSSTAT) {
-      response = fsstat(xdr, authSys, client);
+      response = fsstat(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.FSINFO) {
     } else if (nfsproc3 == NFSPROC3.FSINFO) {
-      response = fsinfo(xdr, authSys, client);
+      response = fsinfo(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.PATHCONF) {
     } else if (nfsproc3 == NFSPROC3.PATHCONF) {
-      response = pathconf(xdr, authSys, client);
+      response = pathconf(xdr, securityHandler, client);
     } else if (nfsproc3 == NFSPROC3.COMMIT) {
     } else if (nfsproc3 == NFSPROC3.COMMIT) {
-      response = commit(xdr, authSys, client);
+      response = commit(xdr, securityHandler, client);
     } else {
     } else {
       // Invalid procedure
       // Invalid procedure
       RpcAcceptedReply.voidReply(out, xid,
       RpcAcceptedReply.voidReply(out, xid,

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java

@@ -42,7 +42,7 @@ public class TestMountd {
     // Start minicluster
     // Start minicluster
     Configuration config = new Configuration();
     Configuration config = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(config).numDataNodes(1)
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(config).numDataNodes(1)
-        .manageNameDfsDirs(false).build();
+        .build();
     cluster.waitActive();
     cluster.waitActive();
     
     
     // Start nfs
     // Start nfs

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

@@ -38,6 +38,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5118. Provide testing support for DFSClient to drop RPC responses.
     HDFS-5118. Provide testing support for DFSClient to drop RPC responses.
     (jing9)
     (jing9)
 
 
+    HDFS-5085. Refactor o.a.h.nfs to support different types of 
+    authentications. (jing9)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may