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 su...@apache.org on 2012/11/08 20:09:56 UTC

svn commit: r1407217 [2/2] - in /hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common: ./ src/ src/main/docs/ src/main/java/ src/main/java/org/apache/hadoop/conf/ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoo...

Modified: hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java Thu Nov  8 19:09:46 2012
@@ -17,12 +17,15 @@
  */
 package org.apache.hadoop.fs.viewfs;
 
+import java.net.URI;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContextTestHelper;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.mortbay.log.Log;
 
 
 /**
@@ -31,13 +34,20 @@ import org.apache.hadoop.fs.viewfs.Confi
  * 
  * If tests launched via ant (build.xml) the test root is absolute path
  * If tests launched via eclipse, the test root is 
- * is a test dir below the working directory. (see FileContextTestHelper).
- * Since viewFs has no built-in wd, its wd is /user/<username>.
+ * is a test dir below the working directory. (see FileContextTestHelper)
+ * 
+ * We set a viewfs with 3 mount points: 
+ * 1) /<firstComponent>" of testdir  pointing to same in  target fs
+ * 2)   /<firstComponent>" of home  pointing to same in  target fs 
+ * 3)  /<firstComponent>" of wd  pointing to same in  target fs
+ * (note in many cases the link may be the same - viewfs handles this)
  * 
- * We set up fc to be the viewFs with mount point for 
- * /<firstComponent>" pointing to the local file system's testdir 
+ * We also set the view file system's wd to point to the wd.  
  */
+
 public class ViewFsTestSetup {
+  
+  static public String ViewFSTestDir = "/testDir";
 
 
    /* 
@@ -47,30 +57,31 @@ public class ViewFsTestSetup {
     /**
      * create the test root on local_fs - the  mount table will point here
      */
-    FileContext fclocal = FileContext.getLocalFSFileContext();
-    Path targetOfTests = FileContextTestHelper.getTestRootPath(fclocal);
+    FileContext fsTarget = FileContext.getLocalFSFileContext();
+    Path targetOfTests = FileContextTestHelper.getTestRootPath(fsTarget);
     // In case previous test was killed before cleanup
-    fclocal.delete(targetOfTests, true);
+    fsTarget.delete(targetOfTests, true);
     
-    fclocal.mkdir(targetOfTests, FileContext.DEFAULT_PERM, true);
-  
-    String srcTestFirstDir;
-    if (FileContextTestHelper.TEST_ROOT_DIR.startsWith("/")) {
-      int indexOf2ndSlash = FileContextTestHelper.TEST_ROOT_DIR.indexOf('/', 1);
-      srcTestFirstDir = FileContextTestHelper.TEST_ROOT_DIR.substring(0, indexOf2ndSlash);
-    } else {
-      srcTestFirstDir = "/user"; 
-  
-    }
-    //System.out.println("srcTestFirstDir=" + srcTestFirstDir);
-  
-    // Set up the defaultMT in the config with mount point links
-    // The test dir is root is below  /user/<userid>
+    fsTarget.mkdir(targetOfTests, FileContext.DEFAULT_PERM, true);
     Configuration conf = new Configuration();
-    ConfigUtil.addLink(conf, srcTestFirstDir,
-        targetOfTests.toUri());
+    
+    // Set up viewfs link for test dir as described above
+    String testDir = FileContextTestHelper.getTestRootPath(fsTarget).toUri()
+        .getPath();
+    linkUpFirstComponents(conf, testDir, fsTarget, "test dir");
+    
+    
+    // Set up viewfs link for home dir as described above
+    setUpHomeDir(conf, fsTarget);
+      
+    // the test path may be relative to working dir - we need to make that work:
+    // Set up viewfs link for wd as described above
+    String wdDir = fsTarget.getWorkingDirectory().toUri().getPath();
+    linkUpFirstComponents(conf, wdDir, fsTarget, "working dir");
     
     FileContext fc = FileContext.getFileContext(FsConstants.VIEWFS_URI, conf);
+    fc.setWorkingDirectory(new Path(wdDir)); // in case testdir relative to wd.
+    Log.info("Working dir is: " + fc.getWorkingDirectory());
     //System.out.println("SRCOfTests = "+ getTestRootPath(fc, "test"));
     //System.out.println("TargetOfTests = "+ targetOfTests.toUri());
     return fc;
@@ -85,5 +96,36 @@ public class ViewFsTestSetup {
     Path targetOfTests = FileContextTestHelper.getTestRootPath(fclocal);
     fclocal.delete(targetOfTests, true);
   }
+  
+  
+  static void setUpHomeDir(Configuration conf, FileContext fsTarget) {
+    String homeDir = fsTarget.getHomeDirectory().toUri().getPath();
+    int indexOf2ndSlash = homeDir.indexOf('/', 1);
+    if (indexOf2ndSlash >0) {
+      linkUpFirstComponents(conf, homeDir, fsTarget, "home dir");
+    } else { // home dir is at root. Just link the home dir itse
+      URI linkTarget = fsTarget.makeQualified(new Path(homeDir)).toUri();
+      ConfigUtil.addLink(conf, homeDir, linkTarget);
+      Log.info("Added link for home dir " + homeDir + "->" + linkTarget);
+    }
+    // Now set the root of the home dir for viewfs
+    String homeDirRoot = fsTarget.getHomeDirectory().getParent().toUri().getPath();
+    ConfigUtil.setHomeDirConf(conf, homeDirRoot);
+    Log.info("Home dir base for viewfs" + homeDirRoot);  
+  }
+  
+  /*
+   * Set up link in config for first component of path to the same
+   * in the target file system.
+   */
+  static void linkUpFirstComponents(Configuration conf, String path,
+      FileContext fsTarget, String info) {
+    int indexOf2ndSlash = path.indexOf('/', 1);
+    String firstComponent = path.substring(0, indexOf2ndSlash);
+    URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
+    ConfigUtil.addLink(conf, firstComponent, linkTarget);
+    Log.info("Added link for " + info + " " 
+        + firstComponent + "->" + linkTarget);    
+  }
 
 }

