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 bo...@apache.org on 2010/12/24 01:45:04 UTC

svn commit: r1052420 [1/3] - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/security/ src/java/org/apache/hadoop/security/authorize/ src/java/org/apache/hadoop/util/ src/native/ src/native/src/org/apache/hadoop/security/ src/test/core/org/apach...

Author: boryas
Date: Fri Dec 24 00:45:03 2010
New Revision: 1052420

URL: http://svn.apache.org/viewvc?rev=1052420&view=rev
Log:
HADOOP-6864. Provide a JNI-based implementation of ShellBasedUnixGroupsNetgroupMapping (implementation of GroupMappingServiceProvider)

Added:
    hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/NetgroupCache.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java
    hadoop/common/trunk/src/native/src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/build.xml
    hadoop/common/trunk/src/java/org/apache/hadoop/security/GroupMappingServiceProvider.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/Groups.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsMapping.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/AccessControlList.java
    hadoop/common/trunk/src/java/org/apache/hadoop/util/Shell.java
    hadoop/common/trunk/src/native/Makefile.am
    hadoop/common/trunk/src/native/Makefile.in
    hadoop/common/trunk/src/native/configure
    hadoop/common/trunk/src/test/core/org/apache/hadoop/security/authorize/TestAccessControlList.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Fri Dec 24 00:45:03 2010
@@ -242,6 +242,9 @@ Release 0.22.0 - Unreleased
     classpath. (Patrick Angeles via eli)
 
     HADOOP-6298. Add copyBytes to Text and BytesWritable. (omalley)
+  
+    HADOOP-6864. Provide a JNI-based implementation of ShellBasedUnixGroupsNetgroupMapping 
+    (implementation of GroupMappingServiceProvider) (Erik Seffl via boryas)
 
   OPTIMIZATIONS
 

Modified: hadoop/common/trunk/build.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/build.xml?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/build.xml (original)
+++ hadoop/common/trunk/build.xml Fri Dec 24 00:45:03 2010
@@ -396,6 +396,15 @@
   	  <class name="org.apache.hadoop.io.nativeio.NativeIO" />
   	</javah>
 
+  	<javah
+  	  classpath="${build.classes}"
+  	  destdir="${build.native}/src/org/apache/hadoop/security"
+      force="yes"
+  	  verbose="yes"
+  	  >
+  	  <class name="org.apache.hadoop.security.JniBasedUnixGroupsNetgroupMapping" />
+  	</javah>
+
 	<exec dir="${build.native}" executable="sh" failonerror="true">
 	  <env key="OS_NAME" value="${os.name}"/>
 	  <env key="OS_ARCH" value="${os.arch}"/>
@@ -717,6 +726,10 @@
         </syspropertyset>
         <sysproperty key="test.system.hdrc.deployed.hadoopconfdir"
                      value="@{hadoop.conf.dir.deployed}" />
+        <!-- user to group mapping class for TestAccessControlList -->
+        <syspropertyset dynamic="no">
+          <propertyref name="TestAccessControlListGroupMapping"/>
+        </syspropertyset>
         <formatter type="${test.junit.output.format}" />
         <batchtest todir="@{test.dir}" if="tests.notestcase">
           <fileset dir="@{fileset.dir}/core"

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/security/GroupMappingServiceProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/GroupMappingServiceProvider.java?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/GroupMappingServiceProvider.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/GroupMappingServiceProvider.java Fri Dec 24 00:45:03 2010
@@ -39,4 +39,15 @@ public interface GroupMappingServiceProv
    * @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/trunk/src/java/org/apache/hadoop/security/Groups.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/Groups.java?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/Groups.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/Groups.java Fri Dec 24 00:45:03 2010
@@ -98,22 +98,56 @@ 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();
   }
