Browse Source

HADOOP-4656. Add a user to groups mapping service (boryas and acmurthy_)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@892066 13f79535-47bb-0310-9956-ffa450edef68
Boris Shkolnik 15 years ago
parent
commit
ae93ba7501

+ 2 - 0
CHANGES.txt

@@ -68,6 +68,8 @@ Trunk (unreleased changes)
 
     HADOOP-6426. Create ant build for running EC2 unit tests. (tomwhite)
 
+    HADOOP-4656. Add a user to groups mapping service. (boryas, acmurthy)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 2 - 0
src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -126,5 +126,7 @@ public class CommonConfigurationKeys {
   public static final String  HADOOP_JOB_UGI_KEY = "hadoop.job.ugi";
   public static final String  HADOOP_UTIL_HASH_TYPE_KEY = "hadoop.util.hash.type";
   public static final String  HADOOP_UTIL_HASH_TYPE_DEFAULT = "murmur";
+  public static final String  HADOOP_SECURITY_GROUP_MAPPING = "hadoop.security.group.mapping";
+  public static final String  HADOOP_SECURITY_GROUPS_CACHE_SECS = "hadoop.security.groups.cache.secs";
 }
 

+ 9 - 3
src/java/org/apache/hadoop/ipc/Server.java

@@ -67,6 +67,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.ipc.metrics.RpcMetrics;
 import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.UserGroupInformation;
 
 /** An abstract IPC service.  IPC calls take a single {@link Writable} as a
  * parameter, and return a {@link Writable} as their value.  A service runs on
@@ -893,8 +894,13 @@ public abstract class Server {
       }
       
       // TODO: Get the user name from the GSS API for Kerberbos-based security
-      // Create the user subject
-      user = SecurityUtil.getSubject(header.getUgi());
+      // Create the user subject; however use the groups as defined on the
+      // server-side, don't trust the user groups provided by the client
+      UserGroupInformation ugi = header.getUgi();
+      user = null;
+      if(ugi != null) {
+        user = SecurityUtil.getSubject(conf, header.getUgi().getUserName());
+      }
     }
     
     private void processData() throws  IOException, InterruptedException {
@@ -905,7 +911,7 @@ public abstract class Server {
       if (LOG.isDebugEnabled())
         LOG.debug(" got #" + id);
 
-      Writable param = ReflectionUtils.newInstance(paramClass, conf);           // read param
+      Writable param = ReflectionUtils.newInstance(paramClass, conf);//read param
       param.readFields(dis);        
         
       Call call = new Call(id, param, this);

+ 37 - 0
src/java/org/apache/hadoop/security/GroupMappingServiceProvider.java

@@ -0,0 +1,37 @@
+/**
+ * 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.List;
+
+/**
+ * An interface for the implementation of a user-to-groups mapping service
+ * used by {@link Groups}.
+ */
+interface GroupMappingServiceProvider {
+  
+  /**
+   * Get all various {@link Group} memberships of a given {@link User}.
+   * Returns EMPTY list in case of non-existing user
+   * @param user <code>User</code> name
+   * @return <code>Group</code> memberships of <code>user</code>
+   * @throws IOException
+   */
+  public List<String> getGroups(String user) throws IOException;
+}

+ 112 - 0
src/java/org/apache/hadoop/security/Groups.java

