You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by bo...@apache.org on 2012/11/05 19:26:50 UTC

svn commit: r1405904 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/security/ src/test/java/org/apache/hadoop/ipc/ src/test/java/org/apache/hadoop/security/

Author: bobby
Date: Mon Nov  5 18:26:49 2012
New Revision: 1405904

URL: http://svn.apache.org/viewvc?rev=1405904&view=rev
Log:
HADOOP-9009. Add SecurityUtil methods to get/set authentication method (daryn via bobby)

Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Mon Nov  5 18:26:49 2012
@@ -333,6 +333,9 @@ 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)
+
   OPTIMIZATIONS
 
     HADOOP-8866. SampleQuantiles#query is O(N^2) instead of O(N). (Andrew Wang

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java Mon Nov  5 18:26:49 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/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java Mon Nov  5 18:26:49 2012
@@ -236,15 +236,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(

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java Mon Nov  5 18:26:49 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/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java Mon Nov  5 18:26:49 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/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java Mon Nov  5 18:26:49 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;
@@ -78,7 +79,7 @@ public class TestSaslRPC {
   @BeforeClass
   public static void setup() {
     conf = new Configuration();
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
   }
 
@@ -263,7 +264,6 @@ public class TestSaslRPC {
     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 +345,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);
@@ -536,15 +536,15 @@ public class TestSaslRPC {
                                              final boolean useToken
                                              
       ) throws Exception {
+    Configuration serverConf = new Configuration(conf);
+    SecurityUtil.setAuthenticationMethod(
+        isSecureServer ? KERBEROS : SIMPLE, 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();
-    }
     server.start();
 
     final UserGroupInformation current = UserGroupInformation.getCurrentUser();
@@ -558,8 +558,10 @@ public class TestSaslRPC {
       current.addToken(token);
     }
 
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, isSecureClient ? "kerberos" : "simple");
-    UserGroupInformation.setConfiguration(conf);
+    final Configuration clientConf = new Configuration(conf);
+    SecurityUtil.setAuthenticationMethod(
+        isSecureClient ? KERBEROS : SIMPLE, clientConf);
+    UserGroupInformation.setConfiguration(clientConf);
     try {
       return current.doAs(new PrivilegedExceptionAction<AuthenticationMethod>() {
         @Override
@@ -567,7 +569,7 @@ public class TestSaslRPC {
           TestSaslProtocol proxy = null;
           try {
             proxy = (TestSaslProtocol) RPC.getProxy(TestSaslProtocol.class,
-                TestSaslProtocol.versionID, addr, conf);
+                TestSaslProtocol.versionID, addr, clientConf);
             return proxy.getAuthMethod();
           } finally {
             if (proxy != null) {

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java Mon Nov  5 18:26:49 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/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java Mon Nov  5 18:26:49 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/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java?rev=1405904&r1=1405903&r2=1405904&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java Mon Nov  5 18:26:49 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