-  
+
+  /**
+   * Add groups to cache
+   *
+   * @param groups list of groups to add to cache
+   */
+  public void cacheGroupsAdd(List<String> groups) {
+    try {
+      impl.cacheGroupsAdd(groups);
+    } catch (IOException e) {
+      LOG.warn("Error caching groups", e);
+    }
+  }
+
+  /**
+   * Class to hold the cached groups
+   */
   private static class CachedGroups {
     final long timestamp;
     final List<String> groups;
     
+    /**
+     * Create and initialize group cache
+     */
     CachedGroups(List<String> groups) {
       this.groups = groups;
       this.timestamp = System.currentTimeMillis();
     }
 
+    /**
+     * Returns time of last cache update
+     *
+     * @return time of last cache update
+     */
     public long getTimestamp() {
       return timestamp;
     }
 
+    /**
+     * Get list of cached groups
+     *
+     * @return cached groups
+     */
     public List<String> getGroups() {
       return groups;
     }
@@ -128,13 +162,15 @@ public class Groups {
   public static Groups getUserToGroupsMappingService() {
     return getUserToGroupsMappingService(new Configuration()); 
   }
-  
+
   /**
    * Get the groups being used to map user-to-groups.
    * @param conf
    * @return the groups being used to map user-to-groups.
    */
-  public static Groups getUserToGroupsMappingService(Configuration conf) {
+  public static synchronized Groups getUserToGroupsMappingService(
+    Configuration conf) {
+
     if(GROUPS == null) {
       if(LOG.isDebugEnabled()) {
         LOG.debug(" Creating new Groups object");

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsMapping.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsMapping.java?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsMapping.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsMapping.java Fri Dec 24 00:45:03 2010
@@ -22,6 +22,9 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.util.NativeCodeLoader;
@@ -31,17 +34,19 @@ import org.apache.hadoop.util.NativeCode
  * that invokes libC calls to get the group
  * memberships of a given user.
  */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
 public class JniBasedUnixGroupsMapping implements GroupMappingServiceProvider {
   
   private static final Log LOG = 
-    LogFactory.getLog(ShellBasedUnixGroupsMapping.class);
+    LogFactory.getLog(JniBasedUnixGroupsMapping.class);
   
   native String[] getGroupForUser(String user);
   
   static {
     if (!NativeCodeLoader.isNativeCodeLoaded()) {
       throw new RuntimeException("Bailing out since native library couldn't " +
-      		                       "be loaded");
+        "be loaded");
     }
     LOG.info("Using JniBasedUnixGroupsMapping for Group resolution");
   }
@@ -52,9 +57,18 @@ public class JniBasedUnixGroupsMapping i
     try {
       groups = getGroupForUser(user);
     } catch (Exception e) {
-      LOG.warn("Got exception while trying to obtain the groups for user " 
-               + user);
+      LOG.warn("Error getting groups for " + user, e);
     }
     return Arrays.asList(groups);
   }
+
+  @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
+  }
 }

Added: hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.java?rev=1052420&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.java Fri Dec 24 00:45:03 2010
@@ -0,0 +1,125 @@
+/**
+ * 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.Arrays;
+import java.util.List;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.NativeCodeLoader;
+
+import org.apache.hadoop.security.NetgroupCache;
+
+/**
+ * A JNI-based implementation of {@link GroupMappingServiceProvider} 
+ * that invokes libC calls to get the group
+ * memberships of a given user.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class JniBasedUnixGroupsNetgroupMapping
+  extends JniBasedUnixGroupsMapping {
+  
+  private static final Log LOG = LogFactory.getLog(
+    JniBasedUnixGroupsNetgroupMapping.class);
+
+  native String[] getUsersForNetgroupJNI(String group);
+
+  static {
+    if (!NativeCodeLoader.isNativeCodeLoaded()) {
+      throw new RuntimeException("Bailing out since native library couldn't " +
+        "be loaded");
+    }
+    LOG.info("Using JniBasedUnixGroupsNetgroupMapping for Netgroup resolution");
+  }
+
+  /**
+   * Gets unix groups and netgroups for the user.
+   *
+   * It gets all unix groups as returned by id -Gn but it
+   * only returns netgroups that are used in ACLs (there is
+   * no way to get all netgroups for a given user, see
+   * documentation for getent netgroup)
+   */
+  @Override
+  public List<String> getGroups(String user) throws IOException {
+    // parent gets unix groups
+    List<String> groups = new LinkedList<String>(super.getGroups(user));
+    NetgroupCache.getNetgroups(user, groups);
+    return groups;
+  }
+
+  /**
+   * Refresh the netgroup cache
+   */
+  @Override
+  public void cacheGroupsRefresh() throws IOException {
+    List<String> groups = NetgroupCache.getNetgroupNames();
+    NetgroupCache.clear();
+    cacheGroupsAdd(groups);
+  }
+
+  /**
+   * Add a group to cache, only netgroups are cached
+   *
+   * @param groups list of group names to add to cache
+   */
+  @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) == '@') {
+        if(!NetgroupCache.isCached(group)) {
+          NetgroupCache.add(group, getUsersForNetgroup(group));
+        }
+      } else {
+        // unix group, not caching
+      }
+    }
+  }
+
+  /**
+   * Calls JNI function to get users for a netgroup, since C functions
+   * are not reentrant we need to make this synchronized (see
+   * documentation for setnetgrent, getnetgrent and endnetgrent)
+   *
+   * @param netgroup return users for this netgroup
+   * @return list of users for a given netgroup
+   */
+  protected synchronized List<String> getUsersForNetgroup(String netgroup) {
+    String[] users = null;
+    try {
+      // JNI code does not expect '@' at the begining of the group name
+      users = getUsersForNetgroupJNI(netgroup.substring(1));
+    } catch (Exception e) {
+      LOG.warn("error getting users for netgroup " + netgroup, e);
+    }
+    if (users != null && users.length != 0) {
+      return Arrays.asList(users);
+    }
+    return new LinkedList<String>();
+  }
+}

