You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2013/09/10 21:29:46 UTC

svn commit: r1521601 - in /hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src: main/java/org/apache/hadoop/mount/ main/java/org/apache/hadoop/nfs/ main/java/org/apache/hadoop/nfs/nfs3/ main/java/org/apache/hadoop/nfs/security/ main/java/org/apach...

Author: jing9
Date: Tue Sep 10 19:29:45 2013
New Revision: 1521601

URL: http://svn.apache.org/r1521601
Log:
HDFS-5085. Refactor o.a.h.nfs to support different types of authentications. Contributed by Jing Zhao.

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/AccessPrivilege.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Credentials.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsGSS.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsNone.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsSys.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/RpcAuthInfo.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SecurityHandler.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Verifier.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierGSS.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierNone.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java
Removed:
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/AccessPrivilege.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/NfsExports.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAuthInfo.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAuthSys.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/security/TestNfsExports.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAuthInfo.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAuthSys.java
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapRequest.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java Tue Sep 10 19:29:45 2013
@@ -19,10 +19,10 @@ package org.apache.hadoop.mount;
 
 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.XDR;
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
 
 /**
  * Helper class for sending MountResponse

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/AccessPrivilege.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/AccessPrivilege.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/AccessPrivilege.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/AccessPrivilege.java Tue Sep 10 19:29:45 2013
@@ -0,0 +1,24 @@
+/**
+ * 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.nfs;
+
+public enum AccessPrivilege {
+  READ_ONLY,
+  READ_WRITE,
+  NONE;
+}
\ No newline at end of file

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java Tue Sep 10 19:29:45 2013
@@ -0,0 +1,388 @@
+/**
+ * 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.nfs;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.net.util.SubnetUtils;
+import org.apache.commons.net.util.SubnetUtils.SubnetInfo;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.util.LightWeightCache;
+import org.apache.hadoop.util.LightWeightGSet;
+import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class provides functionality for loading and checking the mapping 
+ * between client hosts and their access privileges.
+ */
+public class NfsExports {
+  
+  private static NfsExports exports = null;
+  
+  public static synchronized NfsExports getInstance(Configuration conf) {
+    if (exports == null) {
+      String matchHosts = conf.get(Nfs3Constant.EXPORTS_ALLOWED_HOSTS_KEY,
+          Nfs3Constant.EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT);
+      int cacheSize = conf.getInt(Nfs3Constant.EXPORTS_CACHE_SIZE_KEY,
+          Nfs3Constant.EXPORTS_CACHE_SIZE_DEFAULT);
+      long expirationPeriodNano = conf.getLong(
+          Nfs3Constant.EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY,
+          Nfs3Constant.EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT) * 1000 * 1000;
+      exports = new NfsExports(cacheSize, expirationPeriodNano, matchHosts);
+    }
+    return exports;
+  }
+  
+  public static final Log LOG = LogFactory.getLog(NfsExports.class);
+  
+  // only support IPv4 now
+  private static final String IP_ADDRESS = 
+      "(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})";
+  private static final String SLASH_FORMAT_SHORT = IP_ADDRESS + "/(\\d{1,3})";
+  private static final String SLASH_FORMAT_LONG = IP_ADDRESS + "/" + IP_ADDRESS;
+  
+  private static final Pattern CIDR_FORMAT_SHORT = 
+      Pattern.compile(SLASH_FORMAT_SHORT);
+  
+  private static final Pattern CIDR_FORMAT_LONG = 
+      Pattern.compile(SLASH_FORMAT_LONG);
+  
+  static class AccessCacheEntry implements LightWeightCache.Entry{
+    private final String hostAddr;
+    private AccessPrivilege access;
+    private final long expirationTime; 
+    
+    private LightWeightGSet.LinkedElement next;
+    
+    AccessCacheEntry(String hostAddr, AccessPrivilege access,
+        long expirationTime) {
+      Preconditions.checkArgument(hostAddr != null);
+      this.hostAddr = hostAddr;
+      this.access = access;
+      this.expirationTime = expirationTime;
+    }
+    
+    @Override
+    public int hashCode() {
+      return hostAddr.hashCode();
+    }
+    
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj instanceof AccessCacheEntry) {
+        AccessCacheEntry entry = (AccessCacheEntry) obj;
+        return this.hostAddr.equals(entry.hostAddr);
+      }
+      return false;
+    }
+    
+    @Override
+    public void setNext(LinkedElement next) {
+      this.next = next;
+    }
+
+    @Override
+    public LinkedElement getNext() {
+      return this.next;
+    }
+
+    @Override
+    public void setExpirationTime(long timeNano) {
+      // we set expiration time in the constructor, and the expiration time 
+      // does not change
+    }
+
+    @Override
+    public long getExpirationTime() {
+      return this.expirationTime;
+    }
+  }
+
+  private final List<Match> mMatches;
+  
+  private final LightWeightCache<AccessCacheEntry, AccessCacheEntry> accessCache;
+  private final long cacheExpirationPeriod;
+
+  /**
+   * Constructor.
+   * @param cacheSize The size of the access privilege cache.
+   * @param expirationPeriodNano The period 
+   * @param matchingHosts A string specifying one or multiple matchers. 
+   */
+  NfsExports(int cacheSize, long expirationPeriodNano, String matchHosts) {
+    this.cacheExpirationPeriod = expirationPeriodNano;
+    accessCache = new LightWeightCache<AccessCacheEntry, AccessCacheEntry>(
+        cacheSize, cacheSize, expirationPeriodNano, 0);        
+    String[] matchStrings = matchHosts.split(
+        Nfs3Constant.EXPORTS_ALLOWED_HOSTS_SEPARATOR);
+    mMatches = new ArrayList<Match>(matchStrings.length);
+    for(String mStr : matchStrings) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Processing match string '" + mStr + "'");
+      }
+      mStr = mStr.trim();
+      if(!mStr.isEmpty()) {
+        mMatches.add(getMatch(mStr));
+      }
+    }
+  }
+  
+  /**
+   * Return the configured group list
+   */
+  public String[] getHostGroupList() {
+    int listSize = mMatches.size();
+    String[] hostGroups = new String[listSize];
+
+    for (int i = 0; i < mMatches.size(); i++) {
+      hostGroups[i] = mMatches.get(i).getHostGroup();
+    }
+    return hostGroups;
+  }
+  
+  public AccessPrivilege getAccessPrivilege(InetAddress addr) {
+    return getAccessPrivilege(addr.getHostAddress(),
+        addr.getCanonicalHostName());
+  }
+  
+  AccessPrivilege getAccessPrivilege(String address, String hostname) {
+    long now = System.nanoTime();
+    AccessCacheEntry newEntry = new AccessCacheEntry(address,
+        AccessPrivilege.NONE, now + this.cacheExpirationPeriod);
+    // check if there is a cache entry for the given address
+    AccessCacheEntry cachedEntry = accessCache.get(newEntry);
+    if (cachedEntry != null && now < cachedEntry.expirationTime) {
+      // get a non-expired cache entry, use it
+      return cachedEntry.access;
+    } else {
+      for(Match match : mMatches) {
+        if(match.isIncluded(address, hostname)) {
+          if (match.accessPrivilege == AccessPrivilege.READ_ONLY) {
+            newEntry.access = AccessPrivilege.READ_ONLY;
+            break;
+          } else if (match.accessPrivilege == AccessPrivilege.READ_WRITE) {
+            newEntry.access = AccessPrivilege.READ_WRITE;
+          }
+        }
+      }
+      accessCache.put(newEntry);
+      return newEntry.access;
+    }
+  }
+
+  private static abstract class Match {
+    private final AccessPrivilege accessPrivilege;
+
+    private Match(AccessPrivilege accessPrivilege) {
+      this.accessPrivilege = accessPrivilege;
+    }
+
+    public abstract boolean isIncluded(String address, String hostname);
+    public abstract String getHostGroup();
+  }
+  
+  /**
+   * Matcher covering all client hosts (specified by "*")
+   */
+  private static class AnonymousMatch extends Match {
+    private AnonymousMatch(AccessPrivilege accessPrivilege) {
+      super(accessPrivilege);
+    }
+  
+    @Override
+    public boolean isIncluded(String address, String hostname) {
+      return true;
+    }
+
+    @Override
+    public String getHostGroup() {
+      return "*";
+    }
+  }
+  
+  /**
+   * Matcher using CIDR for client host matching
+   */
+  private static class CIDRMatch extends Match {
+    private final SubnetInfo subnetInfo;
+    
+    private CIDRMatch(AccessPrivilege accessPrivilege, SubnetInfo subnetInfo) {
+      super(accessPrivilege);
+      this.subnetInfo = subnetInfo;
+    }
+    
+    @Override
+    public boolean isIncluded(String address, String hostname) {
+      if(subnetInfo.isInRange(address)) {
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("CIDRNMatcher low = " + subnetInfo.getLowAddress() +
+              ", high = " + subnetInfo.getHighAddress() +
+              ", allowing client '" + address + "', '" + hostname + "'");
+        }
+        return true;
+      }
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("CIDRNMatcher low = " + subnetInfo.getLowAddress() +
+            ", high = " + subnetInfo.getHighAddress() +
+            ", denying client '" + address + "', '" + hostname + "'");
+      }
+      return false;
+    }
+
+    @Override
+    public String getHostGroup() {
+      return subnetInfo.getAddress() + "/" + subnetInfo.getNetmask();
+    }
+  }
+  
+  /**
+   * Matcher requiring exact string match for client host
+   */
+  private static class ExactMatch extends Match {
+    private final String ipOrHost;
+    
+    private ExactMatch(AccessPrivilege accessPrivilege, String ipOrHost) {
+      super(accessPrivilege);
+      this.ipOrHost = ipOrHost;
+    }
+    
+    @Override
+    public boolean isIncluded(String address, String hostname) {
+      if(ipOrHost.equalsIgnoreCase(address) ||
+          ipOrHost.equalsIgnoreCase(hostname)) {
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("ExactMatcher '" + ipOrHost + "', allowing client " +
+              "'" + address + "', '" + hostname + "'");
+        }
+        return true;
+      }
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("ExactMatcher '" + ipOrHost + "', denying client " +
+            "'" + address + "', '" + hostname + "'");
+      }
+      return false;
+    }
+
+    @Override
+    public String getHostGroup() {
+      return ipOrHost;
+    }
+  }
+
+  /**
+   * Matcher where client hosts are specified by regular expression
+   */
+  private static class RegexMatch extends Match {
+    private final Pattern pattern;
+
+    private RegexMatch(AccessPrivilege accessPrivilege, String wildcard) {
+      super(accessPrivilege);
+      this.pattern = Pattern.compile(wildcard, Pattern.CASE_INSENSITIVE);
+    }
+
+    @Override
+    public boolean isIncluded(String address, String hostname) {
+      if (pattern.matcher(address).matches()
+          || pattern.matcher(hostname).matches()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("RegexMatcher '" + pattern.pattern()
+              + "', allowing client '" + address + "', '" + hostname + "'");
+        }
+        return true;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RegexMatcher '" + pattern.pattern()
+            + "', denying client '" + address + "', '" + hostname + "'");
+      }
+      return false;
+    }
+
+    @Override
+    public String getHostGroup() {
+      return pattern.toString();
+    }
+  }
+
+  /**
+   * Loading a matcher from a string. The default access privilege is read-only.
+   * The string contains 1 or 2 parts, separated by whitespace characters, where
+   * the first part specifies the client hosts, and the second part (if 
+   * existent) specifies the access privilege of the client hosts. I.e.,
+   * 
+   * "client-hosts [access-privilege]"
+   */
+  private static Match getMatch(String line) {
+    String[] parts = line.split("\\s+");
+    final String host;
+    AccessPrivilege privilege = AccessPrivilege.READ_ONLY;
+    switch (parts.length) {
+    case 1:
+      host = parts[0].toLowerCase().trim();
+      break;
+    case 2:
+      host = parts[0].toLowerCase().trim();
+      String option = parts[1].trim();
+      if ("rw".equalsIgnoreCase(option)) {
+        privilege = AccessPrivilege.READ_WRITE;
+      }
+      break;
+    default:
+      throw new IllegalArgumentException("Incorrectly formatted line '" + line
+          + "'");
+    }
+    if (host.equals("*")) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using match all for '" + host + "' and " + privilege);
+      }
+      return new AnonymousMatch(privilege);
+    } else if (CIDR_FORMAT_SHORT.matcher(host).matches()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using CIDR match for '" + host + "' and " + privilege);
+      }
+      return new CIDRMatch(privilege, new SubnetUtils(host).getInfo());
+    } else if (CIDR_FORMAT_LONG.matcher(host).matches()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using CIDR match for '" + host + "' and " + privilege);
+      }
+      String[] pair = host.split("/");
+      return new CIDRMatch(privilege,
+          new SubnetUtils(pair[0], pair[1]).getInfo());
+    } else if (host.contains("*") || host.contains("?") || host.contains("[")
+        || host.contains("]")) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using Regex match for '" + host + "' and " + privilege);
+      }
+      return new RegexMatch(privilege, host);
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using exact match for '" + host + "' and " + privilege);
+    }
+    return new ExactMatch(privilege, host);
+  }
+}
\ No newline at end of file

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java Tue Sep 10 19:29:45 2013
@@ -147,7 +147,7 @@ public class IdUserGroup {
 
   synchronized public String getUserName(int uid, String unknown) {
     checkAndUpdateMaps();
-    String uname = uidNameMap.get(Integer.valueOf(uid));
+    String uname = uidNameMap.get(uid);
     if (uname == null) {
       uname = unknown;
     }
@@ -156,7 +156,7 @@ public class IdUserGroup {
 
   synchronized public String getGroupName(int gid, String unknown) {
     checkAndUpdateMaps();
-    String gname = gidNameMap.get(Integer.valueOf(gid));
+    String gname = gidNameMap.get(gid);
     if (gname == null) {
       gname = unknown;
     }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java Tue Sep 10 19:29:45 2013
@@ -205,4 +205,7 @@ public class Nfs3Constant {
   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 boolean ENABLE_FILE_DUMP_DEFAULT = true;
+  
+  public final static String UNKNOWN_USER = "nobody";
+  public final static String UNKNOWN_GROUP = "nobody";
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java Tue Sep 10 19:29:45 2013
@@ -20,67 +20,83 @@ package org.apache.hadoop.nfs.nfs3;
 import java.net.InetAddress;
 
 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.security.SecurityHandler;
 import org.jboss.netty.channel.Channel;
 
 /**
  * RPC procedures as defined in RFC 1813.
  */
 public interface Nfs3Interface {
-  
+
   /** NULL: Do nothing */
   public NFS3Response nullProcedure();
-  
+
   /** 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 */
-  public NFS3Response setattr(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response setattr(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** 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 */
-  public NFS3Response read(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response read(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** WRITE: Write to file */
   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 */
-  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 */
-  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 */
-  public NFS3Response fsinfo(XDR xdr, RpcAuthSys authSys, InetAddress client);
-  
+  public NFS3Response fsinfo(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client);
+
   /** 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);
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java Tue Sep 10 19:29:45 2013
@@ -17,7 +17,9 @@
  */
 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.
@@ -54,7 +56,7 @@ public class RpcAcceptedReply extends Rp
 
   public static RpcAcceptedReply read(int xid, RpcMessage.Type messageType,
       ReplyState replyState, XDR xdr) {
-    RpcAuthInfo verifier = RpcAuthInfo.read(xdr);
+    Verifier verifier = Verifier.readFlavorAndVerifier(xdr);
     AcceptState acceptState = AcceptState.fromValue(xdr.readInt());
     return new RpcAcceptedReply(xid, messageType, replyState, verifier,
         acceptState);

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java Tue Sep 10 19:29:45 2013
@@ -19,6 +19,8 @@ package org.apache.hadoop.oncrpc;
 
 import org.apache.commons.logging.Log;
 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
@@ -30,11 +32,12 @@ public class RpcCall extends RpcMessage 
   private final int program;
   private final int version;
   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);
     this.rpcVersion = rpcVersion;
     this.program = program;
@@ -79,19 +82,19 @@ public class RpcCall extends RpcMessage 
     return procedure;
   }
   
-  public RpcAuthInfo getCredential() {
+  public Credentials getCredential() {
     return credential;
   }
 
-  public RpcAuthInfo getVerifier() {
+  public Verifier getVerifier() {
     return verifier;
   }
   
   public static RpcCall read(XDR xdr) {
     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,

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java Tue Sep 10 19:29:45 2013
@@ -17,7 +17,7 @@
  */
 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.

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java Tue Sep 10 19:29:45 2013
@@ -280,7 +280,7 @@ public class XDR {
 
   public byte[] readVariableOpaque() {
     int size = this.readInt();
-    return size != 0 ? this.readFixedOpaque(size) : null;
+    return size != 0 ? this.readFixedOpaque(size) : new byte[0];
   }
 
   public void skipVariableOpaque() {

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Credentials.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Credentials.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Credentials.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Credentials.java Tue Sep 10 19:29:45 2013
@@ -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);
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsGSS.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsGSS.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsGSS.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsGSS.java Tue Sep 10 19:29:45 2013
@@ -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;
+
+/** Credential used by RPCSEC_GSS */
+public class CredentialsGSS extends Credentials {
+
+  public CredentialsGSS() {
+    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
+    
+  }
+
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsNone.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsNone.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsNone.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsNone.java Tue Sep 10 19:29:45 2013
@@ -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);
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsSys.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsSys.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsSys.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/CredentialsSys.java Tue Sep 10 19:29:45 2013
@@ -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]);
+      }
+    }
+  }
+
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/RpcAuthInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/RpcAuthInfo.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/RpcAuthInfo.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/RpcAuthInfo.java Tue Sep 10 19:29:45 2013
@@ -0,0 +1,74 @@
+/**
+ * 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;
+
+/**
+ *  Authentication Info. Base class of Verifier and Credential.
+ */
+public abstract class RpcAuthInfo {
+  /** Different types of authentication as defined in RFC 1831 */
+  public enum AuthFlavor {
+    AUTH_NONE(0),
+    AUTH_SYS(1),
+    AUTH_SHORT(2),
+    AUTH_DH(3),
+    RPCSEC_GSS(6);
+    
+    private int value;
+    
+    AuthFlavor(int value) {
+      this.value = value;
+    }
+    
+    public int getValue() {
+      return value;
+    }
+    
+    static AuthFlavor fromValue(int value) {
+      for (AuthFlavor v : values()) {
+        if (v.value == value) {
+          return v;
+        }
+      }
+      throw new IllegalArgumentException("Invalid AuthFlavor value " + value);
+    }
+  }
+  
+  private final AuthFlavor flavor;
+  
+  protected RpcAuthInfo(AuthFlavor flavor) {
+    this.flavor = flavor;
+  }
+  
+  /** Load auth info */
+  public abstract void read(XDR xdr);
+  
+  /** Write auth info */
+  public abstract void write(XDR xdr);
+  
+  public AuthFlavor getFlavor() {
+    return flavor;
+  }
+  
+  @Override
+  public String toString() {
+    return "(AuthFlavor:" + flavor + ")";
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SecurityHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SecurityHandler.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SecurityHandler.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SecurityHandler.java Tue Sep 10 19:29:45 2013
@@ -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();
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java Tue Sep 10 19:29:45 2013
@@ -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();
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Verifier.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Verifier.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Verifier.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/Verifier.java Tue Sep 10 19:29:45 2013
@@ -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;
+  }
+  
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierGSS.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierGSS.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierGSS.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierGSS.java Tue Sep 10 19:29:45 2013
@@ -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
+    
+  }
+
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierNone.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierNone.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierNone.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/VerifierNone.java Tue Sep 10 19:29:45 2013
@@ -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);
+  }
+}

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapRequest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapRequest.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapRequest.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapRequest.java Tue Sep 10 19:29:45 2013
@@ -17,10 +17,14 @@
  */
 package org.apache.hadoop.portmap;
 
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
 import org.apache.hadoop.oncrpc.RpcCall;
 import org.apache.hadoop.oncrpc.RpcUtil;
 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;
 
 /**
@@ -38,9 +42,11 @@ public class PortmapRequest {
         RpcProgramPortmap.PROGRAM, RpcProgramPortmap.VERSION,
         Procedure.PMAPPROC_SET.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);
   }
 }

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java Tue Sep 10 19:29:45 2013
@@ -0,0 +1,191 @@
+/**
+ * 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.nfs;
+
+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.junit.Test;
+
+public class TestNfsExports {
+
+  private final String address1 = "192.168.0.1";
+  private final String address2 = "10.0.0.1";
+  private final String hostname1 = "a.b.com";
+  private final String hostname2 = "a.b.org";
+  
+  private static final long ExpirationPeriod = 
+      Nfs3Constant.EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT * 1000 * 1000;
+  
+  private static final int CacheSize = Nfs3Constant.EXPORTS_CACHE_SIZE_DEFAULT;
+
+  @Test
+  public void testWildcardRW() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod, "* rw");
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address1, hostname1));
+  }
+
+  @Test
+  public void testWildcardRO() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod, "* ro");
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+  }
+
+  @Test
+  public void testExactAddressRW() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod, address1
+        + " rw");
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertFalse(AccessPrivilege.READ_WRITE == matcher
+        .getAccessPrivilege(address2, hostname1));
+  }
+
+  @Test
+  public void testExactAddressRO() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod, address1);
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertEquals(AccessPrivilege.NONE,
+        matcher.getAccessPrivilege(address2, hostname1));
+  }
+
+  @Test
+  public void testExactHostRW() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod, hostname1
+        + " rw");
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address1, hostname1));
+  }
+
+  @Test
+  public void testExactHostRO() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod, hostname1);
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+  }
+
+  @Test
+  public void testCidrShortRW() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod,
+        "192.168.0.0/22 rw");
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertEquals(AccessPrivilege.NONE,
+        matcher.getAccessPrivilege(address2, hostname1));
+  }
+
+  @Test
+  public void testCidrShortRO() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod,
+        "192.168.0.0/22");
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertEquals(AccessPrivilege.NONE,
+        matcher.getAccessPrivilege(address2, hostname1));
+  }
+
+  @Test
+  public void testCidrLongRW() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod, 
+        "192.168.0.0/255.255.252.0 rw");
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertEquals(AccessPrivilege.NONE,
+        matcher.getAccessPrivilege(address2, hostname1));
+  }
+
+  @Test
+  public void testCidrLongRO() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod, 
+        "192.168.0.0/255.255.252.0");
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertEquals(AccessPrivilege.NONE,
+        matcher.getAccessPrivilege(address2, hostname1));
+  }
+
+  @Test
+  public void testRegexIPRW() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod,
+        "192.168.0.[0-9]+ rw");
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertEquals(AccessPrivilege.NONE,
+        matcher.getAccessPrivilege(address2, hostname1));
+  }
+
+  @Test
+  public void testRegexIPRO() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod,
+        "192.168.0.[0-9]+");
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertEquals(AccessPrivilege.NONE,
+        matcher.getAccessPrivilege(address2, hostname1));
+  }
+
+  @Test
+  public void testRegexHostRW() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod,
+        "[a-z]+.b.com rw");
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address1, hostname1));
+    // address1 will hit the cache
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address1, hostname2));
+  }
+
+  @Test
+  public void testRegexHostRO() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod,
+        "[a-z]+.b.com");
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+    // address1 will hit the cache
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname2));
+  }
+  
+  @Test
+  public void testMultiMatchers() throws Exception {
+    long shortExpirationPeriod = 1 * 1000 * 1000 * 1000; // 1s
+    NfsExports matcher = new NfsExports(CacheSize, shortExpirationPeriod, 
+        "192.168.0.[0-9]+;[a-z]+.b.com rw");
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname2));
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, address1));
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address2, hostname1));
+    // address2 will hit the cache
+    Assert.assertEquals(AccessPrivilege.READ_WRITE,
+        matcher.getAccessPrivilege(address2, hostname2));
+    
+    Thread.sleep(1000);
+    // no cache for address2 now
+    Assert.assertEquals(AccessPrivilege.NONE,
+        matcher.getAccessPrivilege(address2, address2));
+  }
+}

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java Tue Sep 10 19:29:45 2013
@@ -20,8 +20,9 @@ package org.apache.hadoop.oncrpc;
 import static org.junit.Assert.assertEquals;
 
 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.security.Verifier;
+import org.apache.hadoop.oncrpc.security.VerifierNone;
 import org.junit.Test;
 
 /**
@@ -45,7 +46,7 @@ public class TestRpcAcceptedReply {
   
   @Test
   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,
         ReplyState.MSG_ACCEPTED, verifier, AcceptState.SUCCESS);
     assertEquals(0, reply.getXid());

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java?rev=1521601&r1=1521600&r2=1521601&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java Tue Sep 10 19:29:45 2013
@@ -17,8 +17,12 @@
  */
 package org.apache.hadoop.oncrpc;
 
-import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
 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;
 
 /**
@@ -28,8 +32,8 @@ public class TestRpcCall {
   
   @Test
   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 program = 2;
     int version = 3;

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestCredentialsSys.java Tue Sep 10 19:29:45 2013
@@ -0,0 +1,46 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.oncrpc.security.CredentialsSys;
+import org.junit.Test;
+
+/**
+ * Test for {@link CredentialsSys}
+ */
+public class TestCredentialsSys {
+
+  @Test
+  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());
+  }
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java?rev=1521601&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/security/TestRpcAuthInfo.java Tue Sep 10 19:29:45 2013
@@ -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 static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo;
+import org.apache.hadoop.oncrpc.security.RpcAuthInfo.AuthFlavor;
+import org.junit.Test;
+
+/**
+ * Tests for {@link RpcAuthInfo}
+ */
+public class TestRpcAuthInfo {
+  @Test
+  public void testAuthFlavor() {
+    assertEquals(AuthFlavor.AUTH_NONE, AuthFlavor.fromValue(0));
+    assertEquals(AuthFlavor.AUTH_SYS, AuthFlavor.fromValue(1));
+    assertEquals(AuthFlavor.AUTH_SHORT, AuthFlavor.fromValue(2));
+    assertEquals(AuthFlavor.AUTH_DH, AuthFlavor.fromValue(3));
+    assertEquals(AuthFlavor.RPCSEC_GSS, AuthFlavor.fromValue(6));
+  }
+  
+  @Test(expected=IllegalArgumentException.class)
+  public void testInvalidAuthFlavor() {
+    assertEquals(AuthFlavor.AUTH_NONE, AuthFlavor.fromValue(4));
+  }
+}