Modified: hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java Thu Nov  8 19:09:46 2012
@@ -224,7 +224,10 @@ public class TestNativeIO {
       // we should just skip the unit test on machines where we don't
       // have fadvise support
       assumeTrue(false);
-    } finally {
+    } catch (NativeIOException nioe) {
+      // ignore this error as FreeBSD returns EBADF even if length is zero
+    }
+      finally {
       fis.close();
     }
 

Modified: hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java Thu Nov  8 19:09:46 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/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java Thu Nov  8 19:09:46 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/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java Thu Nov  8 19:09:46 2012
@@ -36,6 +36,7 @@ import org.apache.hadoop.ipc.protobuf.Pr
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
 import org.apache.hadoop.net.NetUtils;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 /** Unit test for supporting method-name based compatible RPCs. */
@@ -114,6 +115,11 @@ public class TestRPCCompatibility {
     }
 
   }
+
+  @Before
+  public void setUp() {
+    ProtocolSignature.resetCache();
+  }
   
   @After
   public void tearDown() throws IOException {
@@ -219,7 +225,6 @@ System.out.println("echo int is NOT supp
   
   @Test // equal version client and server
   public void testVersion2ClientVersion2Server() throws Exception {
-    ProtocolSignature.resetCache();
     // create a server with two handlers
     TestImpl2 impl = new TestImpl2();
     server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)

Modified: hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java Thu Nov  8 19:09:46 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,15 +43,7 @@ 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;
-import org.apache.hadoop.security.SaslRpcServer;
-import org.apache.hadoop.security.SecurityInfo;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.TestUserGroupInformation;
-import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.*;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
@@ -57,7 +51,10 @@ import org.apache.hadoop.security.token.
 import org.apache.hadoop.security.token.TokenInfo;
 import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+
 import org.apache.log4j.Level;
+import org.apache.tools.ant.types.Assertions.EnabledAssertion;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -73,13 +70,21 @@ public class TestSaslRPC {
   static final String SERVER_KEYTAB_KEY = "test.ipc.server.keytab";
   static final String SERVER_PRINCIPAL_1 = "p1/foo@BAR";
   static final String SERVER_PRINCIPAL_2 = "p2/foo@BAR";
-  
   private static Configuration conf;
+  static Boolean forceSecretManager = null;
+  
   @BeforeClass
-  public static void setup() {
+  public static void setupKerb() {
+    System.setProperty("java.security.krb5.kdc", "");
+    System.setProperty("java.security.krb5.realm", "NONE"); 
+  }    
+
+  @Before
+  public void setup() {
     conf = new Configuration();
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
+    forceSecretManager = null;
   }
 
   static {
@@ -186,6 +191,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 +200,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 {
@@ -259,16 +269,6 @@ public class TestSaslRPC {
   }
 
   @Test
-  public void testSecureToInsecureRpc() throws Exception {
-    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);
-  }
-  
-  @Test
   public void testErrorMessage() throws Exception {
     BadTokenSecretManager sm = new BadTokenSecretManager();
     final Server server = new RPC.Builder(conf)
@@ -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);
@@ -448,127 +448,176 @@ 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 .*");
+  private static Pattern NoDigest =
+      Pattern.compile(".*Server is not configured to do DIGEST auth.*");
+  
+  /*
+   *  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));
+    forceSecretManager = true;
+    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));
+    forceSecretManager = true;
+    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));
+    // can't fallback to simple when using kerberos w/o tokens
+    forceSecretManager = false;
+    assertAuthEquals(NoDigest, getAuthMethod(SIMPLE,   KERBEROS, true));
+    assertAuthEquals(NoDigest, 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));
+    forceSecretManager = false;
+    assertAuthEquals(NoDigest, getAuthMethod(SIMPLE,   KERBEROS, true));
+    assertAuthEquals(NoDigest, getAuthMethod(KERBEROS, KERBEROS, true));
   }
 
-  // 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 {
-    TestTokenSecretManager sm = new TestTokenSecretManager();
-    Server server = new RPC.Builder(conf).setProtocol(TestSaslProtocol.class)
-        .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(5).setVerbose(true).setSecretManager(sm).build();      
-    if (isSecureServer) {
-      server.enableSecurity();
-    } else {
-      server.disableSecurity();
+  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();
     }
-    server.start();
+  }
+  
+  private String internalGetAuthMethod(
+      final AuthenticationMethod clientAuth,
+      final AuthenticationMethod serverAuth,
+      final boolean useToken,
+      final boolean useValidToken) throws Exception {
+    
+    String currentUser = UserGroupInformation.getCurrentUser().getUserName();
+    
+    final Configuration serverConf = new Configuration(conf);
+    SecurityUtil.setAuthenticationMethod(serverAuth, serverConf);
+    UserGroupInformation.setConfiguration(serverConf);
+    
+    final UserGroupInformation serverUgi =
+        UserGroupInformation.createRemoteUser(currentUser + "-SERVER");
+    serverUgi.setAuthenticationMethod(serverAuth);
+
+    final TestTokenSecretManager sm = new TestTokenSecretManager();
+    boolean useSecretManager = (serverAuth != SIMPLE);
+    if (forceSecretManager != null) {
+      useSecretManager &= forceSecretManager.booleanValue();
+    }
+    final SecretManager<?> serverSm = useSecretManager ? sm : null;
+    
+    Server server = serverUgi.doAs(new PrivilegedExceptionAction<Server>() {
+      @Override
+      public Server run() throws IOException {
+        Server server = new RPC.Builder(serverConf)
+        .setProtocol(TestSaslProtocol.class)
+        .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0)
+        .setNumHandlers(5).setVerbose(true)
+        .setSecretManager(serverSm)
+        .build();      
+        server.start();
+        return server;
+      }
+    });
+
+    final Configuration clientConf = new Configuration(conf);
+    SecurityUtil.setAuthenticationMethod(clientAuth, clientConf);
+    UserGroupInformation.setConfiguration(clientConf);
+    
+    final UserGroupInformation clientUgi =
+        UserGroupInformation.createRemoteUser(currentUser + "-CLIENT");
+    clientUgi.setAuthenticationMethod(clientAuth);    
 
-    final UserGroupInformation current = UserGroupInformation.getCurrentUser();
     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);
     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);
+            
+            proxy.ping();
+            // verify sasl completed
+            if (serverAuth != SIMPLE) {
+              assertEquals(SaslRpcServer.SASL_PROPS.get(Sasl.QOP), "auth");
+            }
+            
+            // 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 +629,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 +657,4 @@ public class TestSaslRPC {
     String keytab = args[1];
     testKerberosRpc(principal, keytab);
   }
-
 }

Modified: hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java Thu Nov  8 19:09:46 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/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java Thu Nov  8 19:09:46 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/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java Thu Nov  8 19:09:46 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/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithSecurityOn.java Thu Nov  8 19:09:46 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/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java Thu Nov  8 19:09:46 2012
@@ -324,7 +324,6 @@ public class TestUserGroupInformation {
     assertSame(secret, ugi.getCredentials().getSecretKey(secretKey));
   }
 
-  @SuppressWarnings("unchecked") // from Mockito mocks
   @Test
   public <T extends TokenIdentifier> void testGetCredsNotSame()
       throws Exception {
@@ -449,6 +448,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/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/proto/test.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/proto/test.proto?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/proto/test.proto (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/proto/test.proto Thu Nov  8 19:09:46 2012
@@ -19,6 +19,7 @@
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_outer_classname = "TestProtos";
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 message EmptyRequestProto {
 }

Modified: hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto Thu Nov  8 19:09:46 2012
@@ -19,6 +19,7 @@ option java_package = "org.apache.hadoop
 option java_outer_classname = "TestRpcServiceProtos";
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 import "test.proto";
 

Modified: hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml?rev=1407217&r1=1407216&r2=1407217&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml Thu Nov  8 19:09:46 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>