Added: hadoop/common/trunk/src/java/org/apache/hadoop/security/NetgroupCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/NetgroupCache.java?rev=1052420&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/NetgroupCache.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/NetgroupCache.java Fri Dec 24 00:45:03 2010
@@ -0,0 +1,122 @@
+/**
+ * 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.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Class that caches the netgroups and inverts group-to-user map
+ * to user-to-group map, primarily intented for use with
+ * netgroups (as returned by getent netgrgoup) which only returns
+ * group to user mapping.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Unstable
+public class NetgroupCache {
+
+  private static final Log LOG = LogFactory.getLog(NetgroupCache.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>>();
+
+
+  /**
+   * Get netgroups for a given user
+   *
+   * @param user get groups for this user
+   * @param groups put groups into this List
+   */
+  public static void getNetgroups(final String user,
+      List<String> groups) {
+    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);
+      }
+    }
+  }
+
+  /**
+   * Get the list of cached netgroups
+   *
+   * @return list of cached groups
+   */
+  public static List<String> getNetgroupNames() {
+    return new LinkedList<String>(netgroupToUsersMap.keySet());
+  }
+
+  /**
+   * Returns true if a given netgroup is cached
+   *
+   * @param group check if this group is cached
+   * @return true if group is cached, false otherwise
+   */
+  public static boolean isCached(String group) {
+    return netgroupToUsersMap.containsKey(group);
+  }
+
+  /**
+   * Clear the cache
+   */
+  public static void clear() {
+    netgroupToUsersMap.clear();
+  }
+
+  /**
+   * Add group to cache
+   *
+   * @param group name of the group to add to cache
+   * @param users list of users for a given group
+   */
+  public static void add(String group, List<String> users) {
+    if(!isCached(group)) {
+      netgroupToUsersMap.put(group, new HashSet<String>());
+      for(String user: users) {
+        netgroupToUsersMap.get(group).add(user);
+      }
+    }
+    netgroupToUsersMapUpdated = true; // at the end to avoid race
+  }
+}

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java Fri Dec 24 00:45:03 2010
@@ -38,15 +38,41 @@ import org.apache.hadoop.util.Shell.Exit
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
-public class ShellBasedUnixGroupsMapping implements GroupMappingServiceProvider {
+public class ShellBasedUnixGroupsMapping
+  implements GroupMappingServiceProvider {
   
-  private static final Log LOG = LogFactory.getLog(ShellBasedUnixGroupsMapping.class);
+  private static final Log LOG =
+    LogFactory.getLog(ShellBasedUnixGroupsMapping.class);
   
+  /**
+   * Returns list of groups for a user
+   *
+   * @param user get groups for this user
+   * @return list of groups for a given user
+   */
   @Override
   public List<String> getGroups(String user) throws IOException {
     return getUnixGroups(user);
   }
 
+  /**
+   * Caches groups, no need to do that for this provider
+   */
+  @Override
+  public void cacheGroupsRefresh() throws IOException {
+    // does nothing in this provider of user to groups mapping
+  }
+
+  /** 
+   * Adds groups to cache, no need to do that for this provider
+   *
+   * @param groups unused
+   */
+  @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/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java?rev=1052420&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java Fri Dec 24 00:45:03 2010
@@ -0,0 +1,151 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+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;
+
+import org.apache.hadoop.security.NetgroupCache;
+
+/**
+ * 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.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class ShellBasedUnixGroupsNetgroupMapping
+  extends ShellBasedUnixGroupsMapping {
+  
+  private static final Log LOG =
+    LogFactory.getLog(ShellBasedUnixGroupsNetgroupMapping.class);
+
+  /**
+   * Get unix groups (parent) and netgroups for given user
+   *
+   * @param user get groups and netgroups for this user
+   * @return groups and netgroups for user
+   */
+  @Override
+  public List<String> getGroups(String user) throws IOException {
+    // parent get unix groups
+    List<String> groups = new LinkedList<String>(super.getGroups(user));
+    NetgroupCache.getNetgroups(user, groups);
+    return groups;
+  }
+
+  /**
+   * Refresh the netgroup cache
+   */
+  @Override
+  public void cacheGroupsRefresh() throws IOException {
+    List<String> groups = NetgroupCache.getNetgroupNames();
+    NetgroupCache.clear();
+    cacheGroupsAdd(groups);
+  }
+
+  /**
+   * Add a group to cache, only netgroups are cached
+   *
+   * @param groups list of group names to add to cache
+   */
+  @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) == '@') {
+        if(!NetgroupCache.isCached(group)) {
+          NetgroupCache.add(group, getUsersForNetgroup(group));
+        }
+      } else {
+        // unix group, not caching
+      }
+    }
+  }
+
+  /**
+   * Gets users for a netgroup
+   *
+   * @param netgroup return users for this netgroup
+   * @return list of users for a given netgroup
+   */
+  protected List<String> getUsersForNetgroup(String netgroup) 
+    throws IOException {
+
+    List<String> users = new LinkedList<String>();
+
+    // returns a string similar to this:
+    // group               ( , user, ) ( domain, user1, host.com )
+    String usersRaw = execShellGetUserForNetgroup(netgroup);
+    // get rid of spaces, makes splitting much easier
+    usersRaw = usersRaw.replaceAll(" +", "");
+    // remove netgroup name at the beginning of the string
+    usersRaw = usersRaw.replaceFirst(
+      netgroup.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!
+      users.add(user);
+    }
+
+    return users;
+  }
+
+  /**
+   * Calls shell to get users for a netgroup by calling getent
+   * netgroup, this is a low level function that just returns string
+   * that 
+   *
+   * @param netgroup get users for this netgroup
+   * @return string of users for a given netgroup in getent netgroups format
+   */
+  protected String execShellGetUserForNetgroup(final String netgroup)
+      throws IOException {
+    String result = "";
+    try {
+      // shell command does not expect '@' at the begining of the group name
+      result = Shell.execCommand(
+        Shell.getUsersForNetgroupCommand(netgroup.substring(1)));
+    } catch (ExitCodeException e) {
+      // if we didn't get the group - just return empty list;
+      LOG.warn("error getting users for netgroup " + netgroup, e);
+    }
+    return result;
+  }
+}

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/AccessControlList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/AccessControlList.java?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/AccessControlList.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/AccessControlList.java Fri Dec 24 00:45:03 2010
@@ -22,6 +22,10 @@ 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.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -30,6 +34,8 @@ 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.
@@ -57,6 +63,8 @@ public class AccessControlList implement
   // Whether all users are granted access.
   private boolean allAllowed;
 
