You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sz...@apache.org on 2012/11/06 22:11:45 UTC

svn commit: r1406337 - in /hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common: ./ src/main/docs/ src/main/java/ src/main/java/org/apache/hadoop/fs/shell/ src/main/java/org/apache/hadoop/ipc/ src/main/java/org/apache/hadoop/security/ s...

Author: szetszwo
Date: Tue Nov  6 21:11:42 2012
New Revision: 1406337

URL: http://svn.apache.org/viewvc?rev=1406337&view=rev
Log:
Merge r1406007 through r1406326 from trunk.

Added:
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java
      - copied unchanged from r1406326, hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithExternalKdc.java
Modified:
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt   (contents, props changed)
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/docs/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/core/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
    hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt Tue Nov  6 21:11:42 2012
@@ -129,6 +129,9 @@ Trunk (Unreleased)
     HADOOP-8776. Provide an option in test-patch that can enable/disable
     compiling native code. (Chris Nauroth via suresh)
 
+    HADOOP-9004. Allow security unit tests to use external KDC. (Stephen Chu
+    via suresh)
+
   BUG FIXES
 
     HADOOP-8177. MBeans shouldn't try to register when it fails to create MBeanName.
@@ -333,6 +336,12 @@ Release 2.0.3-alpha - Unreleased 
     HADOOP-8985. Add namespace declarations in .proto files for languages 
     other than java. (Binglin Chan via suresh)
 
+    HADOOP-9009. Add SecurityUtil methods to get/set authentication method
+    (daryn via bobby)
+
+    HADOOP-9010. Map UGI authenticationMethod to RPC authMethod (daryn via
+    bobby)
+
   OPTIMIZATIONS
 
     HADOOP-8866. SampleQuantiles#query is O(N^2) instead of O(N). (Andrew Wang
@@ -389,6 +398,8 @@ Release 2.0.3-alpha - Unreleased 
     HADOOP-8713. TestRPCCompatibility fails intermittently with JDK7
     (Trevor Robinson via tgraves)
 
+    HADOOP-9012. IPC Client sends wrong connection context (daryn via bobby)
+
 Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES

Propchange: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt:r1405252-1406326

Propchange: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/docs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/docs:r1405252-1406326

Propchange: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java:r1405252-1406326

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java Tue Nov  6 21:11:42 2012
@@ -37,16 +37,21 @@ class Test extends FsCommand {  
   }
 
   public static final String NAME = "test";
-  public static final String USAGE = "-[ezd] <path>";
+  public static final String USAGE = "-[defsz] <path>";
   public static final String DESCRIPTION =
