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 om...@apache.org on 2011/03/04 05:27:32 UTC

svn commit: r1077544 - in /hadoop/common/branches/branch-0.20-security-patches/src: core/org/apache/hadoop/security/ core/org/apache/hadoop/security/authorize/ core/org/apache/hadoop/util/ test/org/apache/hadoop/security/ test/org/apache/hadoop/securit...

Author: omalley
Date: Fri Mar  4 04:27:31 2011
New Revision: 1077544

URL: http://svn.apache.org/viewvc?rev=1077544&view=rev
Log:
commit 7e9ee06732decbe69611ba7dfb225f0bd2c15bb0
Author: Erik Steffl <st...@yahoo-inc.com>
Date:   Wed Jul 14 18:39:35 2010 -0700

    HADOOP:6855 from https://issues.apache.org/jira/secure/attachment/12449473/HADOOP-6855-0.20-2.patch
    
    +++ b/YAHOO-CHANGES.txt
    +    HADOOP-6855. Add support for netgroups, as returned by command
    +    getent netgroup. (Erik Steffl)
    +

Added:
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMappingTestWrapper.java
Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/GroupMappingServiceProvider.java
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/Groups.java
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/AccessControlList.java
    hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/util/Shell.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestMapredGroupMappingServiceRefresh.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/authorize/TestAccessControlList.java

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/GroupMappingServiceProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/GroupMappingServiceProvider.java?rev=1077544&r1=1077543&r2=1077544&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/GroupMappingServiceProvider.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/GroupMappingServiceProvider.java Fri Mar  4 04:27:31 2011
@@ -34,4 +34,15 @@ interface GroupMappingServiceProvider {
    * @throws IOException
    */
   public List<String> getGroups(String user) throws IOException;
+  /**
+   * Refresh the cache of groups and user mapping
+   * @throws IOException
+   */
+  public void cacheGroupsRefresh() throws IOException;
+  /**
+   * Caches the group user information
+   * @param groups list of groups to add to cache
+   * @throws IOException
+   */
+  public void cacheGroupsAdd(List<String> groups) throws IOException;
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/Groups.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/Groups.java?rev=1077544&r1=1077543&r2=1077544&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/Groups.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/Groups.java Fri Mar  4 04:27:31 2011
@@ -87,8 +87,21 @@ public class Groups {
    */
   public void refresh() {
     LOG.info("clearing userToGroupsMap cache");
+    try {
+      impl.cacheGroupsRefresh();
+    } catch (IOException e) {
+      LOG.warn("Error refreshing groups cache", e);
+    }
     userToGroupsMap.clear();
   }
+
+  public void cacheGroupsAdd(List<String> groups) {
+    try {
+      impl.cacheGroupsAdd(groups);
+    } catch (IOException e) {
+      LOG.warn("Error caching groups", e);
+    }
+  }
   
   private static class CachedGroups {
     final long timestamp;

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java?rev=1077544&r1=1077543&r2=1077544&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java Fri Mar  4 04:27:31 2011
@@ -21,6 +21,8 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.StringTokenizer;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -43,6 +45,16 @@ public class ShellBasedUnixGroupsMapping
     return getUnixGroups(user);
   }
 
+  @Override
+  public void cacheGroupsRefresh() throws IOException {
+    // does nothing in this provider of user to groups mapping
+  }
+
+  @Override
+  public void cacheGroupsAdd(List<String> groups) throws IOException {
+    // does nothing in this provider of user to groups mapping
+  }
+
   /** 
    * Get the current user's group list from Unix by running the command 'groups'
    * NOTE. For non-existing user it will return EMPTY list

Added: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java?rev=1077544&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java Fri Mar  4 04:27:31 2011
@@ -0,0 +1,171 @@
+/**
+ * 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.security;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.StringTokenizer;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.Shell.ExitCodeException;
+
+/**
+ * A simple shell-based implementation of {@link GroupMappingServiceProvider} 
+ * that exec's the <code>groups</code> shell command to fetch the group
+ * memberships of a given user.
+ */
+public class ShellBasedUnixGroupsNetgroupMapping extends ShellBasedUnixGroupsMapping {
+  
+  private static final Log LOG = LogFactory.getLog(ShellBasedUnixGroupsNetgroupMapping.class);
+
+  private static boolean netgroupToUsersMapUpdated = true;
+  private static Map<String, Set<String>> netgroupToUsersMap =
+    new ConcurrentHashMap<String, Set<String>>();
+
+  private static Map<String, Set<String>> userToNetgroupsMap =
+    new ConcurrentHashMap<String, Set<String>>();
+  
+  @Override
+  public List<String> getGroups(String user) throws IOException {
+    List<String> groups = new LinkedList<String>();
+    getUnixGroups(user, groups);
+    getNetgroups(user, groups);
+    return groups;
+  }
+
+  @Override
+  public void cacheGroupsRefresh() throws IOException {
+    List<String> groups = new LinkedList<String>(netgroupToUsersMap.keySet());
+    netgroupToUsersMap.clear();
+    cacheGroupsAdd(groups);
+    netgroupToUsersMapUpdated = true; // at the end to avoid race
+  }
+
+  @Override
+  public void cacheGroupsAdd(List<String> groups) throws IOException {
+    for(String group: groups) {
+      if(group.length() == 0) {
+        // better safe than sorry (should never happen)
+      } else if(group.charAt(0) == '@') {
+        cacheNetgroup(group);
+      } else {
+        // unix group, not caching
+      }
+    }
+  }
+
+  private void cacheNetgroup(String group) throws IOException {
+    if(netgroupToUsersMap.containsKey(group)) {
+      return;
+    } else {
+      // returns a string similar to this:
+      // group               ( , user, ) ( domain, user1, host.com )
+      String usersRaw = execShellGetUserForNetgroup(group);
+      // get rid of spaces, makes splitting much easier
+      usersRaw = usersRaw.replaceAll(" +", "");
+      // remove netgroup name at the beginning of the string
+      usersRaw = usersRaw.replaceFirst(
+        group.replaceFirst("@", "") + "[()]+",
+        "");
+      // split string into user infos
+      String[] userInfos = usersRaw.split("[()]+");
+      for(String userInfo : userInfos) {
+        // userInfo: xxx,user,yyy (xxx, yyy can be empty strings)
+        // get rid of everything before first and after last comma
+        String user = userInfo.replaceFirst("[^,]*,", "");
+        user = user.replaceFirst(",.*$", "");
+        // voila! got username!
+        if(!netgroupToUsersMap.containsKey(group)) {
+          netgroupToUsersMap.put(group, new HashSet<String>());
+        }
+        netgroupToUsersMap.get(group).add(user);
+      }
+      netgroupToUsersMapUpdated = true; // at the end to avoid race
+    }
+  }
+
+  /** 
+   * Get the current user's group list from Unix by running the command 'groups'
+   * NOTE. For non-existing user it will return EMPTY list
+   * @param user user name
+   * @return the groups list that the <code>user</code> belongs to
+   * @throws IOException if encounter any error when running the command
+   */
+  private void getUnixGroups(final String user,
+      List<String> groups) throws IOException {
+    String result = execShellGetUnixGroups(user);
+
+    StringTokenizer tokenizer = new StringTokenizer(result);
+    while (tokenizer.hasMoreTokens()) {
+      groups.add(tokenizer.nextToken());
+    }
+  }
+
+  private void getNetgroups(final String user,
+      List<String> groups) throws IOException {
+    if(netgroupToUsersMapUpdated) {
+      netgroupToUsersMapUpdated = false; // at the beginning to avoid race
+      //update userToNetgroupsMap
+      for(String netgroup : netgroupToUsersMap.keySet()) {
+        for(String netuser : netgroupToUsersMap.get(netgroup)) {
+          // add to userToNetgroupsMap
+          if(!userToNetgroupsMap.containsKey(netuser)) {
+            userToNetgroupsMap.put(netuser, new HashSet<String>());
+          }
+          userToNetgroupsMap.get(netuser).add(netgroup);
+        }
+      }
+    }
+    if(userToNetgroupsMap.containsKey(user)) {
+      for(String netgroup : userToNetgroupsMap.get(user)) {
+        groups.add(netgroup);
+      }
+    }
+  }
+
+  protected String execShellGetUnixGroups(final String user)
+      throws IOException {
+    String result = "";
+    try {
+      result = Shell.execCommand(Shell.getGroupsForUserCommand(user));
+    } catch (ExitCodeException e) {
+      // if we didn't get the group - just return empty list;
+      LOG.warn("error while getting groups for user " + user, e);
+    }
+    return result;
+  }
+
+  protected String execShellGetUserForNetgroup(final String netgroup)
+      throws IOException {
+    String result = "";
+    try {
+      result = Shell.execCommand(Shell.getUsersForNetgroupCommand(netgroup));
+    } catch (ExitCodeException e) {
+      // if we didn't get the group - just return empty list;
+      LOG.warn("error while getting users for netgroup " + netgroup, e);
+    }
+    return result;
+  }
+}

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/AccessControlList.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/AccessControlList.java?rev=1077544&r1=1077543&r2=1077544&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/AccessControlList.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/security/authorize/AccessControlList.java Fri Mar  4 04:27:31 2011
@@ -17,17 +17,25 @@
  */
 package org.apache.hadoop.security.authorize;
 
+import java.util.Iterator;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.Arrays;
+import java.util.List;
+import java.util.LinkedList;
+import java.util.ListIterator;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.Groups;
+import org.apache.hadoop.conf.Configuration;
 
 /**
  * Class representing a configured access control list.
@@ -82,19 +90,22 @@ public class AccessControlList implement
     } else {
       allAllowed = false;
       String[] userGroupStrings = aclString.split(" ", 2);
-      
+      Configuration conf = new Configuration();
+      Groups groupsMapping = Groups.getUserToGroupsMappingService(conf);
+
       if (userGroupStrings.length >= 1) {
-        String[] usersStr = userGroupStrings[0].split(",");
-        if (usersStr.length >= 1) {
-          addToSet(users, usersStr);
-        }
+        List<String> usersList = new LinkedList<String>(
+          Arrays.asList(userGroupStrings[0].split(",")));
+        cleanupList(usersList);
+        addToSet(users, usersList);
       }
       
       if (userGroupStrings.length == 2) {
-        String[] groupsStr = userGroupStrings[1].split(",");
-        if (groupsStr.length >= 1) {
-          addToSet(groups, groupsStr);
-        }
+        List<String> groupsList = new LinkedList<String>(
+          Arrays.asList(userGroupStrings[1].split(",")));
+        cleanupList(groupsList);
+        addToSet(groups, groupsList);
+        groupsMapping.cacheGroupsAdd(groupsList);
       }
     }
   }
@@ -135,16 +146,26 @@ public class AccessControlList implement
     }
     return false;
   }
-  
-  private static final void addToSet(Set<String> set, String[] strings) {
-    for (String s : strings) {
-      s = s.trim();
-      if (s.length() > 0) {
-        set.add(s);
+
+  private static final void cleanupList(List<String> list) {
+    ListIterator<String> i = list.listIterator();
+    while(i.hasNext()) {
+      String s = i.next();
+      if(s.length() == 0) {
+        i.remove();
+      } else {
+        s = s.trim();
+        i.set(s);
       }
     }
   }
   
+  private static final void addToSet(Set<String> set, List<String> list) {
+    for(String s : list) {
+      set.add(s);
+    }
+  }
+  
   @Override
   public String toString() {
     String str = null;
@@ -235,4 +256,4 @@ public class AccessControlList implement
     }
     return sb.toString();
   }
-}
\ No newline at end of file
+}

Modified: hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/util/Shell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/util/Shell.java?rev=1077544&r1=1077543&r2=1077544&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/util/Shell.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/core/org/apache/hadoop/util/Shell.java Fri Mar  4 04:27:31 2011
@@ -52,6 +52,11 @@ abstract public class Shell {
     //'groups username' command return is non-consistent across different unixes
     return new String [] {"bash", "-c", "id -Gn " + user};
   }
+  /** a Unix command to get a given netgroup's user list */
+  public static String[] getUsersForNetgroupCommand(final String netgroup) {
+    //'groups username' command return is non-consistent across different unixes
+    return new String [] {"bash", "-c", "getent netgroup " + netgroup};
+  }
   /** a Unix command to set permission */
   public static final String SET_PERMISSION_COMMAND = "chmod";
   /** a Unix command to set owner */

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMappingTestWrapper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMappingTestWrapper.java?rev=1077544&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMappingTestWrapper.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMappingTestWrapper.java Fri Mar  4 04:27:31 2011
@@ -0,0 +1,57 @@
+/**
+ * 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.security;
+
+import java.io.IOException;
+
+/**
+ * A wrapper for ShellBasedUnixGroupsMapping that replaces functions
+ * that call external programs (to get users/groups) by static stubs
+ * for the purposes of testing
+ *
+ * Keep in sync with TestAccessControlList.java
+ */
+public class ShellBasedUnixGroupsNetgroupMappingTestWrapper
+    extends ShellBasedUnixGroupsNetgroupMapping {
+  
+  @Override
+  protected String execShellGetUnixGroups(final String user)
+      throws IOException {
+    if(user.equals("ja")) {
+      return "my";
+    } else if(user.equals("sinatra")) {
+      return "ratpack";
+    } else if(user.equals("elvis")) {
+      return "users otherGroup";
+    }
+    return "";
+  }
+
+  @Override
+  protected String execShellGetUserForNetgroup(final String netgroup)
+      throws IOException {
+    if(netgroup.equals("@lasVegas")) {
+      return "lasVegas               ( , sinatra, ) ( domain, elvis, host.com)";
+    } else if(netgroup.equals("@somenetgroup")) {
+      return "somenetgroup           ( , nobody, )";
+    } else {
+      return "";
+    }
+  }  
+
+}

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestMapredGroupMappingServiceRefresh.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestMapredGroupMappingServiceRefresh.java?rev=1077544&r1=1077543&r2=1077544&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestMapredGroupMappingServiceRefresh.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestMapredGroupMappingServiceRefresh.java Fri Mar  4 04:27:31 2011
@@ -71,6 +71,15 @@ public class TestMapredGroupMappingServi
       i++;
       return l;
     }
+
+    @Override
+    public void cacheGroupsRefresh() throws IOException {
+    }
+
+    @Override
+    public void cacheGroupsAdd(List<String> groups) throws IOException {
+    }
+
   }
   
   @Before

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java?rev=1077544&r1=1077543&r2=1077544&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestRefreshUserMappings.java Fri Mar  4 04:27:31 2011
@@ -66,6 +66,15 @@ public class TestRefreshUserMappings {
       i++;
       return l;
     }
