Explorar o código

HADOOP-6299. Reimplement the UserGroupInformation to use the OS
specific and Kerberos JAAS login. (omalley)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@903560 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley %!s(int64=15) %!d(string=hai) anos
pai
achega
0c5734e4ac
Modificáronse 30 ficheiros con 1059 adicións e 1656 borrados
  1. 2 1
      .eclipse.templates/.classpath
  2. 3 0
      CHANGES.txt
  3. 5 0
      ivy.xml
  4. 3 0
      ivy/libraries.properties
  5. 1 0
      src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
  6. 3 14
      src/java/org/apache/hadoop/fs/FileSystem.java
  7. 6 6
      src/java/org/apache/hadoop/ipc/ConnectionHeader.java
  8. 1 7
      src/java/org/apache/hadoop/ipc/RPC.java
  9. 52 49
      src/java/org/apache/hadoop/ipc/Server.java
  10. 0 25
      src/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  11. 0 70
      src/java/org/apache/hadoop/security/Group.java
  12. 3 3
      src/java/org/apache/hadoop/security/GroupMappingServiceProvider.java
  13. 21 7
      src/java/org/apache/hadoop/security/Groups.java
  14. 0 80
      src/java/org/apache/hadoop/security/PermissionChecker.java
  15. 1 1
      src/java/org/apache/hadoop/security/RefreshUserToGroupMappingsProtocol.java
  16. 0 213
      src/java/org/apache/hadoop/security/SecurityUtil.java
  17. 3 3
      src/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java
  18. 0 437
      src/java/org/apache/hadoop/security/UnixUserGroupInformation.java
  19. 44 33
      src/java/org/apache/hadoop/security/User.java
  20. 520 64
      src/java/org/apache/hadoop/security/UserGroupInformation.java
  21. 143 0
      src/java/org/apache/hadoop/security/authorize/AccessControlList.java
  22. 0 181
      src/java/org/apache/hadoop/security/authorize/ConfiguredPolicy.java
  23. 0 74
      src/java/org/apache/hadoop/security/authorize/ConnectionPermission.java
  24. 6 6
      src/java/org/apache/hadoop/security/authorize/Service.java
  25. 45 57
      src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
  26. 1 3
      src/test/core/org/apache/hadoop/ipc/TestRPC.java
  27. 0 176
      src/test/core/org/apache/hadoop/security/TestUnixUserGroupInformation.java
  28. 189 0
      src/test/core/org/apache/hadoop/security/TestUserGroupInformation.java
  29. 7 6
      src/test/core/org/apache/hadoop/security/authorize/TestAccessControlList.java
  30. 0 140
      src/test/core/org/apache/hadoop/security/authorize/TestConfiguredPolicy.java

+ 2 - 1
.eclipse.templates/.classpath

@@ -35,7 +35,8 @@
 	<classpathentry kind="lib" path="build/ivy/lib/Hadoop-Core/test/ftplet-api-1.0.0.jar"/>
 	<classpathentry kind="lib" path="build/ivy/lib/Hadoop-Core/test/ftpserver-core-1.0.0.jar"/>
 	<classpathentry kind="lib" path="build/ivy/lib/Hadoop-Core/test/ftpserver-deprecated-1.0.0-M2.jar"/>
-	<classpathentry kind="lib" path="build/ivy/lib/Hadoop-Core/test/mina-core-2.0.0-M5.jar"/>
+  <classpathentry kind="lib" path="build/ivy/lib/Hadoop-Core/test/mina-core-2.0.0-M5.jar"/>
+  <classpathentry kind="lib" path="build/ivy/lib/Hadoop-Core/test/mockito-all-1.8.0.jar"/>
 	<classpathentry kind="lib" path="build/test/core/classes"/>
 	<classpathentry kind="lib" path="build/classes"/>
 	<classpathentry kind="lib" path="conf"/>

+ 3 - 0
CHANGES.txt

@@ -4,6 +4,9 @@ Trunk (unreleased changes)
 
   INCOMPATIBLE CHANGES
 
+    HADOOP-6299. Reimplement the UserGroupInformation to use the OS
+    specific and Kerberos JAAS login. (omalley)
+
   NEW FEATURES
 
     HADOOP-6284. Add a new parameter, HADOOP_JAVA_PLATFORM_OPTS, to

+ 5 - 0
ivy.xml

@@ -294,5 +294,10 @@
       rev="${aspectj.version}"
       conf="common->default">
     </dependency>
+    <dependency org="org.mockito" 
+      name="mockito-all" 
+      rev="${mockito-all.version}" 
+      conf="common->default">
+    </dependency> 
     </dependencies>
 </ivy-module>

+ 3 - 0
ivy/libraries.properties

@@ -79,3 +79,6 @@ xmlenc.version=0.52
 xerces.version=1.4.4
 
 aspectj.version=1.6.5
+
+mockito-all.version=1.8.0
+

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

@@ -128,5 +128,6 @@ public class CommonConfigurationKeys {
   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";
+  public static final String  HADOOP_SECURITY_AUTHENTICATION = "hadoop.security.authentication";
 }
 