-    "If file exists, has zero length, is a directory\n" +
-    "then return 0, else return 1.";
+    "Answer various questions about <path>, with result via exit status.\n" +
+    "  -d  return 0 if <path> is a directory.\n" +
+    "  -e  return 0 if <path> exists.\n" +
+    "  -f  return 0 if <path> is a file.\n" +
+    "  -s  return 0 if file <path> is greater than zero bytes in size.\n" +
+    "  -z  return 0 if file <path> is zero bytes in size.\n" +
+    "else, return 1.";
 
   private char flag;
   
   @Override
   protected void processOptions(LinkedList<String> args) {
-    CommandFormat cf = new CommandFormat(1, 1, "e", "d", "z");
+    CommandFormat cf = new CommandFormat(1, 1, "e", "d", "f", "s", "z");
     cf.parse(args);
     
     String[] opts = cf.getOpts().toArray(new String[0]);
@@ -71,6 +76,12 @@ class Test extends FsCommand {  
       case 'd':
         test = item.stat.isDirectory();
         break;
+      case 'f':
+        test = item.stat.isFile();
+        break;
+      case 's':
+        test = (item.stat.getLen() > 0);
+        break;
       case 'z':
         test = (item.stat.getLen() == 0);
         break;

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java Tue Nov  6 21:11:42 2012
@@ -69,6 +69,7 @@ import org.apache.hadoop.security.SaslRp
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenInfo;
@@ -222,7 +223,6 @@ public class Client {
   private class Connection extends Thread {
     private InetSocketAddress server;             // server ip:port
     private String serverPrincipal;  // server's krb5 principal name
-    private IpcConnectionContextProto connectionContext;   // connection context
     private final ConnectionId remoteId;                // connection id
     private AuthMethod authMethod; // authentication method
     private Token<? extends TokenIdentifier> token;
@@ -295,16 +295,14 @@ public class Client {
       }
       
       if (token != null) {
-        authMethod = AuthMethod.DIGEST;
+        authMethod = AuthenticationMethod.TOKEN.getAuthMethod();
       } else if (UserGroupInformation.isSecurityEnabled()) {
+        // eventually just use the ticket's authMethod
         authMethod = AuthMethod.KERBEROS;
       } else {
         authMethod = AuthMethod.SIMPLE;
       }
       
-      connectionContext = ProtoUtil.makeIpcConnectionContext(
-          RPC.getProtocolName(protocol), ticket, authMethod);
-      
       if (LOG.isDebugEnabled())
         LOG.debug("Use " + authMethod + " authentication for protocol "
             + protocol.getSimpleName());
@@ -605,11 +603,6 @@ public class Client {
             } else {
               // fall back to simple auth because server told us so.
               authMethod = AuthMethod.SIMPLE;
-              // remake the connectionContext             
-              connectionContext = ProtoUtil.makeIpcConnectionContext(
-                  connectionContext.getProtocol(), 
-                  ProtoUtil.getUgi(connectionContext.getUserInfo()),
-                  authMethod);
             }
           }
         
@@ -620,7 +613,7 @@ public class Client {
             this.in = new DataInputStream(new BufferedInputStream(inStream));
           }
           this.out = new DataOutputStream(new BufferedOutputStream(outStream));
-          writeConnectionContext();
+          writeConnectionContext(remoteId, authMethod);
 
           // update last activity time
           touch();
@@ -742,10 +735,15 @@ public class Client {
     /* Write the connection context header for each connection
      * Out is not synchronized because only the first thread does this.
      */
-    private void writeConnectionContext() throws IOException {
+    private void writeConnectionContext(ConnectionId remoteId,
+                                        AuthMethod authMethod)
+                                            throws IOException {
       // Write out the ConnectionHeader
       DataOutputBuffer buf = new DataOutputBuffer();
-      connectionContext.writeTo(buf);
+      ProtoUtil.makeIpcConnectionContext(
+          RPC.getProtocolName(remoteId.getProtocol()),
+          remoteId.getTicket(),
+          authMethod).writeTo(buf);
       
       // Write out the payload length
       int bufLen = buf.getLength();

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java Tue Nov  6 21:11:42 2012
@@ -1526,11 +1526,11 @@ public abstract class Server {
       if (!useSasl) {
         user = protocolUser;
         if (user != null) {
-          user.setAuthenticationMethod(AuthMethod.SIMPLE.authenticationMethod);
+          user.setAuthenticationMethod(AuthMethod.SIMPLE);
         }
       } else {
         // user is authenticated
-        user.setAuthenticationMethod(authMethod.authenticationMethod);
+        user.setAuthenticationMethod(authMethod);
         //Now we check if this is a proxy user case. If the protocol user is
         //different from the 'user', it is a proxy user scenario. However, 
         //this is not allowed if user authenticated with DIGEST.

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java Tue Nov  6 21:11:42 2012
@@ -42,7 +42,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -137,20 +136,17 @@ public class SaslRpcServer {
   /** Authentication method */
   @InterfaceStability.Evolving
   public static enum AuthMethod {
-    SIMPLE((byte) 80, "", AuthenticationMethod.SIMPLE),
-    KERBEROS((byte) 81, "GSSAPI", AuthenticationMethod.KERBEROS),
-    DIGEST((byte) 82, "DIGEST-MD5", AuthenticationMethod.TOKEN);
+    SIMPLE((byte) 80, ""),
+    KERBEROS((byte) 81, "GSSAPI"),
+    DIGEST((byte) 82, "DIGEST-MD5");
 
     /** The code for this method. */
     public final byte code;
     public final String mechanismName;
-    public final AuthenticationMethod authenticationMethod;
 
-    private AuthMethod(byte code, String mechanismName, 
-                       AuthenticationMethod authMethod) {
+    private AuthMethod(byte code, String mechanismName) { 
       this.code = code;
       this.mechanismName = mechanismName;
-      this.authenticationMethod = authMethod;
     }
 
     private static final int FIRST_CODE = values()[0].code;

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java Tue Nov  6 21:11:42 2012
@@ -16,6 +16,8 @@
  */
 package org.apache.hadoop.security;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -44,6 +46,7 @@ import org.apache.hadoop.fs.CommonConfig
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.ssl.SSLFactory;
@@ -665,4 +668,22 @@ public class SecurityUtil {
     }
   }
 
+  public static AuthenticationMethod getAuthenticationMethod(Configuration conf) {
+    String value = conf.get(HADOOP_SECURITY_AUTHENTICATION, "simple");
+    try {
+      return Enum.valueOf(AuthenticationMethod.class, value.toUpperCase());
+    } catch (IllegalArgumentException iae) {
+      throw new IllegalArgumentException("Invalid attribute value for " +
+          HADOOP_SECURITY_AUTHENTICATION + " of " + value);
+    }
+  }
+
+  public static void setAuthenticationMethod(
+      AuthenticationMethod authenticationMethod, Configuration conf) {
+    if (authenticationMethod == null) {
+      authenticationMethod = AuthenticationMethod.SIMPLE;
+    }
+    conf.set(HADOOP_SECURITY_AUTHENTICATION,
+             authenticationMethod.toString().toLowerCase());
+  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java Tue Nov  6 21:11:42 2012
@@ -59,6 +59,7 @@ import org.apache.hadoop.metrics2.annota
 import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.MutableRate;
+import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.apache.hadoop.security.token.Token;
@@ -236,15 +237,15 @@ public class UserGroupInformation {
    * @param conf the configuration to use
    */
   private static synchronized void initUGI(Configuration conf) {
-    String value = conf.get(HADOOP_SECURITY_AUTHENTICATION);
-    if (value == null || "simple".equals(value)) {
+    AuthenticationMethod auth = SecurityUtil.getAuthenticationMethod(conf);
+    if (auth == AuthenticationMethod.SIMPLE) {
       useKerberos = false;
-    } else if ("kerberos".equals(value)) {
+    } else if (auth == AuthenticationMethod.KERBEROS) {
       useKerberos = true;
     } else {
       throw new IllegalArgumentException("Invalid attribute value for " +
                                          HADOOP_SECURITY_AUTHENTICATION + 
-                                         " of " + value);
+                                         " of " + auth);
     }
     try {
         kerberosMinSecondsBeforeRelogin = 1000L * conf.getLong(
@@ -1019,13 +1020,34 @@ public class UserGroupInformation {
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
   public static enum AuthenticationMethod {
-    SIMPLE,
-    KERBEROS,
-    TOKEN,
-    CERTIFICATE,
-    KERBEROS_SSL,
-    PROXY;
-  }
+    // currently we support only one auth per method, but eventually a 
+    // subtype is needed to differentiate, ex. if digest is token or ldap
+    SIMPLE(AuthMethod.SIMPLE),
+    KERBEROS(AuthMethod.KERBEROS),
+    TOKEN(AuthMethod.DIGEST),
+    CERTIFICATE(null),
+    KERBEROS_SSL(null),
+    PROXY(null);
+    
+    private final AuthMethod authMethod;
+    private AuthenticationMethod(AuthMethod authMethod) {
+      this.authMethod = authMethod;
+    }
+    
+    public AuthMethod getAuthMethod() {
+      return authMethod;
+    }
+    
+    public static AuthenticationMethod valueOf(AuthMethod authMethod) {
+      for (AuthenticationMethod value : values()) {
+        if (value.getAuthMethod() == authMethod) {
+          return value;
+        }
+      }
+      throw new IllegalArgumentException(
+          "no authentication method for " + authMethod);
+    }
+  };
 
   /**
    * Create a proxy user using username of the effective user and the ugi of the
@@ -1291,6 +1313,15 @@ public class UserGroupInformation {
   }
 
   /**
+   * Sets the authentication method in the subject
+   * 
+   * @param authMethod
+   */
+  public void setAuthenticationMethod(AuthMethod authMethod) {
+    user.setAuthenticationMethod(AuthenticationMethod.valueOf(authMethod));
+  }
+
+  /**
    * Get the authentication method from the subject
    * 
    * @return AuthenticationMethod in the subject, null if not present.

Propchange: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/core/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/core:r1405252-1406326

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java Tue Nov  6 21:11:42 2012
@@ -30,7 +30,6 @@ import junit.framework.Assert;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.KerberosInfo;
@@ -380,9 +379,7 @@ public class MiniRPCBenchmark {
       elapsedTime = mb.runMiniBenchmarkWithDelegationToken(
                               conf, count, KEYTAB_FILE_KEY, USER_NAME_KEY);
     } else {
-      String auth = 
-        conf.get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, 
-                        "simple");
+      String auth = SecurityUtil.getAuthenticationMethod(conf).toString();
       System.out.println(
           "Running MiniRPCBenchmark with " + auth + " authentication.");
       elapsedTime = mb.runMiniBenchmark(

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java Tue Nov  6 21:11:42 2012
@@ -55,13 +55,16 @@ import org.apache.hadoop.ipc.Client.Conn
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.Service;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.test.MockitoUtil;
+import org.junit.Before;
 import org.junit.Test;
 
 import com.google.protobuf.DescriptorProtos;
@@ -75,11 +78,14 @@ public class TestRPC {
   public static final Log LOG =
     LogFactory.getLog(TestRPC.class);
   
-  private static Configuration conf = new Configuration();
+  private static Configuration conf;
   
-  static {
+  @Before
+  public void setupConf() {
+    conf = new Configuration();
     conf.setClass("rpc.engine." + StoppedProtocol.class.getName(),
         StoppedRpcEngine.class, RpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
   }
 
   int datasize = 1024*100;
@@ -676,11 +682,17 @@ public class TestRPC {
   
   @Test
   public void testErrorMsgForInsecureClient() throws Exception {
-    final Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
+    Configuration serverConf = new Configuration(conf);
+    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS,
+                                         serverConf);
+    UserGroupInformation.setConfiguration(serverConf);
+    
+    final Server server = new RPC.Builder(serverConf).setProtocol(TestProtocol.class)
         .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
         .setNumHandlers(5).setVerbose(true).build();
-    server.enableSecurity();
     server.start();
+
+    UserGroupInformation.setConfiguration(conf);
     boolean succeeded = false;
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
     TestProtocol proxy = null;
@@ -702,17 +714,18 @@ public class TestRPC {
 
     conf.setInt(CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, 2);
 
-    final Server multiServer = new RPC.Builder(conf)
+    UserGroupInformation.setConfiguration(serverConf);
+    final Server multiServer = new RPC.Builder(serverConf)
         .setProtocol(TestProtocol.class).setInstance(new TestImpl())
         .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true)
         .build();
-    multiServer.enableSecurity();
     multiServer.start();
     succeeded = false;
     final InetSocketAddress mulitServerAddr =
                       NetUtils.getConnectAddress(multiServer);
     proxy = null;
     try {
+      UserGroupInformation.setConfiguration(conf);
       proxy = (TestProtocol) RPC.getProxy(TestProtocol.class,
           TestProtocol.versionID, mulitServerAddr, conf);
       proxy.echo("");

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java Tue Nov  6 21:11:42 2012
@@ -18,8 +18,9 @@
 
 package org.apache.hadoop.ipc;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.*;
 import static org.junit.Assert.*;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -28,6 +29,7 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.Set;
+import java.util.regex.Pattern;
 
 import javax.security.sasl.Sasl;
 
@@ -41,7 +43,6 @@ import org.apache.hadoop.fs.CommonConfig
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.SaslInputStream;
 import org.apache.hadoop.security.SaslRpcClient;
@@ -58,7 +59,7 @@ import org.apache.hadoop.security.token.
 import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.log4j.Level;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 /** Unit tests for using Sasl over RPC. */
@@ -75,10 +76,11 @@ public class TestSaslRPC {
   static final String SERVER_PRINCIPAL_2 = "p2/foo@BAR";
   
   private static Configuration conf;
-  @BeforeClass
-  public static void setup() {
+
+  @Before
+  public void setup() {
     conf = new Configuration();
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
   }
 
@@ -186,6 +188,7 @@ public class TestSaslRPC {
   @TokenInfo(TestTokenSelector.class)
   public interface TestSaslProtocol extends TestRPC.TestProtocol {
     public AuthenticationMethod getAuthMethod() throws IOException;
+    public String getAuthUser() throws IOException;
   }
   
   public static class TestSaslImpl extends TestRPC.TestImpl implements
@@ -194,6 +197,10 @@ public class TestSaslRPC {
     public AuthenticationMethod getAuthMethod() throws IOException {
       return UserGroupInformation.getCurrentUser().getAuthenticationMethod();
     }
+    @Override
+    public String getAuthUser() throws IOException {
+      return UserGroupInformation.getCurrentUser().getUserName();
+    }
   }
 
   public static class CustomSecurityInfo extends SecurityInfo {
@@ -260,10 +267,10 @@ public class TestSaslRPC {
 
   @Test
   public void testSecureToInsecureRpc() throws Exception {
+    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.SIMPLE, conf);
     Server server = new RPC.Builder(conf).setProtocol(TestSaslProtocol.class)
         .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0)
         .setNumHandlers(5).setVerbose(true).build();
-    server.disableSecurity();
     TestTokenSecretManager sm = new TestTokenSecretManager();
     doDigestRpc(server, sm);
   }
@@ -345,7 +352,7 @@ public class TestSaslRPC {
           new InetSocketAddress(0), TestSaslProtocol.class, null, 0, newConf);
       assertEquals(SERVER_PRINCIPAL_1, remoteId.getServerPrincipal());
       // this following test needs security to be off
-      newConf.set(HADOOP_SECURITY_AUTHENTICATION, "simple");
+      SecurityUtil.setAuthenticationMethod(SIMPLE, newConf);
       UserGroupInformation.setConfiguration(newConf);
       remoteId = ConnectionId.getConnectionId(new InetSocketAddress(0),
           TestSaslProtocol.class, null, 0, newConf);
@@ -448,127 +455,135 @@ public class TestSaslRPC {
     System.out.println("Test is successful.");
   }
 
-  // insecure -> insecure
-  @Test
-  public void testInsecureClientInsecureServer() throws Exception {
-    assertEquals(AuthenticationMethod.SIMPLE,
-                 getAuthMethod(false, false, false));
+  private static Pattern BadToken =
+      Pattern.compile(".*DIGEST-MD5: digest response format violation.*");
+  private static Pattern KrbFailed =
+      Pattern.compile(".*Failed on local exception:.* " +
+                      "Failed to specify server's Kerberos principal name.*");
+  private static Pattern Denied = 
+      Pattern.compile(".*Authorization .* is enabled .*");
+  
+  /*
+   *  simple server
+   */
+  @Test
+  public void testSimpleServer() throws Exception {
+    assertAuthEquals(SIMPLE,    getAuthMethod(SIMPLE,   SIMPLE));
+    // SASL methods are reverted to SIMPLE, but test setup fails
+    assertAuthEquals(KrbFailed, getAuthMethod(KERBEROS, SIMPLE));
   }
 
   @Test
-  public void testInsecureClientInsecureServerWithToken() throws Exception {
-    assertEquals(AuthenticationMethod.TOKEN,
-                 getAuthMethod(false, false, true));
+  public void testSimpleServerWithTokens() throws Exception {
+    // Tokens are ignored because client is reverted to simple
+    assertAuthEquals(SIMPLE, getAuthMethod(SIMPLE,   SIMPLE, true));
+    assertAuthEquals(SIMPLE, getAuthMethod(KERBEROS, SIMPLE, true));
   }
-
-  // insecure -> secure
+    
   @Test
-  public void testInsecureClientSecureServer() throws Exception {
-    RemoteException e = null;
-    try {
-      getAuthMethod(false, true, false);
-    } catch (RemoteException re) {
-      e = re;
-    }
-    assertNotNull(e);
-    assertEquals(AccessControlException.class.getName(), e.getClassName());
+  public void testSimpleServerWithInvalidTokens() throws Exception {
+    // Tokens are ignored because client is reverted to simple
+    assertAuthEquals(SIMPLE, getAuthMethod(SIMPLE,   SIMPLE, false));
+    assertAuthEquals(SIMPLE, getAuthMethod(KERBEROS, SIMPLE, false));
   }
-
+  
+  /*
+   * kerberos server
+   */
   @Test
-  public void testInsecureClientSecureServerWithToken() throws Exception {
-    assertEquals(AuthenticationMethod.TOKEN,
-                 getAuthMethod(false, true, true));
+  public void testKerberosServer() throws Exception {
+    assertAuthEquals(Denied,    getAuthMethod(SIMPLE,   KERBEROS));
+    assertAuthEquals(KrbFailed, getAuthMethod(KERBEROS, KERBEROS));    
   }
 
-  // secure -> secure
   @Test
-  public void testSecureClientSecureServer() throws Exception {
-    /* Should be this when multiple secure auths are supported and we can
-     * dummy one out:
-     *     assertEquals(AuthenticationMethod.SECURE_AUTH_METHOD,
-     *                  getAuthMethod(true, true, false));
-     */
-    try {
-      getAuthMethod(true, true, false);
-    } catch (IOException ioe) {
-      // can't actually test kerberos w/o kerberos...
-      String expectedError = "Failed to specify server's Kerberos principal";
-      String actualError = ioe.getMessage();
-      assertTrue("["+actualError+"] doesn't start with ["+expectedError+"]",
-          actualError.contains(expectedError));
-    }
+  public void testKerberosServerWithTokens() throws Exception {
+    // can use tokens regardless of auth
+    assertAuthEquals(TOKEN, getAuthMethod(SIMPLE,   KERBEROS, true));
+    assertAuthEquals(TOKEN, getAuthMethod(KERBEROS, KERBEROS, true));
   }
 
   @Test
-  public void testSecureClientSecureServerWithToken() throws Exception {
-    assertEquals(AuthenticationMethod.TOKEN,
-                 getAuthMethod(true, true, true));
+  public void testKerberosServerWithInvalidTokens() throws Exception {
+    assertAuthEquals(BadToken, getAuthMethod(SIMPLE,   KERBEROS, false));
+    assertAuthEquals(BadToken, getAuthMethod(KERBEROS, KERBEROS, false));
   }
 
-  // secure -> insecure
-  @Test
-  public void testSecureClientInsecureServerWithToken() throws Exception {
-    assertEquals(AuthenticationMethod.TOKEN,
-                 getAuthMethod(true, false, true));
-  }
 
-  @Test
-  public void testSecureClientInsecureServer() throws Exception {
-    /* Should be this when multiple secure auths are supported and we can
-     * dummy one out:
-     *     assertEquals(AuthenticationMethod.SIMPLE
-     *                  getAuthMethod(true, false, false));
-     */
+  // test helpers
+
+  private String getAuthMethod(
+      final AuthenticationMethod clientAuth,
+      final AuthenticationMethod serverAuth) throws Exception {
     try {
-      getAuthMethod(true, false, false);
-    } catch (IOException ioe) {
-      // can't actually test kerberos w/o kerberos...
-      String expectedError = "Failed to specify server's Kerberos principal";
-      String actualError = ioe.getMessage();
-      assertTrue("["+actualError+"] doesn't start with ["+expectedError+"]",
-          actualError.contains(expectedError));
+      return internalGetAuthMethod(clientAuth, serverAuth, false, false);
+    } catch (Exception e) {
+      return e.toString();
     }
   }
 
-
-  private AuthenticationMethod getAuthMethod(final boolean isSecureClient,
-                                             final boolean isSecureServer,
-                                             final boolean useToken
-                                             
-      ) throws Exception {
+  private String getAuthMethod(
+      final AuthenticationMethod clientAuth,
+      final AuthenticationMethod serverAuth,
+      final boolean useValidToken) throws Exception {
+    try {
+      return internalGetAuthMethod(clientAuth, serverAuth, true, useValidToken);
+    } catch (Exception e) {
+      return e.toString();
+    }
+  }
+  
+  private String internalGetAuthMethod(
+      final AuthenticationMethod clientAuth,
+      final AuthenticationMethod serverAuth,
+      final boolean useToken,
+      final boolean useValidToken) throws Exception {
+    
+    Configuration serverConf = new Configuration(conf);
+    SecurityUtil.setAuthenticationMethod(serverAuth, serverConf);
+    UserGroupInformation.setConfiguration(serverConf);
+    
     TestTokenSecretManager sm = new TestTokenSecretManager();
-    Server server = new RPC.Builder(conf).setProtocol(TestSaslProtocol.class)
+    Server server = new RPC.Builder(serverConf).setProtocol(TestSaslProtocol.class)
         .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(5).setVerbose(true).setSecretManager(sm).build();      
-    if (isSecureServer) {
-      server.enableSecurity();
-    } else {
-      server.disableSecurity();
-    }
+        .setNumHandlers(5).setVerbose(true)
+        .setSecretManager((serverAuth != SIMPLE) ? sm : null)
+        .build();      
     server.start();
 
-    final UserGroupInformation current = UserGroupInformation.getCurrentUser();
+    final UserGroupInformation clientUgi =
+        UserGroupInformation.createRemoteUser(
+            UserGroupInformation.getCurrentUser().getUserName()+"-CLIENT");
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
     if (useToken) {
       TestTokenIdentifier tokenId = new TestTokenIdentifier(
-          new Text(current.getUserName()));
-      Token<TestTokenIdentifier> token =
-          new Token<TestTokenIdentifier>(tokenId, sm);
+          new Text(clientUgi.getUserName()));
+      Token<TestTokenIdentifier> token = useValidToken
+          ? new Token<TestTokenIdentifier>(tokenId, sm)
+          : new Token<TestTokenIdentifier>(
+              tokenId.getBytes(), "bad-password!".getBytes(),
+              tokenId.getKind(), null);
+      
       SecurityUtil.setTokenService(token, addr);
-      current.addToken(token);
+      clientUgi.addToken(token);
     }
 
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, isSecureClient ? "kerberos" : "simple");
-    UserGroupInformation.setConfiguration(conf);
+    final Configuration clientConf = new Configuration(conf);
+    SecurityUtil.setAuthenticationMethod(clientAuth, clientConf);
+    UserGroupInformation.setConfiguration(clientConf);
+    
     try {
-      return current.doAs(new PrivilegedExceptionAction<AuthenticationMethod>() {
+      return clientUgi.doAs(new PrivilegedExceptionAction<String>() {
         @Override
-        public AuthenticationMethod run() throws IOException {
+        public String run() throws IOException {
           TestSaslProtocol proxy = null;
           try {
             proxy = (TestSaslProtocol) RPC.getProxy(TestSaslProtocol.class,
-                TestSaslProtocol.versionID, addr, conf);
-            return proxy.getAuthMethod();
+                TestSaslProtocol.versionID, addr, clientConf);
+            
+            // make sure the other side thinks we are who we said we are!!!
+            assertEquals(clientUgi.getUserName(), proxy.getAuthUser());
+            return proxy.getAuthMethod().toString();
           } finally {
             if (proxy != null) {
               RPC.stopProxy(proxy);
@@ -580,7 +595,22 @@ public class TestSaslRPC {
       server.stop();
     }
   }
+
+  private static void assertAuthEquals(AuthenticationMethod expect,
+      String actual) {
+    assertEquals(expect.toString(), actual);
+  }
   
+  private static void assertAuthEquals(Pattern expect,
+      String actual) {
+    // this allows us to see the regexp and the value it didn't match
+    if (!expect.matcher(actual).matches()) {
+      assertEquals(expect, actual); // it failed
+    } else {
+      assertTrue(true); // it matched
+    }
+  }
+
   public static void main(String[] args) throws Exception {
     System.out.println("Testing Kerberos authentication over RPC");
     if (args.length != 2) {
@@ -593,5 +623,4 @@ public class TestSaslRPC {
     String keytab = args[1];
     testKerberosRpc(principal, keytab);
   }
-
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java Tue Nov  6 21:11:42 2012
@@ -27,4 +27,19 @@ public class SecurityUtilTestHelper {
   public static void setTokenServiceUseIp(boolean flag) {
     SecurityUtil.setTokenServiceUseIp(flag);
   }
+
+  /**
+   * Return true if externalKdc=true and the location of the krb5.conf
+   * file has been specified, and false otherwise.
+   */
+  public static boolean isExternalKdcRunning() {
+    String externalKdc = System.getProperty("externalKdc");
+    String krb5Conf = System.getProperty("java.security.krb5.conf");
+    if(externalKdc == null || !externalKdc.equals("true") ||
+       krb5Conf == null) {
+      return false;
+    }
+    return true;
+  }
+
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java Tue Nov  6 21:11:42 2012
@@ -28,13 +28,13 @@ import java.util.Enumeration;
 import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
@@ -416,8 +416,7 @@ public class TestDoAsEffectiveUser {
   public void testProxyWithToken() throws Exception {
     final Configuration conf = new Configuration(masterConf);
     TestTokenSecretManager sm = new TestTokenSecretManager();
-    conf
-        .set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
     final Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
         .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
@@ -471,8 +470,7 @@ public class TestDoAsEffectiveUser {
   public void testTokenBySuperUser() throws Exception {
     TestTokenSecretManager sm = new TestTokenSecretManager();
     final Configuration newConf = new Configuration(masterConf);
-    newConf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
-        "kerberos");
+    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, newConf);
     UserGroupInformation.setConfiguration(newConf);
     final Server server = new RPC.Builder(newConf)
         .setProtocol(TestProtocol.class).setInstance(new TestImpl())

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java Tue Nov  6 21:11:42 2012
@@ -16,6 +16,8 @@
  */
 package org.apache.hadoop.security;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.*;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
@@ -29,10 +31,19 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 public class TestSecurityUtil {
+  @BeforeClass
+  public static void unsetKerberosRealm() {
+    // prevent failures if kinit-ed or on os x with no realm
+    System.setProperty("java.security.krb5.kdc", "");
+    System.setProperty("java.security.krb5.realm", "NONE");    
+  }
+
   @Test
   public void isOriginalTGTReturnsCorrectValues() {
     assertTrue(SecurityUtil.isTGSPrincipal
@@ -111,9 +122,7 @@ public class TestSecurityUtil {
   @Test
   public void testStartsWithIncorrectSettings() throws IOException {
     Configuration conf = new Configuration();
-    conf.set(
-        org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
-        "kerberos");
+    SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
     String keyTabKey="key";
     conf.set(keyTabKey, "");
     UserGroupInformation.setConfiguration(conf);
@@ -256,7 +265,7 @@ public class TestSecurityUtil {
     SecurityUtil.setTokenServiceUseIp(useIp);
     String serviceHost = useIp ? ip : host.toLowerCase();
     
-    Token token = new Token();
+    Token<?> token = new Token<TokenIdentifier>();
     Text service = new Text(serviceHost+":"+port);
     
     assertEquals(service, SecurityUtil.buildTokenService(addr));
@@ -345,4 +354,43 @@ public class TestSecurityUtil {
     NetUtils.addStaticResolution(staticHost, "255.255.255.255");
     verifyServiceAddr(staticHost, "255.255.255.255");
   }
+  
+  @Test
+  public void testGetAuthenticationMethod() {
+    Configuration conf = new Configuration();
+    // default is simple
+    conf.unset(HADOOP_SECURITY_AUTHENTICATION);
+    assertEquals(SIMPLE, SecurityUtil.getAuthenticationMethod(conf));
+    // simple
+    conf.set(HADOOP_SECURITY_AUTHENTICATION, "simple");
+    assertEquals(SIMPLE, SecurityUtil.getAuthenticationMethod(conf));
+    // kerberos
+    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    assertEquals(KERBEROS, SecurityUtil.getAuthenticationMethod(conf));
+    // bad value
+    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kaboom");
+    String error = null;
+    try {
+      SecurityUtil.getAuthenticationMethod(conf);
+    } catch (Exception e) {
+      error = e.toString();
+    }
+    assertEquals("java.lang.IllegalArgumentException: " +
+                 "Invalid attribute value for " +
+                 HADOOP_SECURITY_AUTHENTICATION + " of kaboom", error);
+  }
+  
+  @Test
+  public void testSetAuthenticationMethod() {
+    Configuration conf = new Configuration();
+    // default
+    SecurityUtil.setAuthenticationMethod(null, conf);
+    assertEquals("simple", conf.get(HADOOP_SECURITY_AUTHENTICATION));
+    // simple
+    SecurityUtil.setAuthenticationMethod(SIMPLE, conf);
+    assertEquals("simple", conf.get(HADOOP_SECURITY_AUTHENTICATION));
+    // kerberos
+    SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
+    assertEquals("kerberos", conf.get(HADOOP_SECURITY_AUTHENTICATION));
+  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java Tue Nov  6 21:11:42 2012
@@ -21,7 +21,6 @@ import java.io.IOException;
 import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.junit.Assume;
 import org.junit.Before;
@@ -49,8 +48,7 @@ public class TestUGIWithSecurityOn {
     String user1keyTabFilepath = System.getProperty("kdc.resource.dir") 
         + "/keytabs/user1.keytab";
     Configuration conf = new Configuration();
-    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, 
-        "kerberos");
+    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
     
     UserGroupInformation ugiNn = UserGroupInformation

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java Tue Nov  6 21:11:42 2012
@@ -305,7 +305,6 @@ public class TestUserGroupInformation {
     assertSame(secret, ugi.getCredentials().getSecretKey(secretKey));
   }
 
-  @SuppressWarnings("unchecked") // from Mockito mocks
   @Test
   public <T extends TokenIdentifier> void testGetCredsNotSame()
       throws Exception {
@@ -430,6 +429,18 @@ public class TestUserGroupInformation {
   }
 
   @Test
+  public void testTestAuthMethod() throws Exception {
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    // verify the reverse mappings works
+    for (AuthenticationMethod am : AuthenticationMethod.values()) {
+      if (am.getAuthMethod() != null) {
+        ugi.setAuthenticationMethod(am.getAuthMethod());
+        assertEquals(am, ugi.getAuthenticationMethod());
+      }
+    }
+  }
+  
+  @Test
   public void testUGIAuthMethod() throws Exception {
     final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     final AuthenticationMethod am = AuthenticationMethod.KERBEROS;

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml?rev=1406337&r1=1406336&r2=1406337&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml Tue Nov  6 21:11:42 2012
@@ -591,11 +591,11 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-test -\[ezd\] &lt;path&gt;:\s+If file exists, has zero length, is a directory( )*</expected-output>
+          <expected-output>^-test -\[defsz\] &lt;path&gt;:\sAnswer various questions about &lt;path&gt;, with result via exit status.</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*then return 0, else return 1.( )*</expected-output>
+          <expected-output>^( |\t)*else, return 1.( )*</expected-output>
         </comparator>
       </comparators>
     </test>