@@ -0,0 +1,112 @@
+/**
+ * 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.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * A user-to-groups mapping service.
+ * 
+ * {@link Groups} allows for server to get the various {@link Group} memberships
+ * of a given {@link User} via the {@link #getGroups(String)} call, thus ensuring 
+ * a consistent user-to-groups mapping and protects against vagaries of different 
+ * mappings on servers and clients in a Hadoop cluster. 
+ */
+public class Groups {
+  private static final Log LOG = LogFactory.getLog(Groups.class);
+  
+  private final GroupMappingServiceProvider impl;
+  
+  private final Map<String, CachedGroups> userToGroupsMap = 
+    new ConcurrentHashMap<String, CachedGroups>();
+  private final long cacheTimeout;
+
+  public Groups(Configuration conf) {
+    impl = 
+      ReflectionUtils.newInstance(
+          conf.getClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING, 
+                        ShellBasedUnixGroupsMapping.class, 
+                        GroupMappingServiceProvider.class), 
+          conf);
+    
+    cacheTimeout = 
+      conf.getLong(CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS, 5*60) * 1000;
+    
+    LOG.info("Group mapping impl=" + impl.getClass().getName() + 
+        "; cacheTimeout=" + cacheTimeout);
+  }
+  
+  /**
+   * Get the {@link Group} memberships of a given {@link User}.
+   * @param user <code>User</code> name
+   * @return the <code>Group</code> memberships of <code>user</code>
+   * @throws IOException
+   */
+  public List<String> getGroups(String user) throws IOException {
+    // Return cached value if available
+    CachedGroups groups = userToGroupsMap.get(user);
+    long now = System.currentTimeMillis();
+    // if cache has a value and it hasn't expired
+    if (groups != null && (groups.getTimestamp() + cacheTimeout > now)) {
+      LOG.info("Returning cached groups for '" + user + "'");
+      return groups.getGroups();
+    }
+    
+    // Create and cache user's groups
+    groups = new CachedGroups(impl.getGroups(user));
+    userToGroupsMap.put(user, groups);
+    LOG.info("Returning fetched groups for '" + user + "'");
+    return groups.getGroups();
+  }
+  
+  /**
+   * Refresh all user-to-groups mappings.
+   */
+  public void refresh() {
+    LOG.info("clearing userToGroupsMap cache");
+    userToGroupsMap.clear();
+  }
+  
+  private static class CachedGroups {
+    final long timestamp;
+    final List<String> groups;
+    
+    CachedGroups(List<String> groups) {
+      this.groups = groups;
+      this.timestamp = System.currentTimeMillis();
+    }
+
+    public long getTimestamp() {
+      return timestamp;
+    }
+
+    public List<String> getGroups() {
+      return groups;
+    }
+  }
+}

+ 42 - 0
src/java/org/apache/hadoop/security/RefreshUserToGroupMappingsProtocol.java

@@ -0,0 +1,42 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.VersionedProtocol;
+
+/**
+ * Protocol use 
+ *
+ */
+public interface RefreshUserToGroupMappingsProtocol extends VersionedProtocol {
+  
+  /**
+   * Version 1: Initial version.
+   */
+  public static final long versionID = 1L;
+
+  /**
+   * Refresh {@link User} to {@link Group} mappings.
+   * @param conf
+   * @throws IOException
+   */
+  public void refreshUserToGroupsMappings(Configuration conf) throws IOException;
+}

+ 57 - 3
src/java/org/apache/hadoop/security/SecurityUtil.java

@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.security;
 
+import java.io.IOException;
 import java.security.Policy;
 import java.security.Principal;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
 
@@ -41,6 +43,8 @@ public class SecurityUtil {
                                    PolicyProvider.DEFAULT_POLICY_PROVIDER));
   }
   
+  private static Groups GROUPS = null;
+  
   /**
    * Set the global security policy for Hadoop.
    * 
@@ -61,6 +65,18 @@ public class SecurityUtil {
     return Policy.getPolicy();
   }
   
+  /**
+   * Get the {@link Groups} being used to map user-to-groups.
+   * @return the <code>Groups</code> being used to map user-to-groups.
+   */
+  public static Groups getUserToGroupsMappingService(Configuration conf) {
+    if(GROUPS == null) {
+      LOG.info(" Creating new Groups object");
+      GROUPS = new Groups(conf);
+    }
+    return GROUPS;
+  }
+  
   /**
    * Get the {@link Subject} for the user identified by <code>ugi</code>.
    * @param ugi user
@@ -70,9 +86,9 @@ public class SecurityUtil {
     if (ugi == null) {
       return null;
     }
-    
-    Set<Principal> principals =       // Number of principals = username + #groups 
-      new HashSet<Principal>(ugi.getGroupNames().length+1);
+    // Number of principals = username + #groups + ugi
+    Set<Principal> principals =   
+      new HashSet<Principal>(ugi.getGroupNames().length+1+1);
     User userPrincipal = new User(ugi.getUserName()); 
     principals.add(userPrincipal);
     for (String group : ugi.getGroupNames()) {
@@ -86,6 +102,44 @@ public class SecurityUtil {
     return user;
   }
   
+  /**
+   * Get the {@link Subject} for the user identified by <code>userName</code>.
+   * @param userName user name
+   * @return the {@link Subject} for the user identified by <code>userName</code>
+   * @throws IOException
+   */
+  public static Subject getSubject(Configuration conf, String userName) 
+    throws IOException {
+    if (userName == null) {
+      return null;
+    }
+    
+    Set<Principal> principals = new HashSet<Principal>();
+    User userPrincipal = new User(userName); 
+    principals.add(userPrincipal);
+    
+    // Get user's groups
+    List<String> groups = getUserToGroupsMappingService(conf).getGroups(userName);
+    StringBuffer sb = new StringBuffer("Groups for '" + userName + "': <");
+    for (String group : groups) {
+      Group groupPrincipal = new Group(group);
+      principals.add(groupPrincipal);
+      sb.append(group + " ");
+    }
+    sb.append(">");
+    LOG.info(sb);
+    
+    // Create the ugi with the right groups
+    UserGroupInformation ugi = 
+      new UnixUserGroupInformation(userName, 
+                                   groups.toArray(new String[groups.size()]));
+    principals.add(ugi);
+    Subject user = 
+      new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());
+    
+    return user;
+  }
+  
   /**
    * Class representing a configured access control list.
    */