+  private Groups groupsMapping = Groups.getUserToGroupsMappingService(new Configuration());
+
   /**
    * This constructor exists primarily for AccessControlList to be Writable.
    */
@@ -76,7 +84,12 @@ public class AccessControlList implement
     buildACL(aclString);
   }
 
-  // build ACL from the given string
+  /**
+   * Build ACL from the given string, format of the string is
+   * user1,...,userN group1,...,groupN
+   *
+   * @param aclString build ACL from this string
+   */
   private void buildACL(String aclString) {
     users = new TreeSet<String>();
     groups = new TreeSet<String>();
@@ -85,23 +98,30 @@ public class AccessControlList implement
     } else {
       allAllowed = false;
       String[] userGroupStrings = aclString.split(" ", 2);
-      
+
       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);
       }
     }
   }
   
+  /**
+   * Checks whether ACL string contains wildcard
+   *
+   * @param aclString check this ACL string for wildcard
+   * @return true if ACL string contains wildcard false otherwise
+   */
   private boolean isWildCardACLValue(String aclString) {
     if (aclString.contains(WILDCARD_ACL_VALUE) && 
         aclString.trim().equals(WILDCARD_ACL_VALUE)) {
@@ -140,6 +160,9 @@ public class AccessControlList implement
       throw new IllegalArgumentException("Group " + group + " can not be added");
     }
     if (!isAllAllowed()) {
+      List<String> groupsList = new LinkedList<String>();
+      groupsList.add(group);
+      groupsMapping.cacheGroupsAdd(groupsList);
       groups.add(group);
     }
   }