+ 3 - 14
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -35,8 +35,6 @@ import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
-import javax.security.auth.login.LoginException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -1318,9 +1316,6 @@ public abstract class FileSystem extends Configured implements Closeable {
     /** Default pattern character: Character set close. */
     private static final char  PAT_SET_CLOSE = ']';
       
-    GlobFilter() {
-    }
-      
     GlobFilter(String filePattern) throws IOException {
       setRegex(filePattern);
     }
@@ -1870,15 +1865,9 @@ public abstract class FileSystem extends Configured implements Closeable {
         scheme = uri.getScheme()==null?"":uri.getScheme().toLowerCase();
         authority = uri.getAuthority()==null?"":uri.getAuthority().toLowerCase();
         this.unique = unique;
-        UserGroupInformation ugi = UserGroupInformation.readFrom(conf);
-        if (ugi == null) {
-          try {
-            ugi = UserGroupInformation.login(conf);
-          } catch(LoginException e) {
-            LOG.warn("uri=" + uri, e);
-          }
-        }
-        username = ugi == null? null: ugi.getUserName();
+        
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        username = ugi.getUserName();
       }
 
       /** {@inheritDoc} */

+ 6 - 6
src/java/org/apache/hadoop/ipc/ConnectionHeader.java

@@ -25,7 +25,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 /**
@@ -36,7 +35,7 @@ class ConnectionHeader implements Writable {
   public static final Log LOG = LogFactory.getLog(ConnectionHeader.class);
   
   private String protocol;
-  private UserGroupInformation ugi = new UnixUserGroupInformation();
+  private UserGroupInformation ugi = null;
   
   public ConnectionHeader() {}
   
@@ -60,9 +59,10 @@ class ConnectionHeader implements Writable {
       protocol = null;
     }
     
-    boolean ugiPresent = in.readBoolean();
-    if (ugiPresent) {
-      ugi.readFields(in);
+    boolean ugiUsernamePresent = in.readBoolean();
+    if (ugiUsernamePresent) {
+      String username = in.readUTF();
+      ugi = UserGroupInformation.createRemoteUser(username);
     } else {
       ugi = null;
     }
@@ -73,7 +73,7 @@ class ConnectionHeader implements Writable {
     Text.writeString(out, (protocol == null) ? "" : protocol);
     if (ugi != null) {
       out.writeBoolean(true);
-      ugi.write(out);
+      out.writeUTF(ugi.getUserName());
     } else {
       out.writeBoolean(false);
     }

+ 1 - 7
src/java/org/apache/hadoop/ipc/RPC.java

@@ -29,7 +29,6 @@ import java.util.Map;
 import java.util.HashMap;
 
 import javax.net.SocketFactory;
-import javax.security.auth.login.LoginException;
 
 import org.apache.commons.logging.*;
 
@@ -205,12 +204,7 @@ public class RPC {
   public static Object getProxy(Class protocol, long clientVersion,
                                 InetSocketAddress addr, Configuration conf,
                                 SocketFactory factory) throws IOException {
-    UserGroupInformation ugi = null;
-    try {
-      ugi = UserGroupInformation.login(conf);
-    } catch (LoginException le) {
-      throw new RuntimeException("Couldn't login!");
-    }
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     return getProxy(protocol, clientVersion, addr, ugi, conf, factory);
   }
   

+ 52 - 49
src/java/org/apache/hadoop/ipc/Server.java

@@ -18,12 +18,18 @@
 
 package org.apache.hadoop.ipc;
 
-import java.io.IOException;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
-
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.net.BindException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketException;
+import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.nio.channels.CancelledKeyException;
 import java.nio.channels.ClosedChannelException;
@@ -33,41 +39,30 @@ import java.nio.channels.Selector;
 import java.nio.channels.ServerSocketChannel;
 import java.nio.channels.SocketChannel;
 import java.nio.channels.WritableByteChannel;
-
-import java.net.BindException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.net.SocketException;
-import java.net.UnknownHostException;
-
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import javax.security.auth.Subject;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
-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;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
 
 /** 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
@@ -76,6 +71,7 @@ import org.apache.hadoop.security.UserGroupInformation;
  * @see Client
  */
 public abstract class Server {
+  private final boolean authorize;
   
   /**
    * The first four bytes of Hadoop RPC connections
@@ -728,7 +724,7 @@ public abstract class Server {
     ConnectionHeader header = new ConnectionHeader();
     Class<?> protocol;
     
-    Subject user = null;
+    UserGroupInformation user = null;
 
     // Fake 'call' for failed authorization response
     private static final int AUTHROIZATION_FAILED_CALLID = -1;
@@ -899,14 +895,7 @@ public abstract class Server {
         throw new IOException("Unknown protocol: " + header.getProtocol());
       }
       
-      // TODO: Get the user name from the GSS API for Kerberbos-based security
-      // 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());
-      }
+      user = header.getUgi();
     }
     
     private void processData() throws  IOException, InterruptedException {
@@ -968,24 +957,23 @@ public abstract class Server {
           try {
             // Make the call as the user via Subject.doAs, thus associating
             // the call with the Subject
-            value = 
-              Subject.doAs(call.connection.user, 
-                           new PrivilegedExceptionAction<Writable>() {
-                              @Override
-                              public Writable run() throws Exception {
-                                // make the call
-                                return call(call.connection.protocol, 
-                                            call.param, call.timestamp);
-
-                              }
-                           }
-                          );
-              
-          } catch (PrivilegedActionException pae) {
-            Exception e = pae.getException();
-            LOG.info(getName()+", call "+call+": error: " + e, e);
-            errorClass = e.getClass().getName();
-            error = StringUtils.stringifyException(e);
+            if (call.connection.user == null) {
+              value = call(call.connection.protocol, call.param, 
+                           call.timestamp);
+            } else {
+              value = 
+                call.connection.user.doAs
+                  (new PrivilegedExceptionAction<Writable>() {
+                     @Override
+                     public Writable run() throws Exception {
+                       // make the call
+                       return call(call.connection.protocol, 
+                                   call.param, call.timestamp);
+
+                     }
+                   }
+                  );
+            }
           } catch (Throwable e) {
             LOG.info(getName()+", call "+call+": error: " + e, e);
             errorClass = e.getClass().getName();
@@ -1045,6 +1033,9 @@ public abstract class Server {
     this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
     this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
     this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
+    this.authorize = 
+      conf.getBoolean(ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, 
+                      false);
     
     // Start the listener here and let it bind to the port
     listener = new Listener();
@@ -1176,8 +1167,20 @@ public abstract class Server {
    * @param connection incoming connection
    * @throws AuthorizationException when the client isn't authorized to talk the protocol
    */
-  public void authorize(Subject user, ConnectionHeader connection) 
-  throws AuthorizationException {}
+  public void authorize(UserGroupInformation user, 
+                        ConnectionHeader connection
+                        ) throws AuthorizationException {
+    if (authorize) {
+      Class<?> protocol = null;
+      try {
+        protocol = getProtocolClass(connection.getProtocol(), getConf());
+      } catch (ClassNotFoundException cfne) {
+        throw new AuthorizationException("Unknown protocol: " + 
+                                         connection.getProtocol());
+      }
+      ServiceAuthorizationManager.authorize(user, protocol);
+    }
+  }
   
   /**
    * The number of open RPC conections

+ 0 - 25
src/java/org/apache/hadoop/ipc/WritableRpcEngine.java

@@ -24,23 +24,17 @@ import java.lang.reflect.Array;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 
-import java.net.ConnectException;
 import java.net.InetSocketAddress;
-import java.net.SocketTimeoutException;
 import java.io.*;
 import java.util.Map;
 import java.util.HashMap;
 
 import javax.net.SocketFactory;
-import javax.security.auth.Subject;
-import javax.security.auth.login.LoginException;
 
 import org.apache.commons.logging.*;
 
 import org.apache.hadoop.io.*;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
@@ -291,7 +285,6 @@ class WritableRpcEngine implements RpcEngine {
   public static class Server extends RPC.Server {
     private Object instance;
     private boolean verbose;
-    private boolean authorize = false;
 
     /** Construct an RPC server.
      * @param instance the instance whose methods will be called
@@ -325,9 +318,6 @@ class WritableRpcEngine implements RpcEngine {
       super(bindAddress, port, Invocation.class, numHandlers, conf, classNameBase(instance.getClass().getName()));
       this.instance = instance;
       this.verbose = verbose;
-      this.authorize = 
-        conf.getBoolean(ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, 
-                        false);
     }
 
     public Writable call(Class<?> protocol, Writable param, long receivedTime) 
@@ -390,21 +380,6 @@ class WritableRpcEngine implements RpcEngine {
         throw ioe;
       }
     }
-
-    @Override
-    public void authorize(Subject user, ConnectionHeader connection) 
-    throws AuthorizationException {
-      if (authorize) {
-        Class<?> protocol = null;
-        try {
-          protocol = getProtocolClass(connection.getProtocol(), getConf());
-        } catch (ClassNotFoundException cfne) {
-          throw new AuthorizationException("Unknown protocol: " + 
-                                           connection.getProtocol());
-        }
-        ServiceAuthorizationManager.authorize(user, protocol);
-      }
-    }
   }
 
   private static void log(String value) {

+ 0 - 70
src/java/org/apache/hadoop/security/Group.java

@@ -1,70 +0,0 @@
-/**
- * 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.security.Principal;
-
-/**
- * A group to which a user belongs to.
- */
-public class Group implements Principal {
-  final String group;
-
-  /**
-   * Create a new <code>Group</code> with the given groupname.
-   * @param group group name
-   */
-  public Group(String group) {
-    this.group = group;
-  }
-
-  @Override
-  public String getName() {
-    return group;
-  }
-
-  @Override
-  public String toString() {
-    return group;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((group == null) ? 0 : group.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    Group other = (Group) obj;
-    if (group == null) {
-      if (other.group != null)
-        return false;
-    } else if (!group.equals(other.group))
-      return false;
-    return true;
-  }
-}

+ 3 - 3
src/java/org/apache/hadoop/security/GroupMappingServiceProvider.java

@@ -27,10 +27,10 @@ import java.util.List;
 interface GroupMappingServiceProvider {
   
   /**
-   * Get all various {@link Group} memberships of a given {@link User}.
+   * Get all various group memberships of a given 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>
+   * @param user User's name
+   * @return group memberships of user
    * @throws IOException
    */
   public List<String> getGroups(String user) throws IOException;

+ 21 - 7
src/java/org/apache/hadoop/security/Groups.java

@@ -32,10 +32,10 @@ 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. 
+ * {@link Groups} allows for server to get the various group memberships
+ * of a given 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);
@@ -62,9 +62,9 @@ public class Groups {
   }
   
   /**
-   * 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>
+   * Get the group memberships of a given user.
+   * @param user User's name
+   * @return the group memberships of the user
    * @throws IOException
    */
   public List<String> getGroups(String user) throws IOException {
@@ -109,4 +109,18 @@ public class Groups {
       return groups;
     }
   }
+
+  private static Groups GROUPS = null;
+  
+  /**
+   * Get the groups being used to map user-to-groups.
+   * @return the groups being used to map user-to-groups.
+   */
+  public static Groups getUserToGroupsMappingService(Configuration conf) {
+    if(GROUPS == null) {
+      LOG.debug(" Creating new Groups object");
+      GROUPS = new Groups(conf);
+    }
+    return GROUPS;
+  }
 }

+ 0 - 80
src/java/org/apache/hadoop/security/PermissionChecker.java

@@ -1,80 +0,0 @@
-/**
- * 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.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/** Perform permission checking. */
-public class PermissionChecker {
-  static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
-
-  public final String user;
-  protected final Set<String> groups = new HashSet<String>();
-  public final boolean isSuper;
-
-  /**
-   * Checks if the caller has the required permission.
-   * @param owner username of the owner
-   * @param supergroup supergroup that the owner belongs to
-   */
-  public PermissionChecker(String owner, String supergroup
-      ) throws AccessControlException{
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUGI();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("ugi=" + ugi);
-    }
-
-    if (ugi != null) {
-      user = ugi.getUserName();
-      groups.addAll(Arrays.asList(ugi.getGroupNames()));
-      isSuper = user.equals(owner) || groups.contains(supergroup);
-    }
-    else {
-      throw new AccessControlException("ugi = null");
-    }
-  }
-
-  /**
-   * Check if the callers group contains the required values.
-   * @param group group to check
-   */
-  public boolean containsGroup(String group) {return groups.contains(group);}
-
-  /**
-   * Verify if the caller has the required permission. This will result into 
-   * an exception if the caller is not allowed to access the resource.
-   * @param owner owner of the system
-   * @param supergroup supergroup of the system
-   */
-  public static void checkSuperuserPrivilege(UserGroupInformation owner, 
-                                             String supergroup) 
-  throws AccessControlException {
-    PermissionChecker checker = 
-      new PermissionChecker(owner.getUserName(), supergroup);
-    if (!checker.isSuper) {
-      throw new AccessControlException("Access denied for user " 
-          + checker.user + ". Superuser privilege is required");
-    }
-  }
-}

+ 1 - 1
src/java/org/apache/hadoop/security/RefreshUserToGroupMappingsProtocol.java

@@ -34,7 +34,7 @@ public interface RefreshUserToGroupMappingsProtocol extends VersionedProtocol {
   public static final long versionID = 1L;
 
   /**
-   * Refresh {@link User} to {@link Group} mappings.
+   * Refresh user to group mappings.
    * @param conf
    * @throws IOException
    */

+ 0 - 213
src/java/org/apache/hadoop/security/SecurityUtil.java

@@ -1,213 +0,0 @@
-/**
- * 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.security.Policy;
-import java.security.Principal;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-import javax.security.auth.Subject;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.authorize.ConfiguredPolicy;
-import org.apache.hadoop.security.authorize.PolicyProvider;
-
-public class SecurityUtil {
-
-  private static final Log LOG = LogFactory.getLog(SecurityUtil.class);
-  
-  static {
-    // Set an empty default policy
-    setPolicy(new ConfiguredPolicy(new Configuration(), 
-                                   PolicyProvider.DEFAULT_POLICY_PROVIDER));
-  }
-  
-  private static Groups GROUPS = null;
-  
-  /**
-   * Set the global security policy for Hadoop.
-   * 
-   * @param policy {@link Policy} used for authorization.
-   */
-  public static void setPolicy(Policy policy) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Setting Hadoop security policy");
-    }
-    Policy.setPolicy(policy);
-  }
-
-  /**
-   * Get the current global security policy for Hadoop.
-   * @return the current {@link Policy}
-   */
-  public static Policy getPolicy() {
-    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
-   * @return the {@link Subject} for the user identified by <code>ugi</code>
-   */
-  public static Subject getSubject(UserGroupInformation ugi) {
-    if (ugi == null) {
-      return null;
-    }
-    // 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()) {
-      Group groupPrincipal = new Group(group);
-      principals.add(groupPrincipal);
-    }
-    principals.add(ugi);
-    Subject user = 
-      new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());
-    
-    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.
-   */
-  public static class AccessControlList {
-    
-    // Indicates an ACL string that represents access to all users
-    public static final String WILDCARD_ACL_VALUE = "*";
-
-    // Set of users who are granted access.
-    private Set<String> users;
-    // Set of groups which are granted access
-    private Set<String> groups;
-    // Whether all users are granted access.
-    private boolean allAllowed;
-    
-    /**
-     * Construct a new ACL from a String representation of the same.
-     * 
-     * The String is a a comma separated list of users and groups.
-     * The user list comes first and is separated by a space followed 
-     * by the group list. For e.g. "user1,user2 group1,group2"
-     * 
-     * @param aclString String representation of the ACL
-     */
-    public AccessControlList(String aclString) {
-      users = new TreeSet<String>();
-      groups = new TreeSet<String>();
-      if (aclString.contains(WILDCARD_ACL_VALUE) && 
-          aclString.trim().equals(WILDCARD_ACL_VALUE)) {
-        allAllowed = true;
-      } else {
-        String[] userGroupStrings = aclString.split(" ", 2);
-        
-        if (userGroupStrings.length >= 1) {
-          String[] usersStr = userGroupStrings[0].split(",");
-          if (usersStr.length >= 1) {
-            addToSet(users, usersStr);
-          }
-        }
-        
-        if (userGroupStrings.length == 2) {
-          String[] groupsStr = userGroupStrings[1].split(",");
-          if (groupsStr.length >= 1) {
-            addToSet(groups, groupsStr);
-          }
-        }
-      }
-    }
-    
-    public boolean allAllowed() {
-      return allAllowed;
-    }
-    
-    public Set<String> getUsers() {
-      return users;
-    }
-    
-    public Set<String> getGroups() {
-      return groups;
-    }
-    
-    private static final void addToSet(Set<String> set, String[] strings) {
-      for (String s : strings) {
-        s = s.trim();
-        if (s.length() > 0) {
-          set.add(s);
-        }
-      }
-    }
-  }
-}