+ 76 - 0
src/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java

@@ -0,0 +1,76 @@
+/**
+ * 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.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} which
+ * exec's the <code>groups</code> shell command to fetch the {@link Group}
+ * memberships of a given {@link User}.
+ */
+public class ShellBasedUnixGroupsMapping implements GroupMappingServiceProvider {
+  Map<String, List<String>> userGroups = 
+    new ConcurrentHashMap<String, List<String>>();
+  
+  private static final Log LOG = LogFactory.getLog(ShellBasedUnixGroupsMapping.class);
+  
+  @Override
+  public List<String> getGroups(String user) throws IOException {
+    List<String> groups = userGroups.get(user);
+    if (groups == null) {
+      groups = getUnixGroups(user);
+      userGroups.put(user, groups);
+    }
+    return groups;
+  }
+
+  /** 
+   * 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 static List<String> getUnixGroups(final String user) throws IOException {
+    String result = "";
+    try {
+      result = Shell.execCommand(Shell.getGROUPS_FOR_USER_COMMAND(user));
+    } catch (ExitCodeException e) {
+      // if we didn't get the group - just return empty list;
+      LOG.warn("got exception trying to get groups for user " + user, e);
+    }
+    
+    StringTokenizer tokenizer = new StringTokenizer(result);
+    List<String> groups = new LinkedList<String>();
+    while (tokenizer.hasMoreTokens()) {
+      groups.add(tokenizer.nextToken());
+    }
+    return groups;
+  }
+}

+ 6 - 3
src/java/org/apache/hadoop/security/UnixUserGroupInformation.java

@@ -95,10 +95,13 @@ public class UnixUserGroupInformation extends UserGroupInformation {
    * @exception IllegalArgumentException if any argument is null
    */
   private void setUserGroupNames(String userName, String[] groupNames) {
-    if (userName==null || userName.length()==0 ||
-        groupNames== null || groupNames.length==0) {
+    if (userName==null || userName.length()==0) {
       throw new IllegalArgumentException(
-          "Parameters should not be null or an empty string/array");
+          "username should not be null nor empty");
+    }
+    if(groupNames == null) {
+      throw new IllegalArgumentException(
+      "group names array should not be null");
     }
     for (int i=0; i<groupNames.length; i++) {
       if(groupNames[i] == null || groupNames[i].length() == 0) {

+ 5 - 0
src/java/org/apache/hadoop/util/Shell.java

@@ -47,6 +47,11 @@ abstract public class Shell {
   public static String[] getGROUPS_COMMAND() {
     return new String[]{"bash", "-c", "groups"};
   }
+  /** a Unix command to get a given user's groups list */
+  public static String[] getGROUPS_FOR_USER_COMMAND(final String user) {
+    //'groups username' command return is non-consistent across different unixes
+    return new String [] {"bash", "-c", "id -Gn " + user};
+  }
   /** a Unix command to set permission */
   public static final String SET_PERMISSION_COMMAND = "chmod";
   /** a Unix command to set owner */

+ 77 - 4
src/test/core/org/apache/hadoop/security/TestUnixUserGroupInformation.java

@@ -18,11 +18,21 @@
 
 package org.apache.hadoop.security;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.TestWritable;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.security.auth.Subject;
 
 import junit.framework.TestCase;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.TestWritable;
+import org.apache.hadoop.util.Shell;
+
 /** Unit tests for UnixUserGroupInformation */
 public class TestUnixUserGroupInformation extends TestCase {
   final private static String USER_NAME = "user1";
@@ -31,6 +41,7 @@ public class TestUnixUserGroupInformation extends TestCase {
   final private static String GROUP3_NAME = "group3";
   final private static String[] GROUP_NAMES = 
                       new String[]{GROUP1_NAME, GROUP2_NAME, GROUP3_NAME};
+  final private static Configuration conf = new Configuration();
   
   /** Test login method */
   public void testLogin() throws Exception {
@@ -63,7 +74,6 @@ public class TestUnixUserGroupInformation extends TestCase {
     testConstructorFailures(null, GROUP_NAMES);
     testConstructorFailures("", GROUP_NAMES);
     testConstructorFailures(USER_NAME, null);
-    testConstructorFailures(USER_NAME, new String[0]);
     testConstructorFailures(USER_NAME, new String[]{null});
     testConstructorFailures(USER_NAME, new String[]{""});
     testConstructorFailures(USER_NAME, new String[]{GROUP1_NAME, null});
@@ -98,6 +108,69 @@ public class TestUnixUserGroupInformation extends TestCase {
   public void testWritable() throws Exception {
     UnixUserGroupInformation ugi = new UnixUserGroupInformation(
         USER_NAME, GROUP_NAMES);
-    TestWritable.testWritable(ugi, new Configuration());
+    TestWritable.testWritable(ugi, conf);
+  }
+  
+  /**
+   * given user name - get all the groups.
+   */
+  public void testGetServerSideGroups() throws IOException {
+    
+    // get the user name
+    Process pp = Runtime.getRuntime().exec("whoami");
+    BufferedReader br = new BufferedReader(new InputStreamReader(pp.getInputStream()));
+    String userName = br.readLine().trim();
+    // get the groups
+    pp = Runtime.getRuntime().exec("id -Gn");
+    br = new BufferedReader(new InputStreamReader(pp.getInputStream()));
+    String line = br.readLine();
+    System.out.println(userName + ":" + line);
+   
+    List<String> groups = new ArrayList<String> ();    
+    for(String s: line.split("[\\s]")) {
+      groups.add(s);
+    }
+    
+    boolean ugiIsIn = false;
+    
+    // get groups on the server side
+    int numberOfGroups = 0;
+    Subject subject = SecurityUtil.getSubject(conf, userName);
+    System.out.println("for user="+userName+" prinicipals are:");
+    for(Principal p : subject.getPrincipals()) {
+      if(p instanceof User) {
+        System.out.println("USER: " + p.getName());
+        assertTrue("user name is not the same as in the Subject: " + p.getName(),
+            userName.equals(p.getName()));
+      }
+      if(p instanceof Group) {
+        numberOfGroups++;
+        System.out.println("GROUP: " + p.getName());
+        assertTrue("Subject contains invalid group " + p.getName(), 
+            groups.contains(p.getName()));
+      }
+      if(p instanceof UserGroupInformation) {
+        System.out.println("UGI: " + p.getName());
+        ugiIsIn = true;
+      }
+    }
+    assertTrue("UGI object is not in the Subject", ugiIsIn);
+    assertEquals("number of groups in subject doesn't match actual # groups", 
+        numberOfGroups, groups.size());
+    
+    // negative test - get Subject for non-existing user
+    // should return empty groups
+    subject = SecurityUtil.getSubject(conf, "fakeUser");
+    for(Principal p : subject.getPrincipals()) {
+      if(p instanceof User) {
+        System.out.println("USER: " + p.getName());
+        assertTrue("user name (fakeUser) is not the same as in the Subject: " +
+            p.getName(), "fakeUser".equals(p.getName()));
+      }
+      if(p instanceof Group) {
+        fail("fakeUser should have no groups");
+      }
+    }
+    
   }
 }