@@ -203,17 +226,38 @@ 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);
+
+  /**
+   * Cleanup list, remove empty strings, trim leading/trailing spaces
+   *
+   * @param list clean this list
+   */
+  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);
       }
     }
   }
 
   /**
+   * Add list to a set
+   *
+   * @param set add list to this set
+   * @param list add items of this list to the set
+   */
+  private static final void addToSet(Set<String> set, List<String> list) {
+    for(String s : list) {
+      set.add(s);
+    }
+  }
+
+  /**
    * Returns descriptive way of users and groups that are part of this ACL.
    * Use {@link #getAclString()} to get the exact String that can be given to
    * the constructor of AccessControlList to create a new instance.
@@ -287,18 +331,30 @@ public class AccessControlList implement
     buildACL(aclString);
   }
 
-  // Returns comma-separated concatenated single String of the set 'users'
+  /**
+   * Returns comma-separated concatenated single String of the set 'users'
+   *
+   * @return comma separated list of users
+   */
   private String getUsersString() {
     return getString(users);
   }
 
-  // Returns comma-separated concatenated single String of the set 'groups'
+  /**
+   * Returns comma-separated concatenated single String of the set 'groups'
+   *
+   * @return comma separated list of groups
+   */
   private String getGroupsString() {
     return getString(groups);
   }
 
-  // Returns comma-separated concatenated single String of all strings of
-  // the given set
+  /**
+   * Returns comma-separated concatenated single String of all strings of
+   * the given set
+   *
+   * @param strings set of strings to concatenate
+   */
   private String getString(Set<String> strings) {
     StringBuilder sb = new StringBuilder(INITIAL_CAPACITY);
     boolean first = true;
@@ -312,4 +368,4 @@ public class AccessControlList implement
     }
     return sb.toString();
   }
-}
\ No newline at end of file
+}

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/util/Shell.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/util/Shell.java?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/util/Shell.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/util/Shell.java Fri Dec 24 00:45:03 2010
@@ -56,6 +56,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 */

Modified: hadoop/common/trunk/src/native/Makefile.am
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/native/Makefile.am?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/native/Makefile.am (original)
+++ hadoop/common/trunk/src/native/Makefile.am Fri Dec 24 00:45:03 2010
@@ -43,6 +43,7 @@ libhadoop_la_SOURCES = src/org/apache/ha
                        src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c \
                        src/org/apache/hadoop/security/getGroup.c \
                        src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c \