+ 3 - 3
src/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java

@@ -30,9 +30,9 @@ 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}.
+ * 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 ShellBasedUnixGroupsMapping implements GroupMappingServiceProvider {
   Map<String, List<String>> userGroups = 

+ 0 - 437
src/java/org/apache/hadoop/security/UnixUserGroupInformation.java

@@ -1,437 +0,0 @@
-/**
- * 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.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.StringTokenizer;
-import java.util.TreeSet;
-
-import javax.security.auth.login.LoginException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
-
-/** An implementation of UserGroupInformation in the Unix system */
-public class UnixUserGroupInformation extends UserGroupInformation {
-  public static final String DEFAULT_USERNAME = "DrWho";
-  public static final String DEFAULT_GROUP = "Tardis";
-
-  final static public String UGI_PROPERTY_NAME = "hadoop.job.ugi";
-  final static private HashMap<String, UnixUserGroupInformation> user2UGIMap =
-    new HashMap<String, UnixUserGroupInformation>();
-
-  /** Create an immutable {@link UnixUserGroupInformation} object. */
-  public static UnixUserGroupInformation createImmutable(String[] ugi) {
-    return new UnixUserGroupInformation(ugi) {
-      public void readFields(DataInput in) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-    };
-  }
-
-  private String userName;
-  private String[] groupNames;
-
-  /** Default constructor
-   */
-  public UnixUserGroupInformation() {
-  }
-
-  /** Constructor with parameters user name and its group names.
-   * The first entry in the groups list is the default  group.
-   * 
-   * @param userName a user's name
-   * @param groupNames groups list, first of which is the default group
-   * @exception IllegalArgumentException if any argument is null
-   */
-  public UnixUserGroupInformation(String userName, String[] groupNames) {
-    setUserGroupNames(userName, groupNames);
-  }
-
-  /** Constructor with parameter user/group names
-   * 
-   * @param ugi an array containing user/group names, the first
-   *                     element of which is the user name, the second of
-   *                     which is the default group name.
-   * @exception IllegalArgumentException if the array size is less than 2 
-   *                                     or any element is null.
-   */
-  public UnixUserGroupInformation(String[] ugi) {
-    if (ugi==null || ugi.length < 2) {
-      throw new IllegalArgumentException( "Parameter does contain at least "+
-          "one user name and one group name");
-    }
-    String[] groupNames = new String[ugi.length-1];
-    System.arraycopy(ugi, 1, groupNames, 0, groupNames.length);
-    setUserGroupNames(ugi[0], groupNames);
-  }
-  
-  /* Set this object's user name and group names
-   * 
-   * @param userName a user's name
-   * @param groupNames groups list, the first of which is the default group
-   * @exception IllegalArgumentException if any argument is null
-   */
-  private void setUserGroupNames(String userName, String[] groupNames) {
-    if (userName==null || userName.length()==0) {
-      throw new IllegalArgumentException(
-          "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) {
-        throw new IllegalArgumentException("A null group name at index " + i);
-      }
-    }
-    this.userName = userName;
-    this.groupNames = groupNames;
-  }
-
-  /** Return an array of group names
-   */
-  public String[] getGroupNames() {
-    return groupNames;
-  }
-
-  /** Return the user's name
-   */
-  public String getUserName() {
-    return userName;
-  }
-
-  /* The following two methods implements Writable interface */
-  final private static String UGI_TECHNOLOGY = "STRING_UGI"; 
-  /** Deserialize this object
-   * First check if this is a UGI in the string format.
-   * If no, throw an IOException; otherwise
-   * set this object's fields by reading them from the given data input
-   *  
-   *  @param in input stream
-   *  @exception IOException is thrown if encounter any error when reading
-   */
-  public void readFields(DataInput in) throws IOException {
-    // read UGI type first
-    String ugiType = Text.readString(in);
-    if (!UGI_TECHNOLOGY.equals(ugiType)) {
-      throw new IOException("Expect UGI prefix: " + UGI_TECHNOLOGY +
-          ", but receive a prefix: " + ugiType);
-    }
-    
-    // read this object
-    userName = Text.readString(in);
-    int numOfGroups = WritableUtils.readVInt(in);
-    groupNames = new String[numOfGroups];
-    for (int i = 0; i < numOfGroups; i++) {
-      groupNames[i] = Text.readString(in);
-    }
-  }
-
-  /** Serialize this object
-   * First write a string marking that this is a UGI in the string format,
-   * then write this object's serialized form to the given data output
-   * 
-   * @param out output stream
-   * @exception IOException if encounter any error during writing
-   */
-  public void write(DataOutput out) throws IOException {
-    // write a prefix indicating the type of UGI being written
-    Text.writeString(out, UGI_TECHNOLOGY);
-    // write this object
-    Text.writeString(out, userName);
-    WritableUtils.writeVInt(out, groupNames.length);
-    for (String groupName : groupNames) {
-      Text.writeString(out, groupName);
-    }
-  }
-
-  /* The following two methods deal with transferring UGI through conf. 
-   * In this pass of implementation we store UGI as a string in conf. 
-   * Later we may change it to be a more general approach that stores 
-   * it as a byte array */
-  /** Store the given <code>ugi</code> as a comma separated string in
-   * <code>conf</code> as a property <code>attr</code>
-   * 
-   * The String starts with the user name followed by the default group names,
-   * and other group names.
-   * 
-   * @param conf configuration
-   * @param attr property name
-   * @param ugi a UnixUserGroupInformation
-   */
-  public static void saveToConf(Configuration conf, String attr, 
-      UnixUserGroupInformation ugi ) {
-    conf.set(attr, ugi.toString());
-  }
-  
-  /** Read a UGI from the given <code>conf</code>
-   * 
-   * The object is expected to store with the property name <code>attr</code>
-   * as a comma separated string that starts
-   * with the user name followed by group names.
-   * If the property name is not defined, return null.
-   * It's assumed that there is only one UGI per user. If this user already
-   * has a UGI in the ugi map, return the ugi in the map.
-   * Otherwise, construct a UGI from the configuration, store it in the
-   * ugi map and return it.
-   * 
-   * @param conf configuration
-   * @param attr property name
-   * @return a UnixUGI
-   * @throws LoginException if the stored string is ill-formatted.
-   */
-  public static UnixUserGroupInformation readFromConf(
-      Configuration conf, String attr) throws LoginException {
-    String[] ugi = conf.getStrings(attr);
-    if(ugi == null) {
-      return null;
-    }
-    UnixUserGroupInformation currentUGI = null;
-    if (ugi.length>0 ){
-      currentUGI = user2UGIMap.get(ugi[0]);
-    }
-    if (currentUGI == null) {
-      try {
-        currentUGI = new UnixUserGroupInformation(ugi);
-        user2UGIMap.put(currentUGI.getUserName(), currentUGI);
-      } catch (IllegalArgumentException e) {
-        throw new LoginException("Login failed: "+e.getMessage());
-      }
-    }
-    
-    return currentUGI;
-  }
-  
-  /**
-   * Get current user's name and the names of all its groups from Unix.
-   * It's assumed that there is only one UGI per user. If this user already
-   * has a UGI in the ugi map, return the ugi in the map.
-   * Otherwise get the current user's information from Unix, store it
-   * in the map, and return it.
-   *
-   * If the current user's UNIX username or groups are configured in such a way
-   * to throw an Exception, for example if the user uses LDAP, then this method
-   * will use a the {@link #DEFAULT_USERNAME} and {@link #DEFAULT_GROUP}
-   * constants.
-   */
-  public static UnixUserGroupInformation login() throws LoginException {
-    try {
-      String userName;
-
-      // if an exception occurs, then uses the
-      // default user
-      try {
-        userName =  getUnixUserName();
-      } catch (Exception e) {
-        LOG.warn("Couldn't get unix username, using " + DEFAULT_USERNAME, e);
-        userName = DEFAULT_USERNAME;
-      }
-
-      // check if this user already has a UGI object in the ugi map
-      UnixUserGroupInformation ugi = user2UGIMap.get(userName);
-      if (ugi != null) {
-        return ugi;
-      }
-
-      /* get groups list from UNIX. 
-       * It's assumed that the first group is the default group.
-       */
-      String[]  groupNames;
-
-      // if an exception occurs, then uses the
-      // default group
-      try {
-        groupNames = getUnixGroups();
-      } catch (Exception e) {
-        LOG.warn("Couldn't get unix groups, using " + DEFAULT_GROUP, e);
-        groupNames = new String[1];
-        groupNames[0] = DEFAULT_GROUP;
-      }
-
-      // construct a Unix UGI
-      ugi = new UnixUserGroupInformation(userName, groupNames);
-      user2UGIMap.put(ugi.getUserName(), ugi);
-      return ugi;
-    } catch (Exception e) {
-      throw new LoginException("Login failed: "+e.getMessage());
-    }
-  }
-
-  /** Equivalent to login(conf, false). */
-  public static UnixUserGroupInformation login(Configuration conf)
-    throws LoginException {
-    return login(conf, false);
-  }
-  
-  /** Get a user's name & its group names from the given configuration; 
-   * If it is not defined in the configuration, get the current user's
-   * information from Unix.
-   * If the user has a UGI in the ugi map, return the one in
-   * the UGI map.
-   * 
-   *  @param conf either a job configuration or client's configuration
-   *  @param save saving it to conf?
-   *  @return UnixUserGroupInformation a user/group information
-   *  @exception LoginException if not able to get the user/group information
-   */
-  public static UnixUserGroupInformation login(Configuration conf, boolean save
-      ) throws LoginException {
-    UnixUserGroupInformation ugi = readFromConf(conf, UGI_PROPERTY_NAME);
-    if (ugi == null) {
-      ugi = login();
-      LOG.debug("Unix Login: " + ugi);
-      if (save) {
-        saveToConf(conf, UGI_PROPERTY_NAME, ugi);
-      }
-    }
-    return ugi;
-  }
-  
-  /* Return a string representation of a string array.
-   * Two strings are separated by a blank.
-   */
-  private static String toString(String[] strArray) {
-    if (strArray==null || strArray.length==0) {
-      return "";
-    }
-    StringBuilder buf = new StringBuilder(strArray[0]);
-    for (int i=1; i<strArray.length; i++) {
-      buf.append(' ');
-      buf.append(strArray[i]);
-    }
-    return buf.toString();
-  }
-  
-  /** Get current user's name from Unix by running the command whoami.
-   * 
-   * @return current user's name
-   * @throws IOException if encounter any error while running the command
-   */
-  static String getUnixUserName() throws IOException {
-    String[] result = executeShellCommand(
-        new String[]{Shell.USER_NAME_COMMAND});
-    if (result.length!=1) {
-      throw new IOException("Expect one token as the result of " + 
-          Shell.USER_NAME_COMMAND + ": " + toString(result));
-    }
-    return result[0];
-  }
-
-  /** Get the current user's group list from Unix by running the command groups
-   * 
-   * @return the groups list that the current user belongs to
-   * @throws IOException if encounter any error when running the command
-   */
-  private static String[] getUnixGroups() throws IOException {
-    return executeShellCommand(Shell.getGROUPS_COMMAND());
-  }
-  
-  /* Execute a command and return the result as an array of Strings */
-  private static String[] executeShellCommand(String[] command)
-  throws IOException {
-    String groups = Shell.execCommand(command);
-    StringTokenizer tokenizer = new StringTokenizer(groups);
-    int numOfTokens = tokenizer.countTokens();
-    String[] tokens = new String[numOfTokens];
-    for (int i=0; tokenizer.hasMoreTokens(); i++) {
-      tokens[i] = tokenizer.nextToken();
-    }
-
-    return tokens;
-  }
-
-  /** Decide if two UGIs are the same
-   *
-   * @param other other object
-   * @return true if they are the same; false otherwise.
-   */
-  public boolean equals(Object other) {
-    if (this == other) {
-      return true;
-    }
-    
-    if (!(other instanceof UnixUserGroupInformation)) {
-      return false;
-    }
-    
-    UnixUserGroupInformation otherUGI = (UnixUserGroupInformation)other;
-    
-    // check userName
-    if (userName == null) {
-      if (otherUGI.getUserName() != null) {
-        return false;
-      }
-    } else {
-      if (!userName.equals(otherUGI.getUserName())) {
-        return false;
-      }
-    }
-    
-    // checkGroupNames
-    if (groupNames == otherUGI.groupNames) {
-      return true;
-    }
-    if (groupNames.length != otherUGI.groupNames.length) {
-      return false;
-    }
-    // check default group name
-    if (groupNames.length>0 && !groupNames[0].equals(otherUGI.groupNames[0])) {
-      return false;
-    }
-    // check all group names, ignoring the order
-    return new TreeSet<String>(Arrays.asList(groupNames)).equals(
-           new TreeSet<String>(Arrays.asList(otherUGI.groupNames)));
-  }
-
-  /** Returns a hash code for this UGI. 
-   * The hash code for a UGI is the hash code of its user name string.
-   * 
-   * @return  a hash code value for this UGI.
-   */
-  public int hashCode() {
-    return getUserName().hashCode();
-  }
-  
-  /** Convert this object to a string
-   * 
-   * @return a comma separated string containing the user name and group names
-   */
-  public String toString() {
-    StringBuilder buf = new StringBuilder();
-    buf.append(userName);
-    for (String groupName : groupNames) {
-      buf.append(',');
-      buf.append(groupName);
-    }
-    return buf.toString();
-  }
-
-  @Override
-  public String getName() {
-    return toString();
-  }
-}

+ 44 - 33
src/java/org/apache/hadoop/security/User.java

@@ -20,51 +20,62 @@ package org.apache.hadoop.security;
 import java.security.Principal;
 
 /**
- * The username of a user.
+ * Save the full and short name of the user as a principal. This allows us to
+ * have a single type that we always look for when picking up user names.
  */
-public class User implements Principal {
-  final String user;
+class User implements Principal {
+  private final String fullName;
+  private final String shortName;
+
+  public User(String name) {
+    fullName = name;
+    int atIdx = name.indexOf('@');
+    if (atIdx == -1) {
+      shortName = name;
+    } else {
+      int slashIdx = name.indexOf('/');
+      if (slashIdx == -1 || atIdx < slashIdx) {
+        shortName = name.substring(0, atIdx);
+      } else {
+        shortName = name.substring(0, slashIdx);
+      }
+    }
+  }
 
   /**
-   * Create a new <code>User</code> with the given username.
-   * @param user user name
+   * Get the full name of the user.
    */
-  public User(String user) {
-    this.user = user;
-  }
-  
   @Override
   public String getName() {
-    return user;
+    return fullName;
   }
-
+  
+  /**
+   * Get the user name up to the first '/' or '@'
+   * @return the leading part of the user name
+   */
+  public String getShortName() {
+    return shortName;
+  }
+  
   @Override
-  public String toString() {
-    return user;
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (o == null || getClass() != o.getClass()) {
+      return false;
+    } else {
+      return fullName.equals(((User) o).fullName);
+    }
   }
-
+  
   @Override
   public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((user == null) ? 0 : user.hashCode());
-    return result;
+    return fullName.hashCode();
   }
-
+  
   @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    User other = (User) obj;
-    if (user == null) {
-      if (other.user != null)
-        return false;
-    } else if (!user.equals(other.user))
-      return false;
-    return true;
+  public String toString() {
+    return fullName;
   }
 }

+ 520 - 64
src/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -17,113 +17,569 @@
  */
 package org.apache.hadoop.security;
 
+import static org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate.Project.HDFS;
+import static org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate.Project.MAPREDUCE;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION;
+
 import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.AccessControlContext;
 import java.security.AccessController;
 import java.security.Principal;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import javax.security.auth.Subject;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.LoginContext;
 import javax.security.auth.login.LoginException;
+import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
+import javax.security.auth.spi.LoginModule;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+import com.sun.security.auth.NTUserPrincipal;
+import com.sun.security.auth.UnixPrincipal;
+import com.sun.security.auth.module.Krb5LoginModule;
 
-/** A {@link Writable} abstract class for storing user and groups information.
+/**
+ * User and group information for Hadoop.
+ * This class wraps around a JAAS Subject and provides methods to determine the
+ * user's username and groups. It supports both the Windows, Unix and Kerberos 
+ * login modules.
  */
-public abstract class UserGroupInformation implements Writable, Principal {
-  public static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
-  private static UserGroupInformation LOGIN_UGI = null;
+public class UserGroupInformation {
+  private static final Log LOG =  LogFactory.getLog(UserGroupInformation.class);
+  
+  /**
+   * A login module that looks at the Kerberos, Unix, or Windows principal and
+   * adds the corresponding UserName.
+   */
+  @InterfaceAudience.Private
+  public static class HadoopLoginModule implements LoginModule {
+    private Subject subject;
+
+    @Override
+    public boolean abort() throws LoginException {
+      return true;
+    }
+
+    private <T extends Principal> T getCanonicalUser(Class<T> cls) {
+      for(T user: subject.getPrincipals(cls)) {
+        return user;
+      }
+      return null;
+    }
+
+    @Override
+    public boolean commit() throws LoginException {
+      Principal user = null;
+      // if we are using kerberos, try it out
+      if (useKerberos) {
+        user = getCanonicalUser(KerberosPrincipal.class);
+      }
+      // if we don't have a kerberos user, use the OS user
+      if (user == null) {
+        user = getCanonicalUser(OS_PRINCIPAL_CLASS);
+      }
+      // if we found the user, add our principal
+      if (user != null) {
+        subject.getPrincipals().add(new User(user.getName()));
+        return true;
+      }
+      LOG.error("Can't find user in " + subject);
+      throw new LoginException("Can't find user name");
+    }
+
+    @Override
+    public void initialize(Subject subject, CallbackHandler callbackHandler,
+                           Map<String, ?> sharedState, Map<String, ?> options) {
+      this.subject = subject;
+    }
+
+    @Override
+    public boolean login() throws LoginException {
+      return true;
+    }
+
+    @Override
+    public boolean logout() throws LoginException {
+      return true;
+    }
+  }
+
+  /** Are the static variables that depend on configuration initialized? */
+  private static boolean isInitialized = false;
+  /** Should we use Kerberos configuration? */
+  private static boolean useKerberos;
+  /** Server-side groups fetching service */
+  private static Groups groups;
+  
+  /** 
+   * A method to initialize the fields that depend on a configuration.
+   * Must be called before useKerberos or groups is used.
+   */
+  private static synchronized void ensureInitialized() {
+    if (!isInitialized) {
+      initialize(new Configuration());
+    }
+  }
+
+  /**
+   * Set the configuration values for UGI.
+   * @param conf the configuration to use
+   */
+  private static synchronized void initialize(Configuration conf) {
+    String value = conf.get(HADOOP_SECURITY_AUTHENTICATION);
+    if ("simple".equals(value)) {
+      useKerberos = false;
+    } else if (value == null || "kerberos".equals(value)) {
+      useKerberos = true;
+    } else {
+      throw new IllegalArgumentException("Invalid attribute value for " +
+                                         HADOOP_SECURITY_AUTHENTICATION + 
+                                         " of " + value);
+    }
+    // If we haven't set up testing groups, use the configuration to find it
+    if (!(groups instanceof TestingGroups)) {
+      groups = Groups.getUserToGroupsMappingService(conf);
+    }
+    // Set the configuration for JAAS to be the Hadoop configuration. 
+    // This is done here rather than a static initializer to avoid a
+    // circular dependence.
+    javax.security.auth.login.Configuration.setConfiguration
+        (new HadoopConfiguration());
+    isInitialized = true;
+  }
+
+  /**
+   * Set the static configuration for UGI.
+   * In particular, set the security authentication mechanism and the
+   * group look up service.
+   * @param conf the configuration to use
+   */
+  public static void setConfiguration(Configuration conf) {
+    initialize(conf);
+  }
   
-  private static final ThreadLocal<Subject> currentUser =
-    new ThreadLocal<Subject>();
+  /**
+   * Determine if UserGroupInformation is using Kerberos to determine
+   * user identities or is relying on simple authentication
+   * 
+   * @return true if UGI is working in a secure environment
+   */
+  public static boolean isSecurityEnabled() {
+    ensureInitialized();
+    return useKerberos;
+  }
   
-  /** @return the {@link UserGroupInformation} for the current thread */ 
-  public static UserGroupInformation getCurrentUGI() {
-    Subject user = getCurrentUser();
+  /**
+   * Information about the logged in user.
+   */
+  private static UserGroupInformation loginUser = null;
+  private static String keytabPrincipal = null;
+  private static String keytabFile = null;
+
+  private final Subject subject;
+  private final Set<Token<? extends TokenIdentifier>> tokens =
+                  new LinkedHashSet<Token<? extends TokenIdentifier>>();
+  
+  private static final String OS_LOGIN_MODULE_NAME;
+  private static final Class<? extends Principal> OS_PRINCIPAL_CLASS;
+  private static final boolean windows = 
+                           System.getProperty("os.name").startsWith("Windows");
+  static {
+    if (windows) {
+      OS_LOGIN_MODULE_NAME = "com.sun.security.auth.module.NTLoginModule";
+      OS_PRINCIPAL_CLASS = NTUserPrincipal.class;
+    } else {
+      OS_LOGIN_MODULE_NAME = "com.sun.security.auth.module.UnixLoginModule";
+      OS_PRINCIPAL_CLASS = UnixPrincipal.class;
+    }
+  }
+  
+  /**
+   * A JAAS configuration that defines the login modules that we want
+   * to use for login.
+   */
+  private static class HadoopConfiguration 
+      extends javax.security.auth.login.Configuration {
+    private static final String SIMPLE_CONFIG_NAME = "hadoop-simple";
+    private static final String USER_KERBEROS_CONFIG_NAME = 
+      "hadoop-user-kerberos";
+    private static final String KEYTAB_KERBEROS_CONFIG_NAME = 
+      "hadoop-keytab-kerberos";
     
-    if (user == null) {
-      user = currentUser.get();
-      if (user == null) {
-        return null;
+    private static final AppConfigurationEntry OS_SPECIFIC_LOGIN =
+      new AppConfigurationEntry(OS_LOGIN_MODULE_NAME,
+                                LoginModuleControlFlag.REQUIRED,
+                                new HashMap<String,String>());
+    private static final AppConfigurationEntry HADOOP_LOGIN =
+      new AppConfigurationEntry(HadoopLoginModule.class.getName(),
+                                LoginModuleControlFlag.REQUIRED,
+                                new HashMap<String,String>());
+    private static final Map<String,String> USER_KERBEROS_OPTIONS = 
+      new HashMap<String,String>();
+    static {
+      USER_KERBEROS_OPTIONS.put("doNotPrompt", "true");
+      USER_KERBEROS_OPTIONS.put("useTicketCache", "true");
+    }
+    private static final AppConfigurationEntry USER_KERBEROS_LOGIN =
+      new AppConfigurationEntry(Krb5LoginModule.class.getName(),
+                                LoginModuleControlFlag.OPTIONAL,
+                                USER_KERBEROS_OPTIONS);
+    private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS = 
+      new HashMap<String,String>();
+    static {
+      KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
+      KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
+      KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
+    }
+    private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
+      new AppConfigurationEntry(Krb5LoginModule.class.getName(),
+                                LoginModuleControlFlag.REQUIRED,
+                                KEYTAB_KERBEROS_OPTIONS);
+    
+    private static final AppConfigurationEntry[] SIMPLE_CONF = 
+      new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, HADOOP_LOGIN};
+
+    private static final AppConfigurationEntry[] USER_KERBEROS_CONF =
+      new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, USER_KERBEROS_LOGIN,
+                                  HADOOP_LOGIN};
+
+    private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
+      new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN, HADOOP_LOGIN};
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
+      if (SIMPLE_CONFIG_NAME.equals(appName)) {
+        return SIMPLE_CONF;
+      } else if (USER_KERBEROS_CONFIG_NAME.equals(appName)) {
+        return USER_KERBEROS_CONF;
+      } else if (KEYTAB_KERBEROS_CONFIG_NAME.equals(appName)) {
+        KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
+        KEYTAB_KERBEROS_OPTIONS.put("principal", keytabPrincipal);
+        return KEYTAB_KERBEROS_CONF;
       }
+      return null;
     }
+  }
+
+  /**
+   * Create a UserGroupInformation for the given subject.
+   * This does not change the subject or acquire new credentials.
+   * @param subject the user's subject
+   */
+  UserGroupInformation(Subject subject) {
+    this.subject = subject;
+  }
+
+  /**
+   * Return the current user, including any doAs in the current stack.
+   * @return the current user
+   * @throws IOException if login fails
+   */
+  public static UserGroupInformation getCurrentUser() throws IOException {
+    AccessControlContext context = AccessController.getContext();
+    Subject subject = Subject.getSubject(context);
+    return subject == null ? getLoginUser() : new UserGroupInformation(subject);
+  }
+
+  /**
+   * Get the currently logged in user.
+   * @return the logged in user
+   * @throws IOException if login fails
+   */
+  public synchronized 
+  static UserGroupInformation getLoginUser() throws IOException {
+    if (loginUser == null) {
+      try {
+        LoginContext login;
+        if (isSecurityEnabled()) {
+          login = new LoginContext(HadoopConfiguration.USER_KERBEROS_CONFIG_NAME);
+        } else {
+          login = new LoginContext(HadoopConfiguration.SIMPLE_CONFIG_NAME);
+        }
+        login.login();
+        loginUser = new UserGroupInformation(login.getSubject());
+      } catch (LoginException le) {
+        throw new IOException("failure to login", le);
+      }
+    }
+    return loginUser;
+  }
+
+  /**
+   * Log a user in from a keytab file. Loads a user identity from a keytab
+   * file and login them in. They become the currently logged-in user.
+   * @param user the principal name to load from the keytab
+   * @param path the path to the keytab file
+   * @throws IOException if the keytab file can't be read
+   */
+  public synchronized
+  static void loginUserFromKeytab(String user,
+                                  String path
+                                  ) throws IOException {
+    if (!isSecurityEnabled())
+      return;
+
+    keytabFile = path;
+    keytabPrincipal = user;
+    try {
+      LoginContext login = 
+        new LoginContext(HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME);
+      login.login();
+      loginUser = new UserGroupInformation(login.getSubject());
+    } catch (LoginException le) {
+      throw new IOException("Login failure for " + user + " from keytab " + 
+                            path, le);
+    }
+  }
+
+  /**
+   * Create a user from a login name. It is intended to be used for remote
+   * users in RPC, since it won't have any credentials.
+   * @param user the full user principal name, must not be empty or null
+   * @return the UserGroupInformation for the remote user.
+   */
+  public static UserGroupInformation createRemoteUser(String user) {
+    if (user == null || "".equals(user)) {
+      throw new IllegalArgumentException("Null user");
+    }
+    Subject subject = new Subject();
+    subject.getPrincipals().add(new User(user));
+    return new UserGroupInformation(subject);
+  }
+  
+  /**
+   * This class is used for storing the groups for testing. It stores a local
+   * map that has the translation of usernames to groups.
+   */
+  private static class TestingGroups extends Groups {
+    private final Map<String, List<String>> userToGroupsMapping = 
+      new HashMap<String,List<String>>();
     
-    Set<UserGroupInformation> ugiPrincipals = 
-      user.getPrincipals(UserGroupInformation.class);
+    private TestingGroups() {
+      super(new org.apache.hadoop.conf.Configuration());
+    }
     
-    UserGroupInformation ugi = null;
-    if (ugiPrincipals != null && ugiPrincipals.size() == 1) {
-      ugi = ugiPrincipals.iterator().next();
-      if (ugi == null) {
-        throw new RuntimeException("Cannot find _current user_ UGI in the Subject!");
+    @Override
+    public List<String> getGroups(String user) {
+      List<String> result = userToGroupsMapping.get(user);
+      if (result == null) {
+        result = new ArrayList<String>();
       }
-    } else {
-      throw new RuntimeException("Cannot resolve current user from subject, " +
-      		                       "which had " + ugiPrincipals.size() + 
-      		                       " UGI principals!");
+      return result;
+    }
+
+    private void setUserGroups(String user, String[] groups) {
+      userToGroupsMapping.put(user, Arrays.asList(groups));
     }
+  }
+
+  /**
+   * Create a UGI for testing HDFS and MapReduce
+   * @param user the full user principal name
+   * @param userGroups the names of the groups that the user belongs to
+   * @return a fake user for running unit tests
+   */
+  @InterfaceAudience.LimitedPrivate({HDFS, MAPREDUCE})
+  public static UserGroupInformation createUserForTesting(String user, 
+                                                          String[] userGroups) {
+    ensureInitialized();
+    UserGroupInformation ugi = createRemoteUser(user);
+    // make sure that the testing object is setup
+    if (!(groups instanceof TestingGroups)) {
+      groups = new TestingGroups();
+    }
+    // add the user groups
+    ((TestingGroups) groups).setUserGroups(ugi.getShortUserName(), userGroups);
     return ugi;
   }
 
-  /** 
-   * Set the {@link UserGroupInformation} for the current thread
-   * @deprecated Use {@link #setCurrentUser(UserGroupInformation)} 
-   */ 
-  @Deprecated
-  public static void setCurrentUGI(UserGroupInformation ugi) {
-    setCurrentUser(ugi);
+  /**
+   * Get the user's login name.
+   * @return the user's name up to the first '/' or '@'.
+   */
+  public String getShortUserName() {
+    for (User p: subject.getPrincipals(User.class)) {
+      return p.getShortName();
+    }
+    return null;
   }
 
   /**
-   * Return the current user <code>Subject</code>.
-   * @return the current user <code>Subject</code>
+   * Get the user's full principal name.
+   * @return the user's full principal name.
    */
-  static Subject getCurrentUser() {
-    return Subject.getSubject(AccessController.getContext());
+  public String getUserName() {
+    for (User p: subject.getPrincipals(User.class)) {
+      return p.getName();
+    }
+    return null;
   }
-  
+
   /**
-   * Set the {@link UserGroupInformation} for the current thread
-   * WARNING - This method should be used only in test cases and other exceptional
-   * cases!
-   * @param ugi {@link UserGroupInformation} for the current thread
+   * Add a token to this UGI
+   * 
+   * @param token Token to be added
+   * @return true on successful add of new token
    */
-  public static void setCurrentUser(UserGroupInformation ugi) {
-    Subject user = SecurityUtil.getSubject(ugi);
-    currentUser.set(user);
+  public synchronized boolean addToken(Token<? extends TokenIdentifier> token) {
+    return tokens.add(token);
   }
   
-  /** Get username
+  /**
+   * Obtain the collection of tokens associated with this user.
    * 
-   * @return the user's name
+   * @return an unmodifiable collection of tokens associated with user
    */
-  public abstract String getUserName();
+  public synchronized Collection<Token<? extends TokenIdentifier>> getTokens() {
+    return Collections.unmodifiableSet(tokens);
+  }
+
+  /**
+   * Get the group names for this user.
+   * @return the list of users with the primary group first. If the command
+   *    fails, it returns an empty list.
+   */
+  public synchronized String[] getGroupNames() {
+    ensureInitialized();
+    try {
+      List<String> result = groups.getGroups(getShortUserName());
+      return result.toArray(new String[result.size()]);
+    } catch (IOException ie) {
+      LOG.warn("No groups available for user " + getShortUserName());
+      return new String[0];
+    }
+  }
   
-  /** Get the name of the groups that the user belong to
-   * 
-   * @return an array of group names
+  /**
+   * Return the username.
    */
-  public abstract String[] getGroupNames();
+  @Override
+  public String toString() {
+    return getUserName();
+  }
 
-  /** Login and return a UserGroupInformation object. */
-  public static UserGroupInformation login(Configuration conf
-      ) throws LoginException {
-    if (LOGIN_UGI == null) {
-      LOGIN_UGI = UnixUserGroupInformation.login(conf);
+  /**
+   * Compare the subjects to see if they are equal to each other.
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (o == this) {
+      return true;
+    } else if (o == null || getClass() != o.getClass()) {
+      return false;
+    } else {
+      return subject.equals(((UserGroupInformation) o).subject);
     }
-    return LOGIN_UGI;
   }
 
-  /** Read a {@link UserGroupInformation} from conf */
-  public static UserGroupInformation readFrom(Configuration conf
-      ) throws IOException {
+  /**
+   * Return the hash of the subject.
+   */
+  @Override
+  public int hashCode() {
+    return subject.hashCode();
+  }
+
+  /**
+   * Get the underlying subject from this ugi.
+   * @return the subject that represents this user.
+   */
+  protected Subject getSubject() {
+    return subject;
+  }
+
+  /**
+   * Run the given action as the user.
+   * @param <T> the return type of the run method
+   * @param action the method to execute
+   * @return the value from the run method
+   */
+  public <T> T doAs(PrivilegedAction<T> action) {
+    return Subject.doAs(subject, action);
+  }
+  
+  /**
+   * Run the given action as the user, potentially throwing an exception.
+   * @param <T> the return type of the run method
+   * @param action the method to execute
+   * @return the value from the run method
+   * @throws IOException if the action throws an IOException
+   * @throws Error if the action throws an Error
+   * @throws RuntimeException if the action throws a RuntimeException
+   * @throws InterruptedException if the action throws an InterruptedException
+   * @throws UndeclaredThrowableException if the action throws something else
+   */
+  public <T> T doAs(PrivilegedExceptionAction<T> action
+                    ) throws IOException, InterruptedException {
     try {
-      return UnixUserGroupInformation.readFromConf(conf,
-        UnixUserGroupInformation.UGI_PROPERTY_NAME);
-    } catch (LoginException e) {
-      throw (IOException)new IOException().initCause(e);
+      return Subject.doAs(subject, action);
+    } catch (PrivilegedActionException pae) {
+      Throwable cause = pae.getCause();
+      if (cause instanceof IOException) {
+        throw (IOException) cause;
+      } else if (cause instanceof Error) {
+        throw (Error) cause;
+      } else if (cause instanceof RuntimeException) {
+        throw (RuntimeException) cause;
+      } else if (cause instanceof InterruptedException) {
+        throw (InterruptedException) cause;
+      } else {
+        throw new UndeclaredThrowableException(pae,"Unknown exception in doAs");
+      }
+    }
+  }
+
+  private void print() throws IOException {
+    System.out.println("User: " + getUserName());
+    System.out.print("Group Ids: ");
+    System.out.println();
+    String[] groups = getGroupNames();
+    System.out.print("Groups: ");
+    for(int i=0; i < groups.length; i++) {
+      System.out.print(groups[i] + " ");
+    }
+    System.out.println();    
+  }
+
+  /**
+   * A test method to print out the current user's UGI.
+   * @param args if there are two arguments, read the user from the keytab
+   * and print it out.
+   * @throws Exception
+   */
+  public static void main(String [] args) throws Exception {
+  System.out.println("Getting UGI for current user");
+    UserGroupInformation ugi = getCurrentUser();
+    ugi.print();
+    System.out.println("UGI: " + ugi);
+    System.out.println("============================================================");
+    
+    if (args.length == 2) {
+      System.out.println("Getting UGI from keytab....");
+      loginUserFromKeytab(args[0], args[1]);
+      getCurrentUser().print();
+      System.out.println("Keytab: " + ugi);
     }
   }
 }

+ 143 - 0
src/java/org/apache/hadoop/security/authorize/AccessControlList.java

@@ -0,0 +1,143 @@
+/**
+ * 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.authorize;
+
+import java.util.Iterator;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * Class representing a configured access control list.
+ */
+public class AccessControlList {
+  
+  // Indicates an ACL string that represents access to all users
+  public static final String WILDCARD_ACL_VALUE = "*";
+
+  // Set of users who are granted access.
+  private Set<String> users;
+  // Set of groups which are granted access
+  private Set<String> groups;
+  // Whether all users are granted access.
+  private boolean allAllowed;
+  
+  /**
+   * Construct a new ACL from a String representation of the same.
+   * 
+   * The String is a a comma separated list of users and groups.
+   * The user list comes first and is separated by a space followed 
+   * by the group list. For e.g. "user1,user2 group1,group2"
+   * 
+   * @param aclString String representation of the ACL
+   */
+  public AccessControlList(String aclString) {
+    users = new TreeSet<String>();
+    groups = new TreeSet<String>();
+    if (aclString.contains(WILDCARD_ACL_VALUE) && 
+        aclString.trim().equals(WILDCARD_ACL_VALUE)) {
+      allAllowed = true;
+    } else {
+      String[] userGroupStrings = aclString.split(" ", 2);
+      
+      if (userGroupStrings.length >= 1) {
+        String[] usersStr = userGroupStrings[0].split(",");
+        if (usersStr.length >= 1) {
+          addToSet(users, usersStr);
+        }
+      }
+      
+      if (userGroupStrings.length == 2) {
+        String[] groupsStr = userGroupStrings[1].split(",");
+        if (groupsStr.length >= 1) {
+          addToSet(groups, groupsStr);
+        }
+      }
+    }
+  }
+  
+  public boolean isAllAllowed() {
+    return allAllowed;
+  }
+  
+  /**
+   * Get the names of users allowed for this service.
+   * @return the set of user names. the set must not be modified.
+   */
+  Set<String> getUsers() {
+    return users;
+  }
+  
+  /**
+   * Get the names of user groups allowed for this service.
+   * @return the set of group names. the set must not be modified.
+   */
+  Set<String> getGroups() {
+    return groups;
+  }
+
+  public boolean isUserAllowed(UserGroupInformation ugi) {
+    if (allAllowed || users.contains(ugi.getUserName())) {
+      return true;
+    } else {
+      for(String group: ugi.getGroupNames()) {
+        if (groups.contains(group)) {
+          return true;
+        }
+      }
+    }
+    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);
+      }
+    }
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    boolean first = true;
+    for(String user: users) {
+      if (!first) {
+        sb.append(",");
+      } else {
+        first = false;
+      }
+      sb.append(user);
+    }
+    if (!groups.isEmpty()) {
+      sb.append(" ");
+    }
+    first = true;
+    for(String group: groups) {
+      if (!first) {
+        sb.append(",");
+      } else {
+        first = false;
+      }
+      sb.append(group);
+    }
+    return sb.toString();    
+  }
+}

+ 0 - 181
src/java/org/apache/hadoop/security/authorize/ConfiguredPolicy.java

@@ -1,181 +0,0 @@
-/**
- * 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.authorize;
-
-import java.security.Permission;
-import java.security.PermissionCollection;
-import java.security.Policy;
-import java.security.Principal;
-import java.security.ProtectionDomain;
-import java.security.CodeSource;
-import java.security.Permissions;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.Group;
-import org.apache.hadoop.security.User;
-import org.apache.hadoop.security.SecurityUtil.AccessControlList;
-
-/**
- * A {@link Configuration} based security {@link Policy} for Hadoop.
- *
- * {@link ConfiguredPolicy} works in conjunction with a {@link PolicyProvider}
- * for providing service-level authorization for Hadoop.
- */
-public class ConfiguredPolicy extends Policy implements Configurable {
-  public static final String HADOOP_POLICY_FILE = "hadoop-policy.xml";
-  private static final Log LOG = LogFactory.getLog(ConfiguredPolicy.class);
-      
-  private Configuration conf;
-  private PolicyProvider policyProvider;
-  private volatile Map<Principal, Set<Permission>> permissions;
-  private volatile Set<Permission> allowedPermissions;
-
-  public ConfiguredPolicy(Configuration conf, PolicyProvider policyProvider) {
-    this.conf = conf;      
-    this.policyProvider = policyProvider;
-    refresh();
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-    refresh();
-  }
-
-  @Override
-  public boolean implies(ProtectionDomain domain, Permission permission) {
-    // Only make checks for domains having principals 
-    if(domain.getPrincipals().length == 0) {
-      return true; 
-    }
-
-    return super.implies(domain, permission);
-  }
-
-  /**
-   * {@inheritDoc}
-   * @return a writable permission collection
-   */
-  @Override
-  public PermissionCollection getPermissions(CodeSource codesource) {
-    return new Permissions();
-  }
-
-  /**
-   * {@inheritDoc}
-   * @return a writable permission collection
-   */
-  @Override
-  public PermissionCollection getPermissions(ProtectionDomain domain) {
-    PermissionCollection permissionCollection = super.getPermissions(domain);
-    for (Principal principal : domain.getPrincipals()) {
-      Set<Permission> principalPermissions = permissions.get(principal);
-      if (principalPermissions != null) {
-        for (Permission permission : principalPermissions) {
-          permissionCollection.add(permission);
-        }
-      }
-
-      for (Permission permission : allowedPermissions) {
-        permissionCollection.add(permission);
-      }
-    }
-    return permissionCollection;
-  }
-
-  @Override
-  public void refresh() {
-    // Get the system property 'hadoop.policy.file'
-    String policyFile = 
-      System.getProperty("hadoop.policy.file", HADOOP_POLICY_FILE);
-    
-    // Make a copy of the original config, and load the policy file
-    Configuration policyConf = new Configuration(conf);
-    policyConf.addResource(policyFile);
-    
-    Map<Principal, Set<Permission>> newPermissions = 
-      new HashMap<Principal, Set<Permission>>();
-    Set<Permission> newAllowPermissions = new HashSet<Permission>();
-
-    // Parse the config file
-    Service[] services = policyProvider.getServices();
-    if (services != null) {
-      for (Service service : services) {
-        AccessControlList acl = 
-          new AccessControlList(
-              policyConf.get(service.getServiceKey(), 
-                             AccessControlList.WILDCARD_ACL_VALUE)
-              );
-        
-        if (acl.allAllowed()) {
-          newAllowPermissions.add(service.getPermission());
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Policy - " + service.getPermission() + " * ");
-          }
-        } else {
-          for (String user : acl.getUsers()) {
-            addPermission(newPermissions, new User(user), service.getPermission());
-          }
-
-          for (String group : acl.getGroups()) {
-            addPermission(newPermissions, new Group(group), service.getPermission());
-          }
-        }
-      }
-    }
-
-    // Flip to the newly parsed permissions
-    allowedPermissions = newAllowPermissions;
-    permissions = newPermissions;
-  }
-
-  private void addPermission(Map<Principal, Set<Permission>> permissions,
-                             Principal principal, Permission permission) {
-    Set<Permission> principalPermissions = permissions.get(principal);
-    if (principalPermissions == null) {
-      principalPermissions = new HashSet<Permission>();
-      permissions.put(principal, principalPermissions);
-    }
-    principalPermissions.add(permission);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Policy - Adding  " + permission + " to " + principal);
-    }
-  }
-
-  /**
-   * For debugging: identify ourselves and the policyproviders 
-   *
-   * @return a string representation of the object.
-   */
-  @Override
-  public String toString() {
-    return "Hadoop ConfiguredPolicy " + super.toString() + " Policy provider "+ policyProvider;
-  }
-}

+ 0 - 74
src/java/org/apache/hadoop/security/authorize/ConnectionPermission.java

@@ -1,74 +0,0 @@
-/**
- * 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.authorize;
-
-import java.security.Permission;
-
-import org.apache.hadoop.ipc.VersionedProtocol;
-
-/**
- * {@link Permission} to initiate a connection to a given service.
- */
-public class ConnectionPermission extends Permission {
-
-  private static final long serialVersionUID = 1L;
-  private final Class<?> protocol;
-
-  /**
-   * {@link ConnectionPermission} for a given service.
-   * @param protocol service to be accessed
-   */
-  public ConnectionPermission(Class<?> protocol) {
-    super(protocol.getName());
-    this.protocol = protocol;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof ConnectionPermission) {
-      return protocol == ((ConnectionPermission)obj).protocol;
-    }
-    return false;
-  }
-
-  @Override
-  public String getActions() {
-    return "ALLOW";
-  }
-
-  @Override
-  public int hashCode() {
-    return protocol.hashCode();
-  }
-
-  @Override
-  public boolean implies(Permission permission) {
-    if (permission instanceof ConnectionPermission) {
-      ConnectionPermission that = (ConnectionPermission)permission;
-      if (that.protocol.equals(VersionedProtocol.class)) {
-        return true;
-      }
-      return this.protocol.equals(that.protocol);
-    }
-    return false;
-  }
-
-  public String toString() {
-    return "ConnectionPermission(" + protocol.getName() + ")";
-  }
-}

+ 6 - 6
src/java/org/apache/hadoop/security/authorize/Service.java

@@ -28,11 +28,11 @@ import java.security.Permission;
  */
 public class Service {
   private String key;
-  private Permission permission;
+  private Class<?> protocol;
   
   public Service(String key, Class<?> protocol) {
     this.key = key;
-    this.permission = new ConnectionPermission(protocol);
+    this.protocol = protocol;
   }
   
   /**
@@ -44,10 +44,10 @@ public class Service {
   }
   
   /**
-   * Get the {@link Permission} required to access the service.
-   * @return the {@link Permission} required to access the service
+   * Get the protocol for the service
+   * @return the {@link Class} for the protocol
    */
-  public Permission getPermission() {
-    return permission;
+  public Class<?> getProtocol() {
+    return protocol;
   }
 }

+ 45 - 57
src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java

@@ -17,19 +17,10 @@
  */
 package org.apache.hadoop.security.authorize;
 
-import java.security.AccessControlException;
-import java.security.AccessController;
-import java.security.Permission;
-import java.security.PrivilegedActionException;
-import java.security.PrivilegedExceptionAction;
-import java.util.Collections;
-import java.util.HashMap;
+import java.util.IdentityHashMap;
 import java.util.Map;
 
-import javax.security.auth.Subject;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 
 /**
@@ -37,9 +28,10 @@ import org.apache.hadoop.security.UserGroupInformation;
  * for incoming service requests.
  */
 public class ServiceAuthorizationManager {
+  private static final String HADOOP_POLICY_FILE = "hadoop-policy.xml";
 
-  private static final Log LOG = 
-    LogFactory.getLog(ServiceAuthorizationManager.class);
+  private static Map<Class<?>, AccessControlList> protocolToAcl =
+    new IdentityHashMap<Class<?>, AccessControlList>();
   
   /**
    * Configuration key for controlling service-level authorization for Hadoop.
@@ -47,9 +39,6 @@ public class ServiceAuthorizationManager {
   public static final String SERVICE_AUTHORIZATION_CONFIG = 
     "hadoop.security.authorization";
   
-  private static Map<Class<?>, Permission> protocolToPermissionMap = 
-    Collections.synchronizedMap(new HashMap<Class<?>, Permission>());
-
   /**
    * Authorize the user to access the protocol being used.
    * 
@@ -57,49 +46,48 @@ public class ServiceAuthorizationManager {
    * @param protocol service being accessed
    * @throws AuthorizationException on authorization failure
    */
-  public static void authorize(Subject user, Class<?> protocol) 
-  throws AuthorizationException {
-    Permission permission = protocolToPermissionMap.get(protocol);
-    if (permission == null) {
-      permission = new ConnectionPermission(protocol);
-      protocolToPermissionMap.put(protocol, permission);
+  public static void authorize(UserGroupInformation user, 
+                               Class<?> protocol
+                               ) throws AuthorizationException {
+    AccessControlList acl = protocolToAcl.get(protocol);
+    if (acl == null) {
+      throw new AuthorizationException("Protocol " + protocol + 
+                                       " is not known.");
+    }
+    if (!acl.isUserAllowed(user)) {
+      throw new AuthorizationException("User " + user.toString() + 
+                                       " is not authorized for protocol " + 
+                                       protocol);
     }
-    
-    checkPermission(user, permission);
   }
-  
-  /**
-   * Check if the given {@link Subject} has all of necessary {@link Permission} 
-   * set.
-   * 
-   * @param user <code>Subject</code> to be authorized
-   * @param permissions <code>Permission</code> set
-   * @throws AuthorizationException if the authorization failed
-   */
-  private static void checkPermission(final Subject user, 
-                                      final Permission... permissions) 
-  throws AuthorizationException {
-    try{
-      Subject.doAs(user, 
-                   new PrivilegedExceptionAction<Void>() {
-                     @Override
-                     public Void run() throws Exception {
-                       try {
-                         for(Permission permission : permissions) {
-                           AccessController.checkPermission(permission);
-                         }
-                       } catch (AccessControlException ace) {
-                         LOG.info("Authorization failed for " + 
-                                  UserGroupInformation.getCurrentUGI(), ace);
-                         throw new AuthorizationException(ace);
-                       }
-                      return null;
-                     }
-                   }
-                  );
-    } catch (PrivilegedActionException e) {
-      throw new AuthorizationException(e.getException());
+
+  public static synchronized void refresh(Configuration conf,
+                                          PolicyProvider provider) {
+    // Get the system property 'hadoop.policy.file'
+    String policyFile = 
+      System.getProperty("hadoop.policy.file", HADOOP_POLICY_FILE);
+    
+    // Make a copy of the original config, and load the policy file
+    Configuration policyConf = new Configuration(conf);
+    policyConf.addResource(policyFile);
+    
+    final Map<Class<?>, AccessControlList> newAcls =
+      new IdentityHashMap<Class<?>, AccessControlList>();
+
+    // Parse the config file
+    Service[] services = provider.getServices();
+    if (services != null) {
+      for (Service service : services) {
+        AccessControlList acl = 
+          new AccessControlList(
+              policyConf.get(service.getServiceKey(), 
+                             AccessControlList.WILDCARD_ACL_VALUE)
+              );
+        newAcls.put(service.getProtocol(), acl);
+      }
     }
+
+    // Flip to the newly parsed permissions
+    protocolToAcl = newAcls;
   }
-  
 }

+ 1 - 3
src/test/core/org/apache/hadoop/ipc/TestRPC.java

@@ -34,9 +34,7 @@ import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
 
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.hadoop.security.authorize.ConfiguredPolicy;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.Service;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
@@ -338,7 +336,7 @@ public class TestRPC extends TestCase {
   }
   
   private void doRPCs(Configuration conf, boolean expectFailure) throws Exception {
-    SecurityUtil.setPolicy(new ConfiguredPolicy(conf, new TestPolicyProvider()));
+    ServiceAuthorizationManager.refresh(conf, new TestPolicyProvider());
     
     Server server = RPC.getServer(TestProtocol.class,
                                   new TestImpl(), ADDRESS, 0, 5, true, conf);

+ 0 - 176
src/test/core/org/apache/hadoop/security/TestUnixUserGroupInformation.java

@@ -1,176 +0,0 @@
-/**
- * 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.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";
-  final private static String GROUP1_NAME = "group1";
-  final private static String GROUP2_NAME = "group2";
-  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 {
-    Configuration conf = new Configuration();
-    
-    // loin from unix
-    String userName = UnixUserGroupInformation.getUnixUserName();
-    UnixUserGroupInformation curUserGroupInfo = 
-        UnixUserGroupInformation.login(conf);
-    assertEquals(curUserGroupInfo.getUserName(), userName);
-    assertTrue(curUserGroupInfo == UnixUserGroupInformation.login(conf));
-    
-    // login from the configuration
-    UnixUserGroupInformation userGroupInfo = new UnixUserGroupInformation(
-        USER_NAME, GROUP_NAMES );
-    UnixUserGroupInformation.saveToConf(conf, 
-        UnixUserGroupInformation.UGI_PROPERTY_NAME, userGroupInfo);
-    curUserGroupInfo = UnixUserGroupInformation.login(conf);
-    assertEquals(curUserGroupInfo, userGroupInfo);
-    assertTrue(curUserGroupInfo == UnixUserGroupInformation.login(conf));
-  }
-
-  /** test constructor */
-  public void testConstructor() throws Exception {
-    UnixUserGroupInformation uugi = 
-      new UnixUserGroupInformation(USER_NAME, GROUP_NAMES);
-    assertEquals(uugi, new UnixUserGroupInformation( new String[]{
-       USER_NAME, GROUP1_NAME, GROUP2_NAME, GROUP3_NAME} ));  
-    // failure test
-    testConstructorFailures(null, GROUP_NAMES);
-    testConstructorFailures("", GROUP_NAMES);
-    testConstructorFailures(USER_NAME, null);
-    testConstructorFailures(USER_NAME, new String[]{null});
-    testConstructorFailures(USER_NAME, new String[]{""});
-    testConstructorFailures(USER_NAME, new String[]{GROUP1_NAME, null});
-    testConstructorFailures(USER_NAME, 
-        new String[]{GROUP1_NAME, null, GROUP2_NAME});
-  }
-  
-  private void testConstructorFailures(String userName, String[] groupNames) {
-    boolean gotException = false;
-    try {
-      new UnixUserGroupInformation(userName, groupNames);
-    } catch (Exception e) {
-      gotException = true;
-    }
-    assertTrue(gotException);
-  }
-  
-  public void testEquals() throws Exception {
-    UnixUserGroupInformation uugi = 
-      new UnixUserGroupInformation(USER_NAME, GROUP_NAMES);
-
-    assertEquals(uugi, uugi);
-    assertEquals(uugi, new UnixUserGroupInformation(USER_NAME, GROUP_NAMES));
-    assertEquals(uugi, new UnixUserGroupInformation(USER_NAME,
-        new String[]{GROUP1_NAME, GROUP3_NAME, GROUP2_NAME}));
-    assertFalse(uugi.equals(new UnixUserGroupInformation()));
-    assertFalse(uugi.equals(new UnixUserGroupInformation(USER_NAME,
-        new String[]{GROUP2_NAME, GROUP3_NAME, GROUP1_NAME})));
-  }
-  
-  /** test Writable */
-  public void testWritable() throws Exception {
-    UnixUserGroupInformation ugi = new UnixUserGroupInformation(
-        USER_NAME, GROUP_NAMES);
-    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");
-      }
-    }
-    
-  }
-}

+ 189 - 0
src/test/core/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -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.security;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import static org.mockito.Mockito.mock;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.junit.Test;
+
+public class TestUserGroupInformation {
+  final private static String USER_NAME = "user1@HADOOP.APACHE.ORG";
+  final private static String GROUP1_NAME = "group1";
+  final private static String GROUP2_NAME = "group2";
+  final private static String GROUP3_NAME = "group3";
+  final private static String[] GROUP_NAMES = 
+    new String[]{GROUP1_NAME, GROUP2_NAME, GROUP3_NAME};
+
+  /**
+   * given user name - get all the groups.
+   * Needs to happen before creating the test users
+   */
+  @Test
+  public void testGetServerSideGroups() throws IOException,
+                                               InterruptedException {
+    // 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);
+    }
+    
+    final UserGroupInformation login = UserGroupInformation.getCurrentUser();
+    assertEquals(userName, login.getShortUserName());
+    String[] gi = login.getGroupNames();
+    assertEquals(groups.size(), gi.length);
+    for(int i=0; i < gi.length; i++) {
+      assertEquals(groups.get(i), gi[i]);
+    }
+    
+    final UserGroupInformation fakeUser = 
+      UserGroupInformation.createRemoteUser("foo.bar");
+    fakeUser.doAs(new PrivilegedExceptionAction<Object>(){
+      @Override
+      public Object run() throws IOException {
+        UserGroupInformation current = UserGroupInformation.getCurrentUser();
+        assertFalse(current.equals(login));
+        assertEquals(current, fakeUser);
+        assertEquals(0, current.getGroupNames().length);
+        return null;
+      }});
+  }
+
+  /** Test login method */
+  @Test
+  public void testLogin() throws Exception {
+    // login from unix
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    assertEquals(UserGroupInformation.getCurrentUser(),
+                 UserGroupInformation.getLoginUser());
+    assertTrue(ugi.getGroupNames().length >= 1);
+
+    // ensure that doAs works correctly
+    UserGroupInformation userGroupInfo = 
+      UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES);
+    UserGroupInformation curUGI = 
+      userGroupInfo.doAs(new PrivilegedExceptionAction<UserGroupInformation>(){
+        public UserGroupInformation run() throws IOException {
+          return UserGroupInformation.getCurrentUser();
+        }});
+    // make sure in the scope of the doAs, the right user is current
+    assertEquals(curUGI, userGroupInfo);
+    // make sure it is not the same as the login user
+    assertFalse(curUGI.equals(UserGroupInformation.getLoginUser()));
+  }
+
+  /** test constructor */
+  @Test
+  public void testConstructor() throws Exception {
+    UserGroupInformation ugi = 
+      UserGroupInformation.createUserForTesting("user2/cron@HADOOP.APACHE.ORG", 
+                                                GROUP_NAMES);
+    // make sure the short and full user names are correct
+    assertEquals("user2/cron@HADOOP.APACHE.ORG", ugi.getUserName());
+    assertEquals("user2", ugi.getShortUserName());
+    ugi = UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES);
+    assertEquals("user1", ugi.getShortUserName());
+    
+    // failure test
+    testConstructorFailures(null);
+    testConstructorFailures("");
+  }
+
+  private void testConstructorFailures(String userName) {
+    boolean gotException = false;
+    try {
+      UserGroupInformation.createRemoteUser(userName);
+    } catch (Exception e) {
+      gotException = true;
+    }
+    assertTrue(gotException);
+  }
+
+  @Test
+  public void testEquals() throws Exception {
+    UserGroupInformation uugi = 
+      UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES);
+
+    assertEquals(uugi, uugi);
+    // The subjects should be equal, so this should work
+    assertTrue(uugi.equals(
+                 UserGroupInformation.createUserForTesting
+                   (USER_NAME, GROUP_NAMES)));
+    // ensure that different UGI with the same subject are equal
+    assertEquals(uugi, new UserGroupInformation(uugi.getSubject()));
+  }
+  
+  @Test
+  public void testGettingGroups() throws Exception {
+    UserGroupInformation uugi = 
+      UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES);
+    assertEquals(USER_NAME, uugi.getUserName());
+    assertArrayEquals(new String[]{GROUP1_NAME, GROUP2_NAME, GROUP3_NAME},
+                      uugi.getGroupNames());
+  }
+  
+  @SuppressWarnings("unchecked") // from Mockito mocks
+  @Test
+  public void testUGITokens() {
+    UserGroupInformation ugi = 
+      UserGroupInformation.createUserForTesting("TheDoctor", 
+                                                new String [] { "TheTARDIS"});
+    Token t1 = mock(Token.class);
+    Token t2 = mock(Token.class);
+    
+    ugi.addToken(t1);
+    ugi.addToken(t2);
+    
+    Collection<Token<? extends TokenIdentifier>> z = ugi.getTokens();
+    assertTrue(z.contains(t1));
+    assertTrue(z.contains(t2));
+    assertEquals(2, z.size());
+    
+    try {
+      z.remove(t1);
+      fail("Shouldn't be able to modify token collection from UGI");
+    } catch(UnsupportedOperationException uoe) {
+      // Can't modify tokens
+    }
+  }
+}

+ 7 - 6
src/test/core/org/apache/hadoop/security/TestAccessControlList.java → src/test/core/org/apache/hadoop/security/authorize/TestAccessControlList.java

@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.security;
+package org.apache.hadoop.security.authorize;
 
 import java.util.Iterator;
 import java.util.Set;
 
-import org.apache.hadoop.security.SecurityUtil.AccessControlList;
+import org.apache.hadoop.security.authorize.AccessControlList;
+
 
 import junit.framework.TestCase;
 
@@ -30,16 +31,16 @@ public class TestAccessControlList extends TestCase {
     AccessControlList acl;
     
     acl = new AccessControlList("*");
-    assertTrue(acl.allAllowed());
+    assertTrue(acl.isAllAllowed());
     
     acl = new AccessControlList("  * ");
-    assertTrue(acl.allAllowed());
+    assertTrue(acl.isAllAllowed());
     
     acl = new AccessControlList(" *");
-    assertTrue(acl.allAllowed());
+    assertTrue(acl.isAllAllowed());
     
     acl = new AccessControlList("*  ");
-    assertTrue(acl.allAllowed());
+    assertTrue(acl.isAllAllowed());
   }
   
   public void testAccessControlList() throws Exception {

+ 0 - 140
src/test/core/org/apache/hadoop/security/authorize/TestConfiguredPolicy.java

@@ -1,140 +0,0 @@
-/**
- * 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.authorize;
-
-import java.security.CodeSource;
-import java.security.CodeSigner;
-import java.security.PermissionCollection;
-import java.security.ProtectionDomain;
-import java.net.URL;
-import java.net.NetPermission;
-import java.net.MalformedURLException;
-
-import javax.security.auth.Subject;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UnixUserGroupInformation;
-import org.apache.hadoop.security.SecurityUtil.AccessControlList;
-
-import junit.framework.TestCase;
-
-public class TestConfiguredPolicy extends TestCase {
-  private static final String USER1 = "drwho";
-  private static final String USER2 = "joe";
-  private static final String[] GROUPS1 = new String[]{"tardis"};
-  private static final String[] GROUPS2 = new String[]{"users"};
-  
-  private static final String KEY_1 = "test.policy.1";
-  private static final String KEY_2 = "test.policy.2";
-
-  public static class Protocol1 {
-    int i;
-  }
-  public static class Protocol2 {
-    int j;
-  }
-  
-  private static class TestPolicyProvider extends PolicyProvider {
-    @Override
-    public Service[] getServices() {
-      return new Service[] {
-          new Service(KEY_1, Protocol1.class),
-          new Service(KEY_2, Protocol2.class),
-          };
-    }
-  }
-  
-  public void testConfiguredPolicy() throws Exception {
-    ConfiguredPolicy policy = createConfiguredPolicy();
-    SecurityUtil.setPolicy(policy);
-    
-    Subject user1 = 
-      SecurityUtil.getSubject(new UnixUserGroupInformation(USER1, GROUPS1));
-
-    // Should succeed
-    ServiceAuthorizationManager.authorize(user1, Protocol1.class);
-    
-    // Should fail
-    Subject user2 = 
-      SecurityUtil.getSubject(new UnixUserGroupInformation(USER2, GROUPS2));
-    boolean failed = false;
-    try {
-      ServiceAuthorizationManager.authorize(user2, Protocol2.class);
-    } catch (AuthorizationException ae) {
-      failed = true;
-    }
-    assertTrue(failed);
-  }
-
-  /**
-   * Create a configured policy with some keys
-   * @return a new configured policy
-   */
-  private ConfiguredPolicy createConfiguredPolicy() {
-    Configuration conf = new Configuration();
-    conf.set(KEY_1, AccessControlList.WILDCARD_ACL_VALUE);
-    conf.set(KEY_2, USER1 + " " + GROUPS1[0]);
-
-    return new ConfiguredPolicy(conf, new TestPolicyProvider());
-  }
-
-  /**
-   * Create a test code source against a test URL
-   * @return a new code source
-   * @throws MalformedURLException
-   */
-  private CodeSource createCodeSource() throws MalformedURLException {
-    return new CodeSource(new URL("http://hadoop.apache.org"),
-        (CodeSigner[]) null);
-  }
-
-  /**
-   * Assert that a permission collection can have new permissions added
-   * @param permissions the collection to check
-   */
-  private void assertWritable(PermissionCollection permissions) {
-    assertFalse(permissions.isReadOnly());
-    NetPermission netPermission = new NetPermission("something");
-    permissions.add(netPermission);
-  }
-
-  /**
-   * test that the {@link PermissionCollection} returned by
-   * {@link ConfiguredPolicy#getPermissions(CodeSource)} is writeable
-   * @throws Throwable on any failure
-   */
-  public void testPolicyWritable() throws Throwable {
-    ConfiguredPolicy policy = createConfiguredPolicy();
-    CodeSource source = createCodeSource();
-    PermissionCollection permissions = policy.getPermissions(source);
-    assertWritable(permissions);
-  }
-
-  /**
-   * test that the {@link PermissionCollection} returned by
-   * {@link ConfiguredPolicy#getPermissions(CodeSource)} is writeable
-   * @throws Throwable on any failure
-   */
-  public void testProtectionDomainPolicyWritable() throws Throwable {
-    ConfiguredPolicy policy = createConfiguredPolicy();
-    CodeSource source = createCodeSource();
-    PermissionCollection permissions = policy.getPermissions(new ProtectionDomain(source, null));
-    assertWritable(permissions);
-  }
-}