+
+    @Override
+    public void cacheGroupsRefresh() throws IOException {
+    }
+
+    @Override
+    public void cacheGroupsAdd(List<String> groups) throws IOException {
+    }
+
   }
   
   @Before

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/authorize/TestAccessControlList.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/authorize/TestAccessControlList.java?rev=1077544&r1=1077543&r2=1077544&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/authorize/TestAccessControlList.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/authorize/TestAccessControlList.java Fri Mar  4 04:27:31 2011
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.security.authorize;
 
+import java.util.List;
 import java.util.Iterator;
 import java.util.Set;
 
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 
@@ -27,6 +31,52 @@ import org.apache.hadoop.security.author
 import junit.framework.TestCase;
 
 public class TestAccessControlList extends TestCase {
+
+  /**
+   * test the netgroups (groups in ACL rules that start with @),
+   */
+  public void testNetgroups() throws Exception {
+    // set the config for Groups (test mapping class)
+    // we rely on hardcoded groups and netgroups in
+    // ShellBasedUnixGroupsMappingTestWrapper
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.group.mapping",
+      "org.apache.hadoop.security.ShellBasedUnixGroupsNetgroupMappingTestWrapper");
+
+    Groups groups = Groups.getUserToGroupsMappingService(conf);
+
+    AccessControlList acl;
+
+    // create these ACLs to populate groups cache
+    acl = new AccessControlList("ja my"); // plain
+    acl = new AccessControlList("sinatra ratpack,@lasVegas"); // netgroup
+    acl = new AccessControlList(" somegroups,@somenetgroup"); // no user
+
+    // check that the netgroups are working
+    List<String> elvisGroups = groups.getGroups("elvis");
+    assertTrue(elvisGroups.contains("@lasVegas"));
+
+    // refresh cache - not testing this directly but if the results are ok
+    // after the refresh that means it worked fine (very likely)
+    groups.refresh();
+
+    // create an ACL with netgroups (@xxx)
+    acl = new AccessControlList("ja ratpack,@lasVegas");
+    // elvis is in @lasVegas
+    UserGroupInformation elvis = 
+      UserGroupInformation.createRemoteUser("elvis");
+    // ja's groups are not in ACL
+    UserGroupInformation ja = 
+      UserGroupInformation.createRemoteUser("ja");
+    // unwanted and unwanted's grops are not in ACL
+    UserGroupInformation unwanted = 
+      UserGroupInformation.createRemoteUser("unwanted");
+
+    // test the ACLs!
+    assertUserAllowed(elvis, acl);
+    assertUserAllowed(ja, acl);
+    assertUserNotAllowed(unwanted, acl);
+  }
   
   public void testWildCardAccessControlList() throws Exception {
     AccessControlList acl;