+                       src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c \
                        src/org/apache/hadoop/io/nativeio/file_descriptor.c \
                        src/org/apache/hadoop/io/nativeio/errno_enum.c \
                        src/org/apache/hadoop/io/nativeio/NativeIO.c

Modified: hadoop/common/trunk/src/native/Makefile.in
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/native/Makefile.in?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/native/Makefile.in (original)
+++ hadoop/common/trunk/src/native/Makefile.in Fri Dec 24 00:45:03 2010
@@ -92,9 +92,14 @@ am__installdirs = "$(DESTDIR)$(libdir)"
 libLTLIBRARIES_INSTALL = $(INSTALL)
 LTLIBRARIES = $(lib_LTLIBRARIES)
 libhadoop_la_DEPENDENCIES =
-am_libhadoop_la_OBJECTS = ZlibCompressor.lo ZlibDecompressor.lo \
-	getGroup.lo JniBasedUnixGroupsMapping.lo file_descriptor.lo \
-	errno_enum.lo NativeIO.lo
+am_libhadoop_la_OBJECTS = ZlibCompressor.lo \
+	ZlibDecompressor.lo \
+	getGroup.lo \
+	JniBasedUnixGroupsMapping.lo \
+	JniBasedUnixGroupsNetgroupMapping.lo \
+	file_descriptor.lo \
+	errno_enum.lo \
+	NativeIO.lo
 libhadoop_la_OBJECTS = $(am_libhadoop_la_OBJECTS)
 DEFAULT_INCLUDES = -I. -I$(srcdir) -I.
 depcomp = $(SHELL) $(top_srcdir)/config/depcomp
@@ -233,6 +238,7 @@ libhadoop_la_SOURCES = src/org/apache/ha
                        src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c \
                        src/org/apache/hadoop/security/getGroup.c \
                        src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c \
+                       src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c \
                        src/org/apache/hadoop/io/nativeio/file_descriptor.c \
                        src/org/apache/hadoop/io/nativeio/errno_enum.c \
                        src/org/apache/hadoop/io/nativeio/NativeIO.c
@@ -331,6 +337,7 @@ distclean-compile:
 	-rm -f *.tab.c
 
 @AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/JniBasedUnixGroupsMapping.Plo@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/JniBasedUnixGroupsNetgroupMapping.Plo@am__quote@
 @AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/NativeIO.Plo@am__quote@
 @AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/ZlibCompressor.Plo@am__quote@
 @AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/ZlibDecompressor.Plo@am__quote@
@@ -408,6 +415,13 @@ NativeIO.lo: src/org/apache/hadoop/io/na
 @AMDEP_TRUE@@am__fastdepCC_FALSE@	DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
 @am__fastdepCC_FALSE@	$(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -c -o NativeIO.lo `test -f 'src/org/apache/hadoop/io/nativeio/NativeIO.c' || echo '$(srcdir)/'`src/org/apache/hadoop/io/nativeio/NativeIO.c
 
+JniBasedUnixGroupsNetgroupMapping.lo: src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c
+@am__fastdepCC_TRUE@	if $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -MT JniBasedUnixGroupsNetgroupMapping.lo -MD -MP -MF "$(DEPDIR)/JniBasedUnixGroupsNetgroupMapping.Tpo" -c -o JniBasedUnixGroupsNetgroupMapping.lo `test -f 'src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c' || echo '$(srcdir)/'`src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c; \
+@am__fastdepCC_TRUE@	then mv -f "$(DEPDIR)/JniBasedUnixGroupsNetgroupMapping.Tpo" "$(DEPDIR)/JniBasedUnixGroupsNetgroupMapping.Plo"; else rm -f "$(DEPDIR)/JniBasedUnixGroupsNetgroupMapping.Tpo"; exit 1; fi
+@AMDEP_TRUE@@am__fastdepCC_FALSE@	source='src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c' object='JniBasedUnixGroupsNetgroupMapping.lo' libtool=yes @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCC_FALSE@	DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCC_FALSE@	$(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -c -o JniBasedUnixGroupsNetgroupMapping.lo `test -f 'src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c' || echo '$(srcdir)/'`src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c
+
 mostlyclean-libtool:
 	-rm -f *.lo