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 br...@apache.org on 2013/08/31 23:12:22 UTC

svn commit: r1519222 - in /hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src: main/java/org/apache/hadoop/mount/ main/java/org/apache/hadoop/nfs/nfs3/ main/java/org/apache/hadoop/nfs/security/ test/java/org/apache/hadoop/nfs/security/

Author: brandonli
Date: Sat Aug 31 21:12:22 2013
New Revision: 1519222

URL: http://svn.apache.org/r1519222
Log:
HDFS-5136 MNT EXPORT should give the full group list which can mount the exports. Contributed by Brandon Li

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/
    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/test/java/org/apache/hadoop/nfs/security/
    hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/security/TestNfsExports.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/Nfs3Constant.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=1519222&r1=1519221&r2=1519222&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 Sat Aug 31 21:12:22 2013
@@ -19,6 +19,7 @@ package org.apache.hadoop.mount;
 
 import java.util.List;
 
+import org.apache.hadoop.nfs.security.NfsExports;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
@@ -59,15 +60,28 @@ public class MountResponse {
     xdr.writeBoolean(false); // Value follows no
     return xdr;
   }
-
+  
   /** Response for RPC call {@link MountInterface.MNTPROC#EXPORT} */
-  public static XDR writeExportList(XDR xdr, int xid, List<String> exports) {
+  public static XDR writeExportList(XDR xdr, int xid, List<String> exports,
+      List<NfsExports> hostMatcher) {
+    assert (exports.size() == hostMatcher.size());
+
     RpcAcceptedReply.voidReply(xdr, xid);
-    for (String export : exports) {
+    for (int i = 0; i < exports.size(); i++) {
       xdr.writeBoolean(true); // Value follows - yes
-      xdr.writeString(export);
-      xdr.writeInt(0);
+      xdr.writeString(exports.get(i));
+
+      // List host groups
+      String[] hostGroups = hostMatcher.get(i).getHostGroupList();
+      if (hostGroups.length > 0) {
+        for (int j = 0; j < hostGroups.length; j++) {
+          xdr.writeBoolean(true); // Value follows - yes
+          xdr.writeVariableOpaque(hostGroups[j].getBytes());
+        }
+      }
+      xdr.writeBoolean(false); // Value follows - no more group
     }
+    
     xdr.writeBoolean(false); // Value follows - no
     return xdr;
   }

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=1519222&r1=1519221&r2=1519222&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 Sat Aug 31 21:12:22 2013
@@ -192,13 +192,13 @@ public class Nfs3Constant {
   
   public static final String EXPORTS_ALLOWED_HOSTS_SEPARATOR = ";";
   /** Allowed hosts for nfs exports */
-  public static final String EXPORTS_ALLOWED_HOSTS_KEY = "hdfs.nfs.exports.allowed.hosts";
+  public static final String EXPORTS_ALLOWED_HOSTS_KEY = "dfs.nfs.exports.allowed.hosts";
   public static final String EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT = "* rw";
   /** Size for nfs exports cache */
-  public static final String EXPORTS_CACHE_SIZE_KEY = "hdfs.nfs.exports.cache.size";
+  public static final String EXPORTS_CACHE_SIZE_KEY = "dfs.nfs.exports.cache.size";
   public static final int EXPORTS_CACHE_SIZE_DEFAULT = 512;
   /** Expiration time for nfs exports cache entry */
-  public static final String EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY = "hdfs.nfs.exports.cache.expirytime.millis";
+  public static final String EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY = "dfs.nfs.exports.cache.expirytime.millis";
   public static final long EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT = 15 * 60 * 1000; // 15 min
 
   public static final String FILE_DUMP_DIR_KEY = "dfs.nfs3.dump.dir";

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/AccessPrivilege.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/AccessPrivilege.java?rev=1519222&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/AccessPrivilege.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/AccessPrivilege.java Sat Aug 31 21:12:22 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.security;
+
+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/security/NfsExports.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/NfsExports.java?rev=1519222&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/NfsExports.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/security/NfsExports.java Sat Aug 31 21:12:22 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.security;
+
+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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/security/TestNfsExports.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/security/TestNfsExports.java?rev=1519222&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/security/TestNfsExports.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/security/TestNfsExports.java Sat Aug 31 21:12:22 2013
@@ -0,0 +1,189 @@
+/**
+ * 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.security;
+
+import junit.framework.Assert;
+
+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));